diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 24f28b10c1..1bd3e49f40 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -469,6 +469,7 @@ set(DORIS_LINK_LIBS DorisGen Webserver Geo + Vec Plugin ${WL_END_GROUP} ) @@ -653,6 +654,7 @@ endif() add_subdirectory(${SRC_DIR}/util) add_subdirectory(${SRC_DIR}/plugin) +add_subdirectory(${SRC_DIR}/vec) # Utility CMake function to make specifying tests and benchmarks less verbose FUNCTION(ADD_BE_TEST TEST_NAME) @@ -705,6 +707,11 @@ if (${MAKE_TEST} STREQUAL "ON") add_subdirectory(${TEST_DIR}/runtime) add_subdirectory(${TEST_DIR}/udf) add_subdirectory(${TEST_DIR}/util) + add_subdirectory(${TEST_DIR}/vec/core) + add_subdirectory(${TEST_DIR}/vec/exprs) + add_subdirectory(${TEST_DIR}/vec/function) + add_subdirectory(${TEST_DIR}/vec/runtime) + add_subdirectory(${TEST_DIR}/vec/aggregate_functions) add_subdirectory(${TEST_DIR}/plugin) add_subdirectory(${TEST_DIR}/plugin/example) add_subdirectory(${TEST_DIR}/tools) @@ -728,3 +735,8 @@ install(FILES ${BASE_DIR}/../conf/odbcinst.ini DESTINATION ${OUTPUT_DIR}/conf) + +get_property(dirs DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES) +foreach(dir ${dirs}) + message(STATUS "dir='${dir}'") + endforeach() diff --git a/be/src/exec/blocking_join_node.cpp b/be/src/exec/blocking_join_node.cpp index da977016fb..ba137860ac 100644 --- a/be/src/exec/blocking_join_node.cpp +++ b/be/src/exec/blocking_join_node.cpp @@ -30,7 +30,8 @@ namespace doris { BlockingJoinNode::BlockingJoinNode(const std::string& node_name, const TJoinOp::type join_op, ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : ExecNode(pool, tnode, descs), _node_name(node_name), _join_op(join_op) {} + : ExecNode(pool, tnode, descs), _node_name(node_name), _join_op(join_op), + _left_side_eos(false) {} Status BlockingJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { return ExecNode::init(tnode, state); diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp index aefa087f45..cf06f09f24 100644 --- a/be/src/exec/data_sink.cpp +++ b/be/src/exec/data_sink.cpp @@ -35,7 +35,10 @@ #include "runtime/result_file_sink.h" #include "runtime/result_sink.h" #include "runtime/runtime_state.h" -#include "util/logging.h" + +#include "vec/sink/result_sink.h" +#include "vec/sink/vdata_stream_sender.h" +#include "vec/sink/vtablet_sink.h" namespace doris { @@ -57,6 +60,9 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink : false; // TODO: figure out good buffer size based on size of output row if (is_vec) { + tmp_sink = new doris::vectorized::VDataStreamSender( + pool, params.sender_id, row_desc, thrift_sink.stream_sink, params.destinations, + 16 * 1024, send_query_statistics_with_every_batch); } else { tmp_sink = new DataStreamSender(pool, params.sender_id, row_desc, thrift_sink.stream_sink, params.destinations, 16 * 1024, @@ -73,6 +79,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink // TODO: figure out good buffer size based on size of output row if (is_vec) { + tmp_sink = new doris::vectorized::VResultSink(row_desc, output_exprs, thrift_sink.result_sink, 4096); } else { tmp_sink = new ResultSink(row_desc, output_exprs, thrift_sink.result_sink, 1024); } @@ -149,7 +156,11 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink case TDataSinkType::OLAP_TABLE_SINK: { Status status; DCHECK(thrift_sink.__isset.olap_table_sink); - sink->reset(new stream_load::OlapTableSink(pool, row_desc, output_exprs, &status)); + if (is_vec) { + sink->reset(new stream_load::VOlapTableSink(pool, row_desc, output_exprs, &status)); + } else { + sink->reset(new stream_load::OlapTableSink(pool, row_desc, output_exprs, &status)); + } RETURN_IF_ERROR(status); break; } diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h index 30f5580934..fcec10aed8 100644 --- a/be/src/exec/data_sink.h +++ b/be/src/exec/data_sink.h @@ -33,11 +33,13 @@ class ObjectPool; class RowBatch; class RuntimeProfile; class RuntimeState; -class TPlanExecRequest; -class TPlanExecParams; class TPlanFragmentExecParams; class RowDescriptor; +namespace vectorized { +class Block; +} + // Superclass of all data sinks. class DataSink { public: @@ -56,8 +58,11 @@ public: // Send a row batch into this sink. // eos should be true when the last batch is passed to send() virtual Status send(RuntimeState* state, RowBatch* batch) = 0; - // virtual Status send(RuntimeState* state, RowBatch* batch, bool eos) = 0; + // Send a Block into this sink. + virtual Status send(RuntimeState* state, vectorized::Block* block) { + return Status::NotSupported("Not support send block"); + }; // Releases all resources that were allocated in prepare()/send(). // Further send() calls are illegal after calling close(). // It must be okay to call this multiple times. Subsequent calls should diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp index 396c0869d6..86cd16a934 100644 --- a/be/src/exec/es/es_scroll_parser.cpp +++ b/be/src/exec/es/es_scroll_parser.cpp @@ -31,6 +31,7 @@ #include "runtime/mem_pool.h" #include "runtime/mem_tracker.h" #include "util/string_parser.hpp" +#include "vec/runtime/vdatetime_value.h" namespace doris { @@ -79,6 +80,8 @@ static const std::string ERROR_MEM_LIMIT_EXCEEDED = static const std::string ERROR_COL_DATA_IS_ARRAY = "Data source returned an array for the type $0" "based on column metadata."; +static const std::string INVALID_NULL_VALUE = + "Invalid null value occurs: Non-null column `$0` contains NULL"; #define RETURN_ERROR_IF_COL_IS_ARRAY(col, type) \ do { \ @@ -169,7 +172,7 @@ static Status get_int_value(const rapidjson::Value& col, PrimitiveType type, voi template static Status get_float_value(const rapidjson::Value& col, PrimitiveType type, void* slot, bool pure_doc_value) { - DCHECK(sizeof(T) == 4 || sizeof(T) == 8); + static_assert(sizeof(T) == 4 || sizeof(T) == 8); if (col.IsNumber()) { *reinterpret_cast(slot) = (T)(sizeof(T) == 4 ? col.GetFloat() : col.GetDouble()); return Status::OK(); @@ -193,6 +196,68 @@ static Status get_float_value(const rapidjson::Value& col, PrimitiveType type, v return Status::OK(); } +template +static Status insert_float_value(const rapidjson::Value& col, PrimitiveType type, + vectorized::IColumn* col_ptr, bool pure_doc_value, + bool nullable) { + static_assert(sizeof(T) == 4 || sizeof(T) == 8); + if (col.IsNumber() && nullable) { + T value = (T)(sizeof(T) == 4 ? col.GetFloat() : col.GetDouble()); + col_ptr->insert_data(const_cast(reinterpret_cast(&value)), 0); + return Status::OK(); + } + + if (pure_doc_value && col.IsArray() && nullable) { + T value = (T)(sizeof(T) == 4 ? col[0].GetFloat() : col[0].GetDouble()); + col_ptr->insert_data(const_cast(reinterpret_cast(&value)), 0); + return Status::OK(); + } + + RETURN_ERROR_IF_COL_IS_ARRAY(col, type); + RETURN_ERROR_IF_COL_IS_NOT_STRING(col, type); + + StringParser::ParseResult result; + const std::string& val = col.GetString(); + size_t len = col.GetStringLength(); + T v = StringParser::string_to_float(val.c_str(), len, &result); + RETURN_ERROR_IF_PARSING_FAILED(result, col, type); + + col_ptr->insert_data(const_cast(reinterpret_cast(&v)), 0); + + return Status::OK(); +} + +template +static Status insert_int_value(const rapidjson::Value& col, PrimitiveType type, + vectorized::IColumn* col_ptr, bool pure_doc_value, + bool nullable) { + if (col.IsNumber()) { + T value = (T)(sizeof(T) < 8 ? col.GetInt() : col.GetInt64()); + col_ptr->insert_data(const_cast(reinterpret_cast(&value)), 0); + return Status::OK(); + } + + if (pure_doc_value && col.IsArray()) { + RETURN_ERROR_IF_COL_IS_NOT_NUMBER(col[0], type); + T value = (T)(sizeof(T) < 8 ? col[0].GetInt() : col[0].GetInt64()); + col_ptr->insert_data(const_cast(reinterpret_cast(&value)), 0); + return Status::OK(); + } + + RETURN_ERROR_IF_COL_IS_ARRAY(col, type); + RETURN_ERROR_IF_COL_IS_NOT_STRING(col, type); + + StringParser::ParseResult result; + const std::string& val = col.GetString(); + size_t len = col.GetStringLength(); + T v = StringParser::string_to_int(val.c_str(), len, &result); + RETURN_ERROR_IF_PARSING_FAILED(result, col, type); + + col_ptr->insert_data(const_cast(reinterpret_cast(&v)), 0); + + return Status::OK(); +} + ScrollParser::ScrollParser(bool doc_value_mode) : _scroll_id(""), _size(0), _line_index(0), _doc_value_mode(doc_value_mode) {} @@ -426,22 +491,53 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple, *reinterpret_cast(slot) = col.GetInt(); break; } - if (pure_doc_value && col.IsArray()) { + + bool is_nested_str = false; + if (pure_doc_value && col.IsArray() && col[0].IsBool()) { *reinterpret_cast(slot) = col[0].GetBool(); break; + } else if (pure_doc_value && col.IsArray() && col[0].IsString()) { + is_nested_str = true; + } else if (pure_doc_value && col.IsArray()) { + return Status::InternalError( + strings::Substitute(ERROR_INVALID_COL_DATA, "BOOLEAN")); } - RETURN_ERROR_IF_COL_IS_ARRAY(col, type); - RETURN_ERROR_IF_COL_IS_NOT_STRING(col, type); - - const std::string& val = col.GetString(); - size_t val_size = col.GetStringLength(); + const rapidjson::Value& str_col = is_nested_str? col[0]: col; + const std::string& val = str_col.GetString(); + size_t val_size = str_col.GetStringLength(); StringParser::ParseResult result; bool b = StringParser::string_to_bool(val.c_str(), val_size, &result); - RETURN_ERROR_IF_PARSING_FAILED(result, col, type); + RETURN_ERROR_IF_PARSING_FAILED(result, str_col, type); *reinterpret_cast(slot) = b; break; } + case TYPE_DECIMALV2: { + DecimalV2Value data; + + if (col.IsDouble()) { + data.assign_from_double(col.GetDouble()); + } else { + std::string val; + if (pure_doc_value) { + if (!col[0].IsString()) { + val = json_value_to_string(col[0]); + } else { + val = col[0].GetString(); + } + } else { + RETURN_ERROR_IF_COL_IS_ARRAY(col, type); + if (!col.IsString()) { + val = json_value_to_string(col); + } else { + val = col.GetString(); + } + } + data.parse_from_str(val.data(), val.length()); + } + reinterpret_cast(slot)->set_value(data.value()); + break; + } case TYPE_DATE: case TYPE_DATETIME: { @@ -482,6 +578,235 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple, return Status::OK(); } +Status ScrollParser::fill_columns(const TupleDescriptor* tuple_desc, + std::vector& columns, + MemPool* tuple_pool, bool* line_eof, + const std::map& docvalue_context) { + *line_eof = true; + + if (_size <= 0 || _line_index >= _size) { + return Status::OK(); + } + + const rapidjson::Value& obj = _inner_hits_node[_line_index++]; + bool pure_doc_value = false; + if (obj.HasMember("fields")) { + pure_doc_value = true; + } + const rapidjson::Value& line = obj.HasMember(FIELD_SOURCE) ? obj[FIELD_SOURCE] : obj["fields"]; + + for (int i = 0; i < tuple_desc->slots().size(); ++i) { + const SlotDescriptor* slot_desc = tuple_desc->slots()[i]; + auto col_ptr = columns[i].get(); + + if (!slot_desc->is_materialized()) { + continue; + } + if (slot_desc->col_name() == FIELD_ID) { + // actually this branch will not be reached, this is guaranteed by Doris FE. + if (pure_doc_value) { + std::stringstream ss; + ss << "obtain `_id` is not supported in doc_values mode"; + return Status::RuntimeError(ss.str()); + } + // obj[FIELD_ID] must not be NULL + std::string _id = obj[FIELD_ID].GetString(); + size_t len = _id.length(); + + col_ptr->insert_data(const_cast(_id.data()), len); + continue; + } + + const char* col_name = pure_doc_value ? docvalue_context.at(slot_desc->col_name()).c_str() + : slot_desc->col_name().c_str(); + + rapidjson::Value::ConstMemberIterator itr = line.FindMember(col_name); + if (itr == line.MemberEnd() && slot_desc->is_nullable()) { + auto nullable_column = reinterpret_cast(col_ptr); + nullable_column->insert_data(nullptr, 0); + continue; + } else if (itr == line.MemberEnd() && !slot_desc->is_nullable()) { + std::string details = strings::Substitute(INVALID_NULL_VALUE, col_name); + return Status::RuntimeError(details); + } + + const rapidjson::Value& col = line[col_name]; + + PrimitiveType type = slot_desc->type().type; + + // when the column value is null, the subsequent type casting will report an error + if (col.IsNull() && slot_desc->is_nullable()) { + col_ptr->insert_data(nullptr, 0); + continue; + } else if (col.IsNull() && !slot_desc->is_nullable()) { + std::string details = strings::Substitute(INVALID_NULL_VALUE, col_name); + return Status::RuntimeError(details); + } + switch (type) { + case TYPE_CHAR: + case TYPE_VARCHAR: + case TYPE_STRING: { + // sometimes elasticsearch user post some not-string value to Elasticsearch Index. + // because of reading value from _source, we can not process all json type and then just transfer the value to original string representation + // this may be a tricky, but we can workaround this issue + std::string val; + if (pure_doc_value) { + if (!col[0].IsString()) { + val = json_value_to_string(col[0]); + } else { + val = col[0].GetString(); + } + } else { + RETURN_ERROR_IF_COL_IS_ARRAY(col, type); + if (!col.IsString()) { + val = json_value_to_string(col); + } else { + val = col.GetString(); + } + } + size_t val_size = val.length(); + col_ptr->insert_data( + const_cast(val.data()), val_size); + break; + } + + case TYPE_TINYINT: { + insert_int_value(col, type, col_ptr, pure_doc_value, slot_desc->is_nullable()); + break; + } + + case TYPE_SMALLINT: { + insert_int_value(col, type, col_ptr, pure_doc_value, slot_desc->is_nullable()); + break; + } + + case TYPE_INT: { + insert_int_value(col, type, col_ptr, pure_doc_value, slot_desc->is_nullable()); + break; + } + + case TYPE_BIGINT: { + insert_int_value(col, type, col_ptr, pure_doc_value, slot_desc->is_nullable()); + break; + } + + case TYPE_LARGEINT: { + insert_int_value<__int128>(col, type, col_ptr, pure_doc_value, slot_desc->is_nullable()); + break; + } + + case TYPE_DOUBLE: { + insert_float_value(col, type, col_ptr, pure_doc_value, slot_desc->is_nullable()); + break; + } + + case TYPE_FLOAT: { + insert_float_value(col, type, col_ptr, pure_doc_value, slot_desc->is_nullable()); + break; + } + + case TYPE_BOOLEAN: { + if (col.IsBool()) { + int8_t val = col.GetBool(); + col_ptr->insert_data(const_cast(reinterpret_cast(&val)), 0); + break; + } + + if (col.IsNumber()) { + int8_t val = col.GetInt(); + col_ptr->insert_data(const_cast(reinterpret_cast(&val)), 0); + break; + } + + bool is_nested_str = false; + if (pure_doc_value && col.IsArray() && col[0].IsBool()) { + int8_t val = col[0].GetBool(); + col_ptr->insert_data(const_cast(reinterpret_cast(&val)), 0); + break; + } else if (pure_doc_value && col.IsArray() && col[0].IsString()) { + is_nested_str = true; + } else if (pure_doc_value && col.IsArray()) { + return Status::InternalError( + strings::Substitute(ERROR_INVALID_COL_DATA, "BOOLEAN")); + } + + const rapidjson::Value& str_col = is_nested_str? col[0]: col; + + const std::string& val = str_col.GetString(); + size_t val_size = str_col.GetStringLength(); + StringParser::ParseResult result; + bool b = StringParser::string_to_bool(val.c_str(), val_size, &result); + RETURN_ERROR_IF_PARSING_FAILED(result, str_col, type); + col_ptr->insert_data(const_cast(reinterpret_cast(&b)), 0); + break; + } + case TYPE_DECIMALV2: { + DecimalV2Value data; + + if (col.IsDouble()) { + data.assign_from_double(col.GetDouble()); + } else { + std::string val; + if (pure_doc_value) { + if (!col[0].IsString()) { + val = json_value_to_string(col[0]); + } else { + val = col[0].GetString(); + } + } else { + RETURN_ERROR_IF_COL_IS_ARRAY(col, type); + if (!col.IsString()) { + val = json_value_to_string(col); + } else { + val = col.GetString(); + } + } + data.parse_from_str(val.data(), val.length()); + } + col_ptr->insert_data( + const_cast(reinterpret_cast(&data)), 0); + break; + } + + case TYPE_DATE: + case TYPE_DATETIME: { + // this would happend just only when `enable_docvalue_scan = false`, and field has timestamp format date from _source + if (col.IsNumber()) { + // ES process date/datetime field would use millisecond timestamp for index or docvalue + // processing date type field, if a number is encountered, Doris On ES will force it to be processed according to ms + // Doris On ES needs to be consistent with ES, so just divided by 1000 because the unit for from_unixtime is seconds + RETURN_IF_ERROR(fill_date_col_with_timestamp(col_ptr, col, type)); + } else if (col.IsArray() && pure_doc_value) { + // this would happened just only when `enable_docvalue_scan = true` + // ES add default format for all field after ES 6.4, if we not provided format for `date` field ES would impose + // a standard date-format for date field as `2020-06-16T00:00:00.000Z` + // At present, we just process this string format date. After some PR were merged into Doris, we would impose `epoch_mills` for + // date field's docvalue + if (col[0].IsString()) { + RETURN_IF_ERROR(fill_date_col_with_strval(col_ptr, col[0], type)); + break; + } + // ES would return millisecond timestamp for date field, divided by 1000 because the unit for from_unixtime is seconds + RETURN_IF_ERROR(fill_date_col_with_timestamp(col_ptr, col, type)); + } else { + // this would happened just only when `enable_docvalue_scan = false`, and field has string format date from _source + RETURN_ERROR_IF_COL_IS_ARRAY(col, type); + RETURN_ERROR_IF_COL_IS_NOT_STRING(col, type); + RETURN_IF_ERROR(fill_date_col_with_strval(col_ptr, col, type)); + } + break; + } + default: { + DCHECK(false); + break; + } + } + } + + *line_eof = false; + return Status::OK(); +} + Status ScrollParser::fill_date_slot_with_strval(void* slot, const rapidjson::Value& col, PrimitiveType type) { DateTimeValue* ts_slot = reinterpret_cast(slot); @@ -511,4 +836,44 @@ Status ScrollParser::fill_date_slot_with_timestamp(void* slot, const rapidjson:: return Status::OK(); } +Status ScrollParser::fill_date_col_with_strval(vectorized::IColumn* col_ptr, + const rapidjson::Value& col, PrimitiveType type) { + vectorized::VecDateTimeValue dt_val; + const std::string& val = col.GetString(); + size_t val_size = col.GetStringLength(); + if (!dt_val.from_date_str(val.c_str(), val_size)) { + RETURN_ERROR_IF_CAST_FORMAT_ERROR(col, type); + } + if (type == TYPE_DATE) { + dt_val.cast_to_date(); + } else { + dt_val.to_datetime(); + } + + auto date_packed_int = binary_cast( + *reinterpret_cast(&dt_val)); + col_ptr->insert_data(const_cast(reinterpret_cast(&date_packed_int)), 0); + return Status::OK(); +} + +Status ScrollParser::fill_date_col_with_timestamp(vectorized::IColumn* col_ptr, + const rapidjson::Value& col, PrimitiveType type) { + vectorized::VecDateTimeValue dt_val; + if (!dt_val.from_unixtime(col.GetInt64() / 1000, "+08:00")) { + RETURN_ERROR_IF_CAST_FORMAT_ERROR(col, type); + } + if (type == TYPE_DATE) { + reinterpret_cast(&dt_val)->cast_to_date(); + } else { + reinterpret_cast(&dt_val)->set_type(TIME_DATETIME); + } + + auto date_packed_int = binary_cast( + *reinterpret_cast(&dt_val)); + col_ptr->insert_data(const_cast(reinterpret_cast(&date_packed_int)), 0); + + return Status::OK(); +} + + } // namespace doris diff --git a/be/src/exec/es/es_scroll_parser.h b/be/src/exec/es/es_scroll_parser.h index f2ff855a7c..30eabaff2c 100644 --- a/be/src/exec/es/es_scroll_parser.h +++ b/be/src/exec/es/es_scroll_parser.h @@ -22,6 +22,7 @@ #include "rapidjson/document.h" #include "runtime/descriptors.h" #include "runtime/tuple.h" +#include "vec/core/block.h" namespace doris { @@ -35,6 +36,9 @@ public: Status parse(const std::string& scroll_result, bool exactly_once = false); Status fill_tuple(const TupleDescriptor* _tuple_desc, Tuple* tuple, MemPool* mem_pool, bool* line_eof, const std::map& docvalue_context); + Status fill_columns(const TupleDescriptor* _tuple_desc, + std::vector& columns, MemPool* mem_pool, + bool* line_eof, const std::map& docvalue_context); const std::string& get_scroll_id(); int get_size(); @@ -44,9 +48,13 @@ private: // type is used for distinguish date and datetime // fill date slot with string format date Status fill_date_slot_with_strval(void* slot, const rapidjson::Value& col, PrimitiveType type); + Status fill_date_col_with_strval(vectorized::IColumn* col_ptr, const rapidjson::Value& col, + PrimitiveType type); // fill date slot with timestamp Status fill_date_slot_with_timestamp(void* slot, const rapidjson::Value& col, PrimitiveType type); + Status fill_date_col_with_timestamp(vectorized::IColumn* col_ptr, const rapidjson::Value& col, + PrimitiveType type); private: std::string _scroll_id; diff --git a/be/src/exec/es_http_scan_node.cpp b/be/src/exec/es_http_scan_node.cpp index 012110d80e..7b67486401 100644 --- a/be/src/exec/es_http_scan_node.cpp +++ b/be/src/exec/es_http_scan_node.cpp @@ -437,10 +437,17 @@ void EsHttpScanNode::scanner_worker(int start_idx, int length, std::promise scanner( - new EsHttpScanner(_runtime_state, runtime_profile(), _tuple_id, properties, - scanner_expr_ctxs, &counter, doc_value_mode)); - status = scanner_scan(std::move(scanner), scanner_expr_ctxs, &counter); + if (!_vectorized) { + std::unique_ptr scanner( + new EsHttpScanner(_runtime_state, runtime_profile(), _tuple_id, properties, + scanner_expr_ctxs, &counter, doc_value_mode)); + status = scanner_scan(std::move(scanner), scanner_expr_ctxs, &counter); + } else { + std::unique_ptr scanner( + new VEsHttpScanner(_runtime_state, runtime_profile(), _tuple_id, properties, + scanner_expr_ctxs, &counter, doc_value_mode)); + status = scanner_scan(std::move(scanner)); + } if (!status.ok()) { LOG(WARNING) << "Scanner[" << start_idx << "] process failed. status=" << status.get_error_msg(); diff --git a/be/src/exec/es_http_scan_node.h b/be/src/exec/es_http_scan_node.h index 640ea0c576..b90bf0e466 100644 --- a/be/src/exec/es_http_scan_node.h +++ b/be/src/exec/es_http_scan_node.h @@ -29,6 +29,7 @@ #include "common/status.h" #include "exec/es_http_scanner.h" +#include "vec/exec/ves_http_scanner.h" #include "exec/scan_node.h" #include "gen_cpp/PaloInternalService_types.h" @@ -56,7 +57,6 @@ protected: // Write debug string of this into out. virtual void debug_string(int indentation_level, std::stringstream* out) const override; -private: // Update process status to one failed status, // NOTE: Must hold the mutex of this scan node bool update_status(const Status& new_status) { @@ -67,20 +67,8 @@ private: return false; } - // Create scanners to do scan job - Status start_scanners(); - - // Collect all scanners 's status - Status collect_scanners_status(); - // One scanner worker, This scanner will handle 'length' ranges start from start_idx - void scanner_worker(int start_idx, int length, std::promise& p_status); - - // Scan one range - Status scanner_scan(std::unique_ptr scanner, - const std::vector& conjunct_ctxs, EsScanCounter* counter); - - Status build_conjuncts_list(); + virtual void scanner_worker(int start_idx, int length, std::promise& p_status); TupleId _tuple_id; RuntimeState* _runtime_state; @@ -92,20 +80,41 @@ private: int _max_buffered_batches; RuntimeProfile::Counter* _wait_scanner_timer; - bool _all_scanners_finished; Status _process_status; + std::map _docvalue_context; + + std::condition_variable _queue_reader_cond; + std::condition_variable _queue_writer_cond; + bool _vectorized = false; + +private: + // Create scanners to do scan job + Status start_scanners(); + + // Collect all scanners 's status + Status collect_scanners_status(); + + // Scan one range + Status scanner_scan(std::unique_ptr scanner, + const std::vector& conjunct_ctxs, EsScanCounter* counter); + + virtual Status scanner_scan(std::unique_ptr scanner) { + return Status::NotSupported("vectorized scan in EsHttpScanNode is not supported!"); + }; + + Status build_conjuncts_list(); + + bool _all_scanners_finished; + std::vector _scanner_threads; std::vector> _scanners_status; std::map _properties; - std::map _docvalue_context; std::map _fields_context; std::vector _scan_ranges; std::vector _column_names; std::mutex _batch_queue_lock; - std::condition_variable _queue_reader_cond; - std::condition_variable _queue_writer_cond; std::deque> _batch_queue; std::vector _predicates; diff --git a/be/src/exec/es_http_scanner.h b/be/src/exec/es_http_scanner.h index 2bade5ae3f..dcebfe1649 100644 --- a/be/src/exec/es_http_scanner.h +++ b/be/src/exec/es_http_scanner.h @@ -68,7 +68,7 @@ public: void close(); -private: +protected: RuntimeState* _state; RuntimeProfile* _profile; TupleId _tuple_id; diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index d692a79772..97c3259327 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -63,6 +63,26 @@ #include "util/debug_util.h" #include "util/runtime_profile.h" +#include "vec/core/block.h" +#include "vec/exec/join/vhash_join_node.h" +#include "vec/exec/vaggregation_node.h" +#include "vec/exec/ves_http_scan_node.h" +#include "vec/exec/vcross_join_node.h" +#include "vec/exec/vexchange_node.h" +#include "vec/exec/vmysql_scan_node.h" +#include "vec/exec/vodbc_scan_node.h" +#include "vec/exec/volap_scan_node.h" +#include "vec/exec/vsort_node.h" +#include "vec/exec/vunion_node.h" +#include "vec/exec/vintersect_node.h" +#include "vec/exec/vexcept_node.h" +#include "vec/exec/vanalytic_eval_node.h" +#include "vec/exec/vassert_num_rows_node.h" +#include "vec/exec/vselect_node.h" +#include "vec/exprs/vexpr.h" +#include "vec/exec/vempty_set_node.h" +#include "vec/exec/vschema_scan_node.h" +#include "vec/exec/vrepeat_node.h" namespace doris { const std::string ExecNode::ROW_THROUGHPUT_COUNTER = "RowsReturnedRate"; @@ -167,6 +187,9 @@ Status ExecNode::init(const TPlanNode& tnode, RuntimeState* state) { init_runtime_profile(profile); if (tnode.__isset.vconjunct) { + _vconjunct_ctx_ptr.reset(new doris::vectorized::VExprContext*); + RETURN_IF_ERROR(doris::vectorized::VExpr::create_expr_tree(_pool, tnode.vconjunct, + _vconjunct_ctx_ptr.get())); } RETURN_IF_ERROR(Expr::create_expr_trees(_pool, tnode.conjuncts, &_conjunct_ctxs)); @@ -189,6 +212,9 @@ Status ExecNode::prepare(RuntimeState* state) { _mem_tracker); _expr_mem_pool.reset(new MemPool(_expr_mem_tracker.get())); + if (_vconjunct_ctx_ptr) { + RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->prepare(state, row_desc(), expr_mem_tracker())); + } RETURN_IF_ERROR(Expr::prepare(_conjunct_ctxs, state, row_desc(), expr_mem_tracker())); // TODO(zc): @@ -202,6 +228,9 @@ Status ExecNode::prepare(RuntimeState* state) { Status ExecNode::open(RuntimeState* state) { RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::OPEN)); + if (_vconjunct_ctx_ptr) { + RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->open(state)); + } return Expr::open(_conjunct_ctxs, state); } @@ -240,6 +269,7 @@ Status ExecNode::close(RuntimeState* state) { } } + if (_vconjunct_ctx_ptr) (*_vconjunct_ctx_ptr)->close(state); Expr::close(_conjunct_ctxs, state); if (expr_mem_pool() != nullptr) { @@ -339,12 +369,42 @@ Status ExecNode::create_tree_helper(RuntimeState* state, ObjectPool* pool, Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, ExecNode** node) { - if (state->enable_vectorized_exec()) { - return Status::InternalError("unsupport enable_vectorized_engine"); - } - std::stringstream error_msg; + if (state->enable_vectorized_exec()) { + switch (tnode.node_type) { + case TPlanNodeType::OLAP_SCAN_NODE: + case TPlanNodeType::ASSERT_NUM_ROWS_NODE: + case TPlanNodeType::HASH_JOIN_NODE: + case TPlanNodeType::AGGREGATION_NODE: + case TPlanNodeType::UNION_NODE: + case TPlanNodeType::CROSS_JOIN_NODE: + case TPlanNodeType::SORT_NODE: + case TPlanNodeType::EXCHANGE_NODE: + case TPlanNodeType::ODBC_SCAN_NODE: + case TPlanNodeType::MYSQL_SCAN_NODE: + case TPlanNodeType::INTERSECT_NODE: + case TPlanNodeType::EXCEPT_NODE: + case TPlanNodeType::ES_HTTP_SCAN_NODE: + case TPlanNodeType::EMPTY_SET_NODE: + case TPlanNodeType::SCHEMA_SCAN_NODE: + case TPlanNodeType::ANALYTIC_EVAL_NODE: + case TPlanNodeType::SELECT_NODE: + case TPlanNodeType::REPEAT_NODE: + break; + default: { + const auto& i = _TPlanNodeType_VALUES_TO_NAMES.find(tnode.node_type); + const char* str = "unknown node type"; + + if (i != _TPlanNodeType_VALUES_TO_NAMES.end()) { + str = i->second; + } + error_msg << "V" << str << " not implemented"; + return Status::InternalError(error_msg.str()); + } + } + } + VLOG_CRITICAL << "tnode:\n" << apache::thrift::ThriftDebugString(tnode); switch (tnode.node_type) { case TPlanNodeType::CSV_SCAN_NODE: @@ -353,14 +413,20 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN case TPlanNodeType::MYSQL_SCAN_NODE: #ifdef DORIS_WITH_MYSQL - *node = pool->add(new MysqlScanNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VMysqlScanNode(pool, tnode, descs)); + } else + *node = pool->add(new MysqlScanNode(pool, tnode, descs)); return Status::OK(); #else return Status::InternalError( "Don't support MySQL table, you should rebuild Doris with WITH_MYSQL option ON"); #endif case TPlanNodeType::ODBC_SCAN_NODE: - *node = pool->add(new OdbcScanNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VOdbcScanNode(pool, tnode, descs)); + } else + *node = pool->add(new OdbcScanNode(pool, tnode, descs)); return Status::OK(); case TPlanNodeType::ES_SCAN_NODE: @@ -368,15 +434,24 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN return Status::OK(); case TPlanNodeType::ES_HTTP_SCAN_NODE: - *node = pool->add(new EsHttpScanNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VEsHttpScanNode(pool, tnode, descs)); + } else { + *node = pool->add(new EsHttpScanNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::SCHEMA_SCAN_NODE: - *node = pool->add(new SchemaScanNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VSchemaScanNode(pool, tnode, descs)); + } else { + *node = pool->add(new SchemaScanNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::OLAP_SCAN_NODE: if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VOlapScanNode(pool, tnode, descs)); } else { *node = pool->add(new OlapScanNode(pool, tnode, descs)); } @@ -384,6 +459,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN case TPlanNodeType::AGGREGATION_NODE: if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::AggregationNode(pool, tnode, descs)); } else { if (config::enable_partitioned_aggregation) { *node = pool->add(new PartitionedAggregationNode(pool, tnode, descs)); @@ -394,11 +470,16 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN return Status::OK(); case TPlanNodeType::HASH_JOIN_NODE: - *node = pool->add(new HashJoinNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::HashJoinNode(pool, tnode, descs)); + } else { + *node = pool->add(new HashJoinNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::CROSS_JOIN_NODE: if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VCrossJoinNode(pool, tnode, descs)); } else { *node = pool->add(new CrossJoinNode(pool, tnode, descs)); } @@ -409,18 +490,27 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN return Status::OK(); case TPlanNodeType::EMPTY_SET_NODE: - *node = pool->add(new EmptySetNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VEmptySetNode(pool, tnode, descs)); + } else { + *node = pool->add(new EmptySetNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::EXCHANGE_NODE: if (state->enable_vectorized_exec()) { + *node = pool->add(new doris::vectorized::VExchangeNode(pool, tnode, descs)); } else { *node = pool->add(new ExchangeNode(pool, tnode, descs)); } return Status::OK(); case TPlanNodeType::SELECT_NODE: - *node = pool->add(new SelectNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new doris::vectorized::VSelectNode(pool, tnode, descs)); + } else { + *node = pool->add(new SelectNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::OLAP_REWRITE_NODE: @@ -429,6 +519,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN case TPlanNodeType::SORT_NODE: if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VSortNode(pool, tnode, descs)); } else { if (tnode.sort_node.use_top_n) { *node = pool->add(new TopNNode(pool, tnode, descs)); @@ -439,8 +530,12 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN return Status::OK(); case TPlanNodeType::ANALYTIC_EVAL_NODE: - *node = pool->add(new AnalyticEvalNode(pool, tnode, descs)); - break; + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VAnalyticEvalNode(pool, tnode, descs)); + } else { + *node = pool->add(new AnalyticEvalNode(pool, tnode, descs)); + } + return Status::OK(); case TPlanNodeType::MERGE_NODE: *node = pool->add(new MergeNode(pool, tnode, descs)); @@ -448,17 +543,26 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN case TPlanNodeType::UNION_NODE: if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VUnionNode(pool, tnode, descs)); } else { *node = pool->add(new UnionNode(pool, tnode, descs)); } return Status::OK(); case TPlanNodeType::INTERSECT_NODE: - *node = pool->add(new IntersectNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VIntersectNode(pool, tnode, descs)); + } else { + *node = pool->add(new IntersectNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::EXCEPT_NODE: - *node = pool->add(new ExceptNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VExceptNode(pool, tnode, descs)); + } else { + *node = pool->add(new ExceptNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::BROKER_SCAN_NODE: @@ -466,11 +570,19 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN return Status::OK(); case TPlanNodeType::REPEAT_NODE: - *node = pool->add(new RepeatNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VRepeatNode(pool, tnode, descs)); + } else { + *node = pool->add(new RepeatNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::ASSERT_NUM_ROWS_NODE: - *node = pool->add(new AssertNumRowsNode(pool, tnode, descs)); + if (state->enable_vectorized_exec()) { + *node = pool->add(new vectorized::VAssertNumRowsNode(pool, tnode, descs)); + } else { + *node = pool->add(new AssertNumRowsNode(pool, tnode, descs)); + } return Status::OK(); case TPlanNodeType::TABLE_FUNCTION_NODE: @@ -637,6 +749,22 @@ Status ExecNode::claim_buffer_reservation(RuntimeState* state) { Status ExecNode::release_unused_reservation() { return _buffer_pool_client.DecreaseReservationTo(_resource_profile.min_reservation); } + +void ExecNode::release_block_memory(vectorized::Block& block, uint16_t child_idx) { + DCHECK(child_idx < _children.size()); + block.clear_column_data(child(child_idx)->row_desc().num_materialized_slots()); +} + +void ExecNode::reached_limit(vectorized::Block* block, bool* eos) { + if (_limit != -1 and _num_rows_returned + block->rows() >= _limit) { + block->set_num_rows(_limit - _num_rows_returned); + *eos = true; + } + + _num_rows_returned += block->rows(); + COUNTER_SET(_rows_returned_counter, _num_rows_returned); +} + /* Status ExecNode::enable_deny_reservation_debug_action() { DCHECK_EQ(debug_action_, TDebugAction::SET_DENY_RESERVATION_PROBABILITY); diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h index c58f862b72..7cad50018d 100644 --- a/be/src/exec/exec_node.h +++ b/be/src/exec/exec_node.h @@ -33,6 +33,8 @@ #include "util/runtime_profile.h" #include "util/uid_util.h" // for print_id +#include "vec/exprs/vexpr_context.h" + namespace doris { class Expr; class ExprContext; @@ -219,6 +221,15 @@ protected: /// fails. Status release_unused_reservation(); + /// Release all memory of block which got from child. The block + // 1. clear mem of valid column get from child, make sure child can reuse the mem + // 2. delete and release the column which create by function all and other reason + void release_block_memory(vectorized::Block& block, uint16_t child_idx = 0); + + /// Only use in vectorized exec engine to check whether reach limit and cut num row for block + // and add block rows for profile + void reached_limit(vectorized::Block* block, bool* eos); + /// Enable the increase reservation denial probability on 'buffer_pool_client_' based on /// the 'debug_action_' set on this node. Returns an error if 'debug_action_param_' is /// invalid. @@ -275,6 +286,8 @@ protected: std::vector _conjunct_ctxs; std::vector _tuple_ids; + std::unique_ptr _vconjunct_ctx_ptr; + std::vector _children; RowDescriptor _row_descriptor; diff --git a/be/src/exec/mysql_scan_node.h b/be/src/exec/mysql_scan_node.h index 6d9d602d6b..ff5fb54f99 100644 --- a/be/src/exec/mysql_scan_node.h +++ b/be/src/exec/mysql_scan_node.h @@ -63,7 +63,7 @@ private: // The Mysql value is converted into the appropriate target type. Status write_text_slot(char* value, int value_length, SlotDescriptor* slot, RuntimeState* state); - +protected: bool _is_init; MysqlScannerParam _my_param; // Name of Mysql table diff --git a/be/src/exec/odbc_scan_node.cpp b/be/src/exec/odbc_scan_node.cpp index 6bdc620993..ea32cd8f14 100644 --- a/be/src/exec/odbc_scan_node.cpp +++ b/be/src/exec/odbc_scan_node.cpp @@ -29,9 +29,11 @@ namespace doris { -OdbcScanNode::OdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) +OdbcScanNode::OdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, + std::string scan_node_type) : ScanNode(pool, tnode, descs), _is_init(false), + _scan_node_type(scan_node_type), _table_name(tnode.odbc_scan_node.table_name), _connect_string(std::move(tnode.odbc_scan_node.connect_string)), _query_string(std::move(tnode.odbc_scan_node.query_string)), @@ -42,7 +44,7 @@ OdbcScanNode::OdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const Descr OdbcScanNode::~OdbcScanNode() {} Status OdbcScanNode::prepare(RuntimeState* state) { - VLOG_CRITICAL << "OdbcScanNode::Prepare"; + VLOG_CRITICAL << _scan_node_type << "::Prepare"; if (_is_init) { return Status::OK(); @@ -91,7 +93,7 @@ Status OdbcScanNode::prepare(RuntimeState* state) { Status OdbcScanNode::open(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::open(state)); - VLOG_CRITICAL << "OdbcScanNode::Open"; + VLOG_CRITICAL << _scan_node_type << "::Open"; if (nullptr == state) { return Status::InternalError("input pointer is null."); @@ -125,7 +127,7 @@ Status OdbcScanNode::write_text_slot(char* value, int value_length, SlotDescript } Status OdbcScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) { - VLOG_CRITICAL << "OdbcScanNode::GetNext"; + VLOG_CRITICAL << _scan_node_type << "::GetNext"; if (nullptr == state || nullptr == row_batch || nullptr == eos) { return Status::InternalError("input is nullptr pointer"); @@ -240,7 +242,7 @@ Status OdbcScanNode::close(RuntimeState* state) { void OdbcScanNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); - *out << "OdbcScanNode(tupleid=" << _tuple_id << " table=" << _table_name; + *out << _scan_node_type << "(tupleid=" << _tuple_id << " table=" << _table_name; *out << ")" << std::endl; for (int i = 0; i < _children.size(); ++i) { diff --git a/be/src/exec/odbc_scan_node.h b/be/src/exec/odbc_scan_node.h index 393d9ac29f..1754993628 100644 --- a/be/src/exec/odbc_scan_node.h +++ b/be/src/exec/odbc_scan_node.h @@ -35,7 +35,8 @@ class Status; class OdbcScanNode : public ScanNode { public: - OdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + OdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, + std::string scan_node_type = "OdbcScanNode"); ~OdbcScanNode(); // initialize _odbc_scanner, and create _text_converter. @@ -53,6 +54,12 @@ public: // No use virtual Status set_scan_ranges(const std::vector& scan_ranges); + const TupleDescriptor* get_tuple_desc() { return _tuple_desc; } + TextConverter* get_text_converter() { return _text_converter.get(); } + ODBCConnector* get_odbc_scanner() { return _odbc_scanner.get(); } + const std::string& get_scan_node_type() { return _scan_node_type; } + + bool is_init() { return _is_init; } protected: // Write debug string of this into out. @@ -65,6 +72,9 @@ private: RuntimeState* state); bool _is_init; + + std::string _scan_node_type; + // Name of Odbc table std::string _table_name; diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index ebf872043b..dfc9a293ee 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -61,7 +61,7 @@ OlapScanNode::OlapScanNode(ObjectPool* pool, const TPlanNode& tnode, const Descr Status OlapScanNode::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(ExecNode::init(tnode, state)); - _direct_conjunct_size = state->enable_vectorized_exec() ? 1 : _conjunct_ctxs.size(); + _direct_conjunct_size = _conjunct_ctxs.size(); const TQueryOptions& query_options = state->query_options(); if (query_options.__isset.max_scan_key_num) { @@ -464,7 +464,6 @@ Status OlapScanNode::start_scan(RuntimeState* state) { VLOG_CRITICAL << "Filter idle conjuncts"; // 5. Filter idle conjunct which already trans to olap filters // this must be after build_scan_key, it will free the StringValue memory - // TODO: filter idle conjunct in vexpr_contexts remove_pushed_conjuncts(state); VLOG_CRITICAL << "StartScanThread"; @@ -516,6 +515,7 @@ void OlapScanNode::remove_pushed_conjuncts(RuntimeState* state) { _conjunct_ctxs = std::move(new_conjunct_ctxs); _direct_conjunct_size = new_direct_conjunct_size; + // TODO: support vbloom_filter_predicate/vbinary_predicate and merge unpushed predicate to _vconjunct_ctx for (auto push_down_ctx : _pushed_conjuncts_index) { auto iter = _conjunctid_to_runtime_filter_ctxs.find(push_down_ctx); if (iter != _conjunctid_to_runtime_filter_ctxs.end()) { @@ -524,7 +524,13 @@ void OlapScanNode::remove_pushed_conjuncts(RuntimeState* state) { } // set vconjunct_ctx is empty, if all conjunct if (_direct_conjunct_size == 0) { + if (_vconjunct_ctx_ptr.get() != nullptr) { + (*_vconjunct_ctx_ptr.get())->close(state); + _vconjunct_ctx_ptr = nullptr; + } } + // filter idle conjunct in vexpr_contexts + _peel_pushed_conjuncts(); } void OlapScanNode::eval_const_conjuncts() { @@ -843,11 +849,6 @@ static bool ignore_cast(SlotDescriptor* slot, Expr* expr) { bool OlapScanNode::should_push_down_in_predicate(doris::SlotDescriptor* slot, doris::InPredicate* pred) { - if (pred->is_not_in()) { - // can not push down NOT IN predicate to storage engine - return false; - } - if (Expr::type_without_cast(pred->get_child(0)) != TExprNodeType::SLOT_REF) { // not a slot ref(column) return false; @@ -1672,4 +1673,46 @@ Status OlapScanNode::add_one_batch(RowBatch* row_batch) { } +vectorized::VExpr* OlapScanNode::_dfs_peel_conjunct(vectorized::VExpr* expr, int& leaf_index) { + static constexpr auto is_leaf = [](vectorized::VExpr* expr) { return !expr->is_and_expr(); }; + + if (is_leaf(expr)) { + return _pushed_conjuncts_index.count(leaf_index++) ? nullptr : expr; + } else { + vectorized::VExpr* left_child = _dfs_peel_conjunct(expr->children()[0], leaf_index); + vectorized::VExpr* right_child = _dfs_peel_conjunct(expr->children()[1], leaf_index); + + if (left_child != nullptr && right_child != nullptr) { + expr->set_children({left_child, right_child}); + return expr; + } + // here do not close Expr* now + return left_child != nullptr ? left_child : right_child; + } +} + +// This function is used to remove pushed expr in expr tree. +// It relies on the logic of function convertConjunctsToAndCompoundPredicate() of FE splicing expr. +// It requires FE to satisfy each splicing with 'and' expr, and spliced from left to right, in order. +// Expr tree specific forms do not require requirements. +void OlapScanNode::_peel_pushed_conjuncts() { + if (_vconjunct_ctx_ptr.get() == nullptr) return; + + int leaf_index = 0; + vectorized::VExpr* conjunct_expr_root = (*_vconjunct_ctx_ptr.get())->root(); + + if (conjunct_expr_root != nullptr) { + vectorized::VExpr* new_conjunct_expr_root = + _dfs_peel_conjunct(conjunct_expr_root, leaf_index); + if (new_conjunct_expr_root == nullptr) { + _vconjunct_ctx_ptr = nullptr; + _scanner_profile->add_info_string("VconjunctExprTree", "null"); + } else { + (*_vconjunct_ctx_ptr.get())->set_root(new_conjunct_expr_root); + _scanner_profile->add_info_string("VconjunctExprTree", + new_conjunct_expr_root->debug_string()); + } + } +} + } // namespace doris diff --git a/be/src/exec/olap_scan_node.h b/be/src/exec/olap_scan_node.h index 3b7df03452..f9905e256d 100644 --- a/be/src/exec/olap_scan_node.h +++ b/be/src/exec/olap_scan_node.h @@ -32,6 +32,8 @@ #include "runtime/vectorized_row_batch.h" #include "util/progress_updater.h" #include "util/spinlock.h" +#include "vec/exec/volap_scanner.h" +#include "vec/exprs/vexpr.h" namespace doris { class IRuntimeFilter; @@ -158,6 +160,7 @@ protected: RuntimeProfile* profile); friend class OlapScanner; + friend class doris::vectorized::VOlapScanner; // Tuple id resolved in prepare() to set _tuple_desc; TupleId _tuple_id; @@ -215,7 +218,7 @@ protected: std::mutex _scan_batches_lock; std::condition_variable _scan_batch_added_cv; - int64_t _running_thread = 0; + std::atomic_int _running_thread = 0; std::condition_variable _scan_thread_exit_cv; std::list _scan_row_batches; @@ -321,6 +324,9 @@ protected: RuntimeProfile::Counter* _scanner_wait_worker_timer = nullptr; RuntimeProfile::Counter* _olap_wait_batch_queue_timer = nullptr; + + vectorized::VExpr* _dfs_peel_conjunct(vectorized::VExpr* expr, int& leaf_index); + void _peel_pushed_conjuncts(); // remove pushed expr from conjunct tree }; } // namespace doris diff --git a/be/src/exec/olap_scanner.cpp b/be/src/exec/olap_scanner.cpp index af1d658054..2e05c5d12c 100644 --- a/be/src/exec/olap_scanner.cpp +++ b/be/src/exec/olap_scanner.cpp @@ -19,9 +19,9 @@ #include -#include "gen_cpp/PaloInternalService_types.h" #include "common/utils.h" #include "exprs/expr_context.h" +#include "gen_cpp/PaloInternalService_types.h" #include "olap/decimal12.h" #include "olap/field.h" #include "olap/uint24.h" @@ -176,8 +176,10 @@ Status OlapScanner::_init_tablet_reader_params( _tablet_reader_params.rs_readers[1]->rowset()->start_version() == 2 && !_tablet_reader_params.rs_readers[1]->rowset()->rowset_meta()->is_segments_overlapping()); + _tablet_reader_params.origin_return_columns = &_return_columns; if (_aggregation || single_version) { _tablet_reader_params.return_columns = _return_columns; + _tablet_reader_params.direct_mode = true; } else { // we need to fetch all key columns to do the right aggregation on storage engine side. for (size_t i = 0; i < _tablet->num_key_columns(); ++i) { @@ -239,7 +241,8 @@ Status OlapScanner::_init_return_columns() { } } if (auto sequence_col_idx = _tablet->tablet_schema().sequence_col_idx(); - has_replace_col && std::find(_return_columns.begin(), _return_columns.end(), sequence_col_idx) == _return_columns.end()) { + has_replace_col && std::find(_return_columns.begin(), _return_columns.end(), + sequence_col_idx) == _return_columns.end()) { _return_columns.push_back(sequence_col_idx); } } diff --git a/be/src/exec/olap_scanner.h b/be/src/exec/olap_scanner.h index f234925647..0c684d9851 100644 --- a/be/src/exec/olap_scanner.h +++ b/be/src/exec/olap_scanner.h @@ -58,7 +58,7 @@ public: Status open(); - Status get_batch(RuntimeState* state, RowBatch* batch, bool* eof); + virtual Status get_batch(RuntimeState* state, RowBatch* batch, bool* eof); Status close(RuntimeState* state); @@ -103,7 +103,7 @@ protected: // Update profile that need to be reported in realtime. void _update_realtime_counter(); - virtual void set_tablet_reader() { _tablet_reader.reset(new TupleReader); } + virtual void set_tablet_reader() { _tablet_reader = std::make_unique(); } protected: RuntimeState* _runtime_state; diff --git a/be/src/exec/partitioned_aggregation_node.cc b/be/src/exec/partitioned_aggregation_node.cc index 2311f82f7b..68f536ffda 100644 --- a/be/src/exec/partitioned_aggregation_node.cc +++ b/be/src/exec/partitioned_aggregation_node.cc @@ -1048,7 +1048,7 @@ Tuple* PartitionedAggregationNode::GetOutputTuple(const vectorrows_returned() == 0); + grouping_exprs_.size() == 0 && child(0)->rows_returned() == 0); } else { NewAggFnEvaluator::Serialize(agg_fn_evals, tuple); } diff --git a/be/src/exec/repeat_node.h b/be/src/exec/repeat_node.h index 01335d2233..d9dce75278 100644 --- a/be/src/exec/repeat_node.h +++ b/be/src/exec/repeat_node.h @@ -40,7 +40,7 @@ public: protected: virtual void debug_string(int indentation_level, std::stringstream* out) const override; -private: +protected: Status get_repeated_batch(RowBatch* child_row_batch, int repeat_id_idx, RowBatch* row_batch); // Slot id set used to indicate those slots need to set to null. diff --git a/be/src/exec/scan_node.h b/be/src/exec/scan_node.h index 5f8ec4b082..bd0999127f 100644 --- a/be/src/exec/scan_node.h +++ b/be/src/exec/scan_node.h @@ -91,7 +91,7 @@ public: protected: RuntimeProfile::Counter* _bytes_read_counter; // # bytes read from the scanner - // # rows/tuples read from the scanner (including those discarded by eval_conjucts()) + // # rows/tuples read from the scanner (including those discarded by eval_conjuncts()) RuntimeProfile::Counter* _rows_read_counter; // Wall based aggregate read throughput [bytes/sec] RuntimeProfile::Counter* _total_throughput_counter; diff --git a/be/src/exec/schema_scan_node.h b/be/src/exec/schema_scan_node.h index 2052de05ca..f45086c78f 100644 --- a/be/src/exec/schema_scan_node.h +++ b/be/src/exec/schema_scan_node.h @@ -62,7 +62,7 @@ private: void debug_string(int indentation_level, std::stringstream* out) const override; // Copy one row from schema table to input tuple void copy_one_row(); - +protected: bool _is_init; const std::string _table_name; SchemaScannerParam _scanner_param; diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp index 7d470bf33c..a1195d5b35 100644 --- a/be/src/exec/tablet_info.cpp +++ b/be/src/exec/tablet_info.cpp @@ -407,4 +407,228 @@ uint32_t OlapTablePartitionParam::_compute_dist_hash(Tuple* key) const { return hash_val; } +VOlapTablePartitionParam::VOlapTablePartitionParam(std::shared_ptr& schema, + const TOlapTablePartitionParam& t_param) + : _schema(schema), + _t_param(t_param), + _slots(_schema->tuple_desc()->slots()), + _mem_tracker(MemTracker::CreateTracker(-1, "OlapTablePartitionParam")) { + for (auto slot : _slots) { + _partition_block.insert({slot->get_empty_mutable_column(), slot->get_data_type_ptr(), slot->col_name()}); + } +} + +VOlapTablePartitionParam::~VOlapTablePartitionParam() { + _mem_tracker->Release(_mem_usage); +} + +Status VOlapTablePartitionParam::init() { + std::vector slot_column_names; + for (auto slot_desc : _schema->tuple_desc()->slots()) { + slot_column_names.emplace_back(slot_desc->col_name()); + } + + auto find_slot_locs = [&slot_column_names](const std::string& slot_name, std::vector& locs, const std::string& column_type) { + auto it = std::find(slot_column_names.begin(), slot_column_names.end(), slot_name); + if (it == slot_column_names.end()) { + return Status::InternalError(column_type + " column not found, column =" + slot_name); + } + locs.emplace_back(it - slot_column_names.begin()); + return Status::OK(); + }; + + if (_t_param.__isset.partition_columns) { + for (auto& part_col : _t_param.partition_columns) { + RETURN_IF_ERROR(find_slot_locs(part_col, _partition_slot_locs, "partition")); + } + } + + _partitions_map.reset(new std::map( + VOlapTablePartKeyComparator(_partition_slot_locs))); + if (_t_param.__isset.distributed_columns) { + for (auto& col : _t_param.distributed_columns) { + RETURN_IF_ERROR(find_slot_locs(col, _distributed_slot_locs, "distributed")); + } + } + + DCHECK(!_t_param.partitions.empty()) << "must have at least 1 partition"; + _is_in_partition = _t_param.partitions[0].__isset.in_keys; + + // initial partitions + for (int i = 0; i < _t_param.partitions.size(); ++i) { + const TOlapTablePartition& t_part = _t_param.partitions[i]; + auto part = _obj_pool.add(new VOlapTablePartition(&_partition_block)); + part->id = t_part.id; + + if (!_is_in_partition) { + if (t_part.__isset.start_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.start_keys, &part->start_key)); + } + + if (t_part.__isset.end_keys) { + RETURN_IF_ERROR(_create_partition_keys(t_part.end_keys, &part->end_key)); + } + } else { + for (const auto& keys : t_part.in_keys) { + RETURN_IF_ERROR(_create_partition_keys(keys, &part->in_keys.emplace_back(&_partition_block, -1))); + } + } + + part->num_buckets = t_part.num_buckets; + auto num_indexes = _schema->indexes().size(); + if (t_part.indexes.size() != num_indexes) { + std::stringstream ss; + ss << "number of partition's index is not equal with schema's" + << ", num_part_indexes=" << t_part.indexes.size() + << ", num_schema_indexes=" << num_indexes; + return Status::InternalError(ss.str()); + } + part->indexes = t_part.indexes; + std::sort(part->indexes.begin(), part->indexes.end(), + [](const OlapTableIndexTablets& lhs, const OlapTableIndexTablets& rhs) { + return lhs.index_id < rhs.index_id; + }); + // check index + for (int j = 0; j < num_indexes; ++j) { + if (part->indexes[j].index_id != _schema->indexes()[j]->index_id) { + std::stringstream ss; + ss << "partition's index is not equal with schema's" + << ", part_index=" << part->indexes[j].index_id + << ", schema_index=" << _schema->indexes()[j]->index_id; + return Status::InternalError(ss.str()); + } + } + _partitions.emplace_back(part); + if (_is_in_partition) { + for (auto& in_key : part->in_keys) { + _partitions_map->emplace(&in_key, part); + } + } else { + _partitions_map->emplace(&part->end_key, part); + } + } + + _mem_usage = _partition_block.allocated_bytes(); + _mem_tracker->Consume(_mem_usage); + return Status::OK(); +} + +bool VOlapTablePartitionParam::find_tablet(BlockRow* block_row, const VOlapTablePartition** partition, + uint32_t* dist_hashes) const { + auto it = _is_in_partition ? _partitions_map->find(block_row) : _partitions_map->upper_bound(block_row); + if (it == _partitions_map->end()) { + return false; + } + if (_is_in_partition || _part_contains(it->second, block_row)) { + *partition = it->second; + *dist_hashes = _compute_dist_hash(block_row); + return true; + } + return false; +} + +Status VOlapTablePartitionParam::_create_partition_keys(const std::vector& t_exprs, + BlockRow* part_key) { + for (int i = 0; i < t_exprs.size(); i++) { + RETURN_IF_ERROR(_create_partition_key(t_exprs[i], part_key->first, + _partition_slot_locs[i])); + } + return Status::OK(); +} + +Status VOlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, vectorized::Block* block, + uint16_t pos) { + auto column = std::move(*block->get_by_position(pos).column).mutate(); + switch (t_expr.node_type) { + case TExprNodeType::DATE_LITERAL: { + vectorized::VecDateTimeValue dt; + if (!dt.from_date_str( + t_expr.date_literal.value.c_str(), t_expr.date_literal.value.size())) { + std::stringstream ss; + ss << "invalid date literal in partition column, date=" << t_expr.date_literal; + return Status::InternalError(ss.str()); + } + column->insert_data(reinterpret_cast(&dt), 0); + break; + } + case TExprNodeType::INT_LITERAL: { + switch (t_expr.type.types[0].scalar_type.type) { + case TPrimitiveType::TINYINT: { + int8_t value = t_expr.int_literal.value; + column->insert_data(reinterpret_cast(&value), 0); + break; + } + case TPrimitiveType::SMALLINT: { + int16_t value = t_expr.int_literal.value; + column->insert_data(reinterpret_cast(&value), 0); + break; + } + case TPrimitiveType::INT: { + int32_t value = t_expr.int_literal.value; + column->insert_data(reinterpret_cast(&value), 0); + break; + } + default: + int64_t value = t_expr.int_literal.value; + column->insert_data(reinterpret_cast(&value), 0); + } + break; + } case TExprNodeType::LARGE_INT_LITERAL: { + StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS; + __int128 value = StringParser::string_to_int<__int128>(t_expr.large_int_literal.value.c_str(), + t_expr.large_int_literal.value.size(), + &parse_result); + if (parse_result != StringParser::PARSE_SUCCESS) { + value = MAX_INT128; + } + column->insert_data(reinterpret_cast(&value), 0); + break; + } case TExprNodeType::STRING_LITERAL: { + int len = t_expr.string_literal.value.size(); + const char* str_val = t_expr.string_literal.value.c_str(); + + // CHAR is a fixed-length string and needs to use the length in the slot definition, + // VARVHAR is a variable-length string and needs to use the length of the string itself + // padding 0 to CHAR field +// if (TYPE_CHAR == slot_desc->type().type && len < slot_desc->type().len) { +// auto new_ptr = (char*)_mem_pool->allocate(slot_desc->type().len); +// memset(new_ptr, 0, slot_desc->type().len); +// memcpy(new_ptr, str_val, len); +// +// str_val = new_ptr; +// len = slot_desc->type().len; +// } + column->insert_data(str_val, len); + break; + } case TExprNodeType::BOOL_LITERAL: { + column->insert_data(reinterpret_cast(&t_expr.bool_literal.value), 0); + break; + } default: { + std::stringstream ss; + ss << "unsupported partition column node type, type=" << t_expr.node_type; + return Status::InternalError(ss.str()); + } + } + return Status::OK(); +} + +uint32_t VOlapTablePartitionParam::_compute_dist_hash(BlockRow* key) const { + uint32_t hash_val = 0; + for (int i = 0; i < _distributed_slot_locs.size(); ++i) { + auto slot_desc = _slots[_distributed_slot_locs[i]]; + auto column = key->first->get_by_position(_distributed_slot_locs[i]).column; + + auto val = column->get_data_at(key->second); + if (val.data != nullptr) { + hash_val = RawValue::zlib_crc32(val.data, val.size, slot_desc->type().type, hash_val); + } else { + // NULL is treat as 0 when hash + static const int INT_VALUE = 0; + static const TypeDescriptor INT_TYPE(TYPE_INT); + hash_val = RawValue::zlib_crc32(&INT_VALUE, INT_TYPE, hash_val); + } + } + return hash_val; +} + } // namespace doris diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h index 7bb2640582..53221baa04 100644 --- a/be/src/exec/tablet_info.h +++ b/be/src/exec/tablet_info.h @@ -31,6 +31,8 @@ #include "runtime/raw_value.h" #include "runtime/tuple.h" +#include "vec/core/block.h" + namespace doris { class MemPool; @@ -205,6 +207,104 @@ private: _partitions_map; }; +using BlockRow = std::pair; + +struct VOlapTablePartition { + int64_t id = 0; + BlockRow start_key; + BlockRow end_key; + std::vector in_keys; + int64_t num_buckets = 0; + std::vector indexes; + + VOlapTablePartition(vectorized::Block* partition_block): + start_key{partition_block, -1}, end_key{partition_block, -1} {}; +}; + +class VOlapTablePartKeyComparator { +public: + VOlapTablePartKeyComparator(const std::vector& slot_locs) + : _slot_locs(slot_locs) {} + + // return true if lhs < rhs + // 'row' is -1 mean + bool operator()(const BlockRow* lhs, const BlockRow* rhs) const { + if (lhs->second == -1) { + return false; + } else if (rhs->second == -1) { + return true; + } + + for (auto slot_loc : _slot_locs) { + auto res = lhs->first->get_by_position(slot_loc).column->compare_at( + lhs->second, rhs->second, *rhs->first->get_by_position(slot_loc).column, -1); + if (res != 0) { + return res < 0; + } + } + // equal, return false + return false; + } + +private: + const std::vector& _slot_locs; +}; + +// store an olap table's tablet information +class VOlapTablePartitionParam { +public: + VOlapTablePartitionParam(std::shared_ptr& schema, + const TOlapTablePartitionParam& param); + + ~VOlapTablePartitionParam(); + + Status init(); + + int64_t db_id() const { return _t_param.db_id; } + int64_t table_id() const { return _t_param.table_id; } + int64_t version() const { return _t_param.version; } + + // return true if we found this tuple in partition + bool find_tablet(BlockRow* block_row, const VOlapTablePartition** partitions, + uint32_t* dist_hash) const; + + const std::vector& get_partitions() const { return _partitions; } + +private: + Status _create_partition_keys(const std::vector& t_exprs, BlockRow* part_key); + + Status _create_partition_key(const TExprNode& t_expr, vectorized::Block* block, uint16_t pos); + + uint32_t _compute_dist_hash(BlockRow* key) const; + + // check if this partition contain this key + bool _part_contains(VOlapTablePartition* part, BlockRow* key) const { + // start_key.second == -1 means only single partition + VOlapTablePartKeyComparator comparator(_partition_slot_locs); + return part->start_key.second == -1 || !comparator(key, &part->start_key); + } + +private: + // this partition only valid in this schema + std::shared_ptr _schema; + TOlapTablePartitionParam _t_param; + + const std::vector& _slots; + std::vector _partition_slot_locs; + std::vector _distributed_slot_locs; + + ObjectPool _obj_pool; + vectorized::Block _partition_block; + std::shared_ptr _mem_tracker; + std::vector _partitions; + std::unique_ptr> + _partitions_map; + + bool _is_in_partition = false; + uint32_t _mem_usage = 0; +}; + + using TabletLocation = TTabletLocation; // struct TTabletLocation { // 1: required i64 tablet_id diff --git a/be/src/exec/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp index 1718f7d9d7..7acde285b8 100644 --- a/be/src/exec/tablet_sink.cpp +++ b/be/src/exec/tablet_sink.cpp @@ -251,6 +251,53 @@ Status NodeChannel::add_row(Tuple* input_tuple, int64_t tablet_id) { return Status::OK(); } +Status NodeChannel::add_row(BlockRow& block_row, int64_t tablet_id) { + // If add_row() when _eos_is_produced==true, there must be sth wrong, we can only mark this channel as failed. + auto st = none_of({_cancelled, _eos_is_produced}); + if (!st.ok()) { + if (_cancelled) { + std::lock_guard l(_cancel_msg_lock); + return Status::InternalError("add row failed. " + _cancel_msg); + } else { + return st.clone_and_prepend("already stopped, can't add row. cancelled/eos: "); + } + } + + // We use OlapTableSink mem_tracker which has the same ancestor of _plan node, + // so in the ideal case, mem limit is a matter for _plan node. + // But there is still some unfinished things, we do mem limit here temporarily. + // _cancelled may be set by rpc callback, and it's possible that _cancelled might be set in any of the steps below. + // It's fine to do a fake add_row() and return OK, because we will check _cancelled in next add_row() or mark_close(). + while (!_cancelled && _parent->_mem_tracker->AnyLimitExceeded(MemLimit::HARD) && + _pending_batches_num > 0) { + SCOPED_ATOMIC_TIMER(&_mem_exceeded_block_ns); + SleepFor(MonoDelta::FromMilliseconds(10)); + } + + auto row_no = _cur_batch->add_row(); + if (row_no == RowBatch::INVALID_ROW_INDEX) { + { + SCOPED_ATOMIC_TIMER(&_queue_push_lock_ns); + std::lock_guard l(_pending_batches_lock); + //To simplify the add_row logic, postpone adding batch into req until the time of sending req + _pending_batches.emplace(std::move(_cur_batch), _cur_add_batch_request); + _pending_batches_num++; + } + + _cur_batch.reset(new RowBatch(*_row_desc, _batch_size, _parent->_mem_tracker.get())); + _cur_add_batch_request.clear_tablet_ids(); + + row_no = _cur_batch->add_row(); + } + DCHECK_NE(row_no, RowBatch::INVALID_ROW_INDEX); + + _cur_batch->get_row(row_no)->set_tuple(0, + block_row.first->deep_copy_tuple(*_tuple_desc, _cur_batch->tuple_data_pool(), block_row.second, 0, true)); + _cur_batch->commit_last_row(); + _cur_add_batch_request.add_tablet_ids(tablet_id); + return Status::OK(); +} + Status NodeChannel::mark_close() { auto st = none_of({_cancelled, _eos_is_produced}); if (!st.ok()) { @@ -501,6 +548,29 @@ Status IndexChannel::add_row(Tuple* tuple, int64_t tablet_id) { return Status::OK(); } +Status IndexChannel::add_row(BlockRow& block_row, int64_t tablet_id) { + auto it = _channels_by_tablet.find(tablet_id); + DCHECK(it != _channels_by_tablet.end()) << "unknown tablet, tablet_id=" << tablet_id; + std::stringstream ss; + for (auto channel : it->second) { + // if this node channel is already failed, this add_row will be skipped + auto st = channel->add_row(block_row, tablet_id); + if (!st.ok()) { + mark_as_failed(channel); + ss << st.get_error_msg() << "; "; + } + } + + if (has_intolerable_failure()) { + std::stringstream ss2; + ss2 << "index channel has intolerable failure. " << BackendOptions::get_localhost() + << ", err: " << ss.str(); + return Status::InternalError(ss2.str()); + } + + return Status::OK(); +} + bool IndexChannel::has_intolerable_failure() { for (const auto& it : _failed_channels) { if (it.second.size() >= ((_parent->_num_replicas + 1) / 2)) { @@ -731,7 +801,7 @@ Status OlapTableSink::send(RuntimeState* state, RowBatch* input_batch) { batch = _output_batch.get(); } int num_invalid_rows = 0; - if (_need_validate_data) { + { SCOPED_RAW_TIMER(&_validate_data_ns); _filter_bitmap.Reset(batch->num_rows()); num_invalid_rows = _validate_data(state, batch, &_filter_bitmap); diff --git a/be/src/exec/tablet_sink.h b/be/src/exec/tablet_sink.h index b5e86f7f92..6f403b1dd8 100644 --- a/be/src/exec/tablet_sink.h +++ b/be/src/exec/tablet_sink.h @@ -160,6 +160,8 @@ public: Status add_row(Tuple* tuple, int64_t tablet_id); + Status add_row(BlockRow& block_row, int64_t tablet_id); + // two ways to stop channel: // 1. mark_close()->close_wait() PS. close_wait() will block waiting for the last AddBatch rpc response. // 2. just cancel() @@ -276,6 +278,8 @@ public: Status add_row(Tuple* tuple, int64_t tablet_id); + Status add_row(BlockRow& block_row, int64_t tablet_id); + void for_each_node_channel(const std::function& func) { for (auto& it : _node_channels) { func(it.second); @@ -350,7 +354,7 @@ private: // only focus on pending batches and channel status, the internal errors of NodeChannels will be handled by the producer void _send_batch_process(); -private: +protected: friend class NodeChannel; friend class IndexChannel; diff --git a/be/src/exec/text_converter.cpp b/be/src/exec/text_converter.cpp index 7801f66208..e2d37d536a 100644 --- a/be/src/exec/text_converter.cpp +++ b/be/src/exec/text_converter.cpp @@ -59,4 +59,26 @@ void TextConverter::unescape_string(const char* src, char* dest, size_t* len) { *len = dest_ptr - dest_start; } +void TextConverter::unescape_string_on_spot(const char* src, size_t* len) { + char* dest_ptr = const_cast(src); + const char* end = src + *len; + bool escape_next_char = false; + + while (src < end) { + if (*src == _escape_char) { + escape_next_char = !escape_next_char; + } else { + escape_next_char = false; + } + + if (escape_next_char) { + ++src; + } else { + *dest_ptr++ = *src++; + } + } + + *len = dest_ptr - src; +} + } // namespace doris diff --git a/be/src/exec/text_converter.h b/be/src/exec/text_converter.h index 4d97b7c62b..36b3254fa7 100644 --- a/be/src/exec/text_converter.h +++ b/be/src/exec/text_converter.h @@ -19,7 +19,7 @@ #define DORIS_BE_SRC_QUERY_EXEC_TEXT_CONVERTER_H #include "runtime/runtime_state.h" - +#include "vec/core/block.h" namespace doris { class MemPool; @@ -48,11 +48,14 @@ public: bool write_slot(const SlotDescriptor* slot_desc, Tuple* tuple, const char* data, int len, bool copy_string, bool need_escape, MemPool* pool); + bool write_column(const SlotDescriptor* slot_desc, vectorized::MutableColumnPtr* column_ptr, + const char* data, size_t len, bool copy_string, bool need_escape); + // Removes escape characters from len characters of the null-terminated string src, // and copies the unescaped string into dest, changing *len to the unescaped length. // No null-terminator is added to dest. void unescape_string(const char* src, char* dest, size_t* len); - + void unescape_string_on_spot(const char* src, size_t* len); // Removes escape characters from 'str', allocating a new string from pool. // 'str' is updated with the new ptr and length. void unescape_string(StringValue* str, MemPool* pool); diff --git a/be/src/exec/text_converter.hpp b/be/src/exec/text_converter.hpp index 1b1a60e64b..44e807ced0 100644 --- a/be/src/exec/text_converter.hpp +++ b/be/src/exec/text_converter.hpp @@ -17,6 +17,7 @@ #ifndef DORIS_BE_SRC_QUERY_EXEC_TEXT_CONVERTER_HPP #define DORIS_BE_SRC_QUERY_EXEC_TEXT_CONVERTER_HPP +#include #include @@ -32,7 +33,7 @@ #include "util/binary_cast.hpp" #include "util/string_parser.hpp" #include "util/types.h" - +#include "vec/runtime/vdatetime_value.h" namespace doris { // Note: this function has a codegen'd version. Changing this function requires @@ -165,6 +166,134 @@ inline bool TextConverter::write_slot(const SlotDescriptor* slot_desc, Tuple* tu return true; } +inline bool TextConverter::write_column(const SlotDescriptor* slot_desc, + vectorized::MutableColumnPtr* column_ptr, const char* data, + size_t len, bool copy_string, bool need_escape) { + vectorized::IColumn* col_ptr = column_ptr->get(); + // \N means it's NULL + if (true == slot_desc->is_nullable()) { + auto* nullable_column = reinterpret_cast(column_ptr->get()); + if ((len == 2 && data[0] == '\\' && data[1] == 'N') || len == SQL_NULL_DATA) { + nullable_column->insert_data(nullptr, 0); + return true; + } else { + nullable_column->get_null_map_data().push_back(0); + col_ptr = &nullable_column->get_nested_column(); + } + } + StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS; + + // Parse the raw-text data. Translate the text string to internal format. + switch (slot_desc->type().type) { + case TYPE_HLL: + case TYPE_VARCHAR: + case TYPE_CHAR: { + if (need_escape) { + unescape_string_on_spot(data, &len); + } + reinterpret_cast(col_ptr)->insert_data(data, len); + break; + } + + case TYPE_BOOLEAN: { + bool num = StringParser::string_to_bool(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value( + (uint8_t)num); + break; + } + case TYPE_TINYINT: { + int8_t num = StringParser::string_to_int(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value(num); + break; + } + case TYPE_SMALLINT: { + int16_t num = StringParser::string_to_int(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value(num); + break; + } + case TYPE_INT: { + int32_t num = StringParser::string_to_int(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value(num); + break; + } + case TYPE_BIGINT: { + int64_t num = StringParser::string_to_int(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value(num); + break; + } + case TYPE_LARGEINT: { + __int128 num = StringParser::string_to_int<__int128>(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value(num); + break; + } + + case TYPE_FLOAT: { + float num = StringParser::string_to_float(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value( + num); + break; + } + case TYPE_DOUBLE: { + double num = StringParser::string_to_float(data, len, &parse_result); + reinterpret_cast*>(col_ptr)->insert_value( + num); + break; + } + case TYPE_DATE: { + vectorized::VecDateTimeValue ts_slot; + if (!ts_slot.from_date_str(data, len)) { + parse_result = StringParser::PARSE_FAILURE; + break; + } + ts_slot.cast_to_date(); + reinterpret_cast*>(col_ptr)->insert_data( + reinterpret_cast(&ts_slot), 0); + break; + } + + case TYPE_DATETIME: { + vectorized::VecDateTimeValue ts_slot; + if (!ts_slot.from_date_str(data, len)) { + parse_result = StringParser::PARSE_FAILURE; + break; + } + ts_slot.to_datetime(); + reinterpret_cast*>(col_ptr)->insert_data( + reinterpret_cast(&ts_slot), 0); + break; + } + + case TYPE_DECIMALV2: { + DecimalV2Value decimal_slot; + if (decimal_slot.parse_from_str(data, len)) { + parse_result = StringParser::PARSE_FAILURE; + break; + } + PackedInt128 num = binary_cast(decimal_slot); + reinterpret_cast*>(col_ptr)->insert_value( + num.value); + break; + } + + default: + DCHECK(false) << "bad slot type: " << slot_desc->type(); + break; + } + + if (parse_result == StringParser::PARSE_FAILURE) { + if (true == slot_desc->is_nullable()) { + auto* nullable_column = + reinterpret_cast(column_ptr->get()); + size_t size = nullable_column->get_null_map_data().size(); + doris::vectorized::NullMap& null_map_data = nullable_column->get_null_map_data(); + null_map_data[size - 1] = 1; + } else { + return false; + } + } + return true; +} + } // namespace doris -#endif +#endif \ No newline at end of file diff --git a/be/src/exprs/CMakeLists.txt b/be/src/exprs/CMakeLists.txt index 5ebf1d1d26..5d69aa79e5 100644 --- a/be/src/exprs/CMakeLists.txt +++ b/be/src/exprs/CMakeLists.txt @@ -71,7 +71,6 @@ add_library(Exprs hll_function.cpp grouping_sets_functions.cpp topn_function.cpp - table_function/explode_split.cpp table_function/explode_bitmap.cpp table_function/explode_json_array.cpp diff --git a/be/src/exprs/aggregate_functions.cpp b/be/src/exprs/aggregate_functions.cpp index 1f8db84085..93166cf463 100644 --- a/be/src/exprs/aggregate_functions.cpp +++ b/be/src/exprs/aggregate_functions.cpp @@ -592,30 +592,14 @@ void AggregateFunctions::sum(FunctionContext* ctx, const DecimalV2Val& src, Deci } if (dst->is_null) { - dst->is_null = false; - dst->set_to_zero(); + init_zero_not_null(ctx, dst); } - DecimalV2Value new_src = DecimalV2Value::from_decimal_val(src); DecimalV2Value new_dst = DecimalV2Value::from_decimal_val(*dst); new_dst = new_dst + new_src; new_dst.to_decimal_val(dst); } -template <> -void AggregateFunctions::sum(FunctionContext* ctx, const LargeIntVal& src, LargeIntVal* dst) { - if (src.is_null) { - return; - } - - if (dst->is_null) { - dst->is_null = false; - dst->val = 0; - } - - dst->val += src.val; -} - template void AggregateFunctions::min_init(FunctionContext* ctx, T* dst) { auto val = AnyValUtil::max_val(ctx); @@ -1890,8 +1874,8 @@ static double compute_knuth_variance(const KnuthVarianceState& state, bool pop) static DecimalV2Value decimalv2_compute_knuth_variance(const DecimalV2KnuthVarianceState& state, bool pop) { DecimalV2Value new_count = DecimalV2Value(); - new_count.assign_from_double(state.count); if (state.count == 1) return new_count; + new_count.assign_from_double(state.count); DecimalV2Value new_m2 = DecimalV2Value::from_decimal_val(state.m2); if (pop) return new_m2 / new_count; @@ -2395,6 +2379,8 @@ template void AggregateFunctions::sum(FunctionContext*, const BigIntVal* dst); template void AggregateFunctions::sum(FunctionContext*, const BigIntVal& src, BigIntVal* dst); +template void AggregateFunctions::sum(FunctionContext*, const LargeIntVal& src, + LargeIntVal* dst); template void AggregateFunctions::sum(FunctionContext*, const FloatVal& src, DoubleVal* dst); template void AggregateFunctions::sum(FunctionContext*, const DoubleVal& src, diff --git a/be/src/exprs/math_functions.cpp b/be/src/exprs/math_functions.cpp index 787887d1fd..00cf3bd9ea 100644 --- a/be/src/exprs/math_functions.cpp +++ b/be/src/exprs/math_functions.cpp @@ -178,11 +178,11 @@ ONE_ARG_MATH_FN(ln, DoubleVal, DoubleVal, std::log); ONE_ARG_MATH_FN(log10, DoubleVal, DoubleVal, std::log10); ONE_ARG_MATH_FN(exp, DoubleVal, DoubleVal, std::exp); -FloatVal MathFunctions::sign(FunctionContext* ctx, const DoubleVal& v) { +TinyIntVal MathFunctions::sign(FunctionContext* ctx, const DoubleVal& v) { if (v.is_null) { - return FloatVal::null(); + return TinyIntVal::null(); } - return FloatVal((v.val > 0) ? 1.0f : ((v.val < 0) ? -1.0f : 0.0f)); + return TinyIntVal((v.val > 0) ? 1 : ((v.val < 0) ? -1 : 0)); } DoubleVal MathFunctions::radians(FunctionContext* ctx, const DoubleVal& v) { diff --git a/be/src/exprs/math_functions.h b/be/src/exprs/math_functions.h index 5d1258e8ac..9d55ed6758 100644 --- a/be/src/exprs/math_functions.h +++ b/be/src/exprs/math_functions.h @@ -50,7 +50,8 @@ public: static doris_udf::IntVal abs(doris_udf::FunctionContext*, const doris_udf::SmallIntVal&); static doris_udf::SmallIntVal abs(doris_udf::FunctionContext*, const doris_udf::TinyIntVal&); - static doris_udf::FloatVal sign(doris_udf::FunctionContext* ctx, const doris_udf::DoubleVal& v); + static doris_udf::TinyIntVal sign(doris_udf::FunctionContext* ctx, + const doris_udf::DoubleVal& v); static doris_udf::DoubleVal sin(doris_udf::FunctionContext*, const doris_udf::DoubleVal&); static doris_udf::DoubleVal asin(doris_udf::FunctionContext*, const doris_udf::DoubleVal&); @@ -182,11 +183,6 @@ public: static double my_double_round(double value, int64_t dec, bool dec_unsigned, bool truncate); -private: - static const int32_t MIN_BASE = 2; - static const int32_t MAX_BASE = 36; - static const char* _s_alphanumeric_chars; - // Converts src_num in decimal to dest_base, // and fills expr_val.string_val with the result. static doris_udf::StringVal decimal_to_base(doris_udf::FunctionContext* ctx, int64_t src_num, @@ -207,6 +203,12 @@ private: // Returns false otherwise, indicating some other error condition. static bool handle_parse_result(int8_t dest_base, int64_t* num, StringParser::ParseResult parse_res); + + static const int32_t MIN_BASE = 2; + static const int32_t MAX_BASE = 36; + +private: + static const char* _s_alphanumeric_chars; }; } // namespace doris diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index b77cf9d71e..3cf45c4b75 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -373,6 +373,35 @@ public: break; } } + void insert(const StringRef& value) { + switch (_column_return_type) { + case TYPE_DATE: + case TYPE_DATETIME: { + // DateTime->DateTimeValue + vectorized::DateTime date_time =*reinterpret_cast(value.data); + vectorized::VecDateTimeValue vec_date_time_value = binary_cast(date_time); + doris::DateTimeValue date_time_value; + vec_date_time_value.convert_vec_dt_to_dt(&date_time_value); + insert(reinterpret_cast(&date_time_value)); + break; + } + + case TYPE_CHAR: + case TYPE_VARCHAR: + case TYPE_HLL: + case TYPE_OBJECT: + case TYPE_STRING: { + // StringRef->StringValue + StringValue data = StringValue(const_cast(value.data), value.size); + insert(reinterpret_cast(&data)); + break; + } + + default: + insert(reinterpret_cast(value.data)); + break; + } + } template Status get_push_context(T* container, RuntimeState* state, ExprContext* prob_expr) { @@ -784,6 +813,11 @@ void IRuntimeFilter::insert(const void* data) { } } +void IRuntimeFilter::insert(const StringRef& value) { + DCHECK(is_producer()); + _wrapper->insert(value); +} + Status IRuntimeFilter::publish() { DCHECK(is_producer()); if (_has_local_target) { diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index 7d406f3600..c5e597aa21 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -128,6 +128,7 @@ public: // insert data to build filter // only used for producer void insert(const void* data); + void insert(const StringRef& data); // publish filter // push filter to remote node or push down it to scan_node diff --git a/be/src/exprs/runtime_filter_slots.h b/be/src/exprs/runtime_filter_slots.h index 576cad04b3..e098adc3d7 100644 --- a/be/src/exprs/runtime_filter_slots.h +++ b/be/src/exprs/runtime_filter_slots.h @@ -20,9 +20,9 @@ #include "exprs/runtime_filter.h" #include "runtime/runtime_filter_mgr.h" #include "runtime/runtime_state.h" +#include "vec/exprs/vexpr.h" namespace doris { - // this class used in a hash join node // Provide a unified interface for other classes template @@ -129,6 +129,40 @@ public: } } } + void insert(std::unordered_map>& datas) { + for (int i = 0; i < _build_expr_context.size(); ++i) { + auto iter = _runtime_filters.find(i); + if (iter == _runtime_filters.end()) continue; + + int result_column_id = _build_expr_context[i]->get_last_result_column_id(); + for (auto it : datas) { + auto& column = it.first->get_by_position(result_column_id).column; + + if (auto* nullable = + vectorized::check_and_get_column(*column)) { + auto& column_nested = nullable->get_nested_column(); + auto& column_nullmap = nullable->get_null_map_column(); + for (int row_num : it.second) { + if (column_nullmap.get_bool(row_num)) { + continue; + } + const auto& ref_data = column_nested.get_data_at(row_num); + for (auto filter : iter->second) { + filter->insert(ref_data); + } + } + + } else { + for (int row_num : it.second) { + const auto& ref_data = column->get_data_at(row_num); + for (auto filter : iter->second) { + filter->insert(ref_data); + } + } + } + } + } + } // should call this method after insert void ready_for_publish() { @@ -166,5 +200,5 @@ private: }; using RuntimeFilterSlots = RuntimeFilterSlotsBase; - +using VRuntimeFilterSlots = RuntimeFilterSlotsBase; } // namespace doris diff --git a/be/src/exprs/v_string_functions.h b/be/src/exprs/v_string_functions.h new file mode 100644 index 0000000000..3fd9845535 --- /dev/null +++ b/be/src/exprs/v_string_functions.h @@ -0,0 +1,219 @@ +// 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 + +#ifndef BE_V_STRING_FUNCTIONS_H +#define BE_V_STRING_FUNCTIONS_H + +#include +#include +#include "runtime/string_value.hpp" + +#ifdef __SSE2__ +#include +#endif + +namespace doris { +class VStringFunctions { +public: +#ifdef __SSE2__ + /// n equals to 16 chars length + static constexpr auto REGISTER_SIZE = sizeof(__m128i); +#endif +public: + static StringVal rtrim(const StringVal& str) { + if (str.is_null || str.len == 0) { + return str; + } + auto begin = 0; + auto end = str.len - 1; +#ifdef __SSE2__ + char blank = ' '; + const auto pattern = _mm_set1_epi8(blank); + while (end - begin + 1 >= REGISTER_SIZE) { + const auto v_haystack = _mm_loadu_si128(reinterpret_cast(str.ptr + end + 1 - REGISTER_SIZE)); + const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, pattern); + const auto mask = _mm_movemask_epi8(v_against_pattern); + int offset = __builtin_clz(~(mask << REGISTER_SIZE)); + /// means not found + if (offset == 0) + { + return StringVal(str.ptr + begin, end - begin + 1); + } else { + end -= offset; + } + } +#endif + while (end >= begin && str.ptr[end] == ' ') { + --end; + } + if (end < 0) { + return StringVal(""); + } + return StringVal(str.ptr + begin, end - begin + 1); + } + + static StringVal ltrim(const StringVal& str) { + if (str.is_null || str.len == 0) { + return str; + } + auto begin = 0; + auto end = str.len - 1; +#ifdef __SSE2__ + char blank = ' '; + const auto pattern = _mm_set1_epi8(blank); + while (end - begin + 1 >= REGISTER_SIZE) { + const auto v_haystack = _mm_loadu_si128(reinterpret_cast(str.ptr + begin)); + const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, pattern); + const auto mask = _mm_movemask_epi8(v_against_pattern); + const auto offset = __builtin_ctz(mask ^ 0xffff); + /// means not found + if (offset == 0) + { + return StringVal(str.ptr + begin, end - begin + 1); + } else if (offset > REGISTER_SIZE) { + begin += REGISTER_SIZE; + } else { + begin += offset; + return StringVal(str.ptr + begin, end - begin + 1); + } + } +#endif + while (begin <= end && str.ptr[begin] == ' ') { + ++begin; + } + return StringVal(str.ptr + begin, end - begin + 1); + } + + static StringVal trim(const StringVal& str) { + if (str.is_null || str.len == 0) { + return str; + } + return rtrim(ltrim(str)); + } + + static bool is_ascii(StringVal str) { + #ifdef __SSE2__ + size_t i = 0; + __m128i binary_code = _mm_setzero_si128(); + if (str.len >= REGISTER_SIZE) { + for (; i <= str.len - REGISTER_SIZE; i += REGISTER_SIZE) { + __m128i chars = _mm_loadu_si128((const __m128i*)(str.ptr + i)); + binary_code = _mm_or_si128(binary_code, chars); + } + } + int mask = _mm_movemask_epi8(binary_code); + + char or_code = 0; + for (; i < str.len; i++) { + or_code |= str.ptr[i]; + } + mask |= (or_code & 0x80); + + return !mask; + #else + char or_code = 0; + for (size_t i = 0; i < str.len; i++) { + or_code |= str.ptr[i]; + } + return !(or_code & 0x80); + #endif + } + + static void reverse(const StringVal& str, StringVal dst) { + if (str.is_null) { + dst.ptr = NULL; + return; + } + const bool is_ascii = VStringFunctions::is_ascii(str); + if (is_ascii) { + int64_t begin = 0; + int64_t end = str.len; + int64_t result_end = dst.len; + #if defined(__SSE2__) + const auto shuffle_array = _mm_set_epi64((__m64)0x00'01'02'03'04'05'06'07ull, (__m64)0x08'09'0a'0b'0c'0d'0e'0full); + for (; (begin + REGISTER_SIZE) < end; begin += REGISTER_SIZE) { + result_end -= REGISTER_SIZE; + _mm_storeu_si128((__m128i*)(dst.ptr + result_end), + _mm_shuffle_epi8(_mm_loadu_si128((__m128i*)(str.ptr + begin)), shuffle_array)); + } + #endif + for (; begin < end; ++begin) { + --result_end; + dst.ptr[result_end] = str.ptr[begin]; + } + } else { + for (size_t i = 0, char_size = 0; i < str.len; i += char_size) { + char_size = get_utf8_byte_length((unsigned)(str.ptr)[i]); + std::copy(str.ptr + i, str.ptr + i + char_size, dst.ptr + str.len - i - char_size); + } + } + } + + static size_t get_utf8_byte_length(unsigned char byte) { + size_t char_size = 0; + if (byte >= 0xFC) { + char_size = 6; + } else if (byte >= 0xF8) { + char_size = 5; + } else if (byte >= 0xF0) { + char_size = 4; + } else if (byte >= 0xE0) { + char_size = 3; + } else if (byte >= 0xC0) { + char_size = 2; + } else { + char_size = 1; + } + return char_size; + } + + static void hex_encode(const unsigned char* src_str, size_t length, char* dst_str) { + static constexpr auto hex_table = "0123456789ABCDEF"; + auto src_str_end = src_str + length; + +#if defined(__SSE2__) + constexpr auto step = sizeof(uint64); + if (src_str + step < src_str_end) { + const auto hex_map = _mm_loadu_si128(reinterpret_cast(hex_table)); + const auto mask_map = _mm_set1_epi8(0x0F); + + do { + auto data = _mm_loadu_si64(src_str); + auto hex_loc = _mm_and_si128(_mm_unpacklo_epi8(_mm_srli_epi64(data, 4), data), mask_map); + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst_str), _mm_shuffle_epi8(hex_map, hex_loc)); + + src_str += step; + dst_str += step * 2; + } while (src_str + step < src_str_end); + } +#endif + char res[2]; + // hex(str) str length is n, result must be 2 * n length + for (; src_str < src_str_end; src_str += 1, dst_str += 2) { + // low 4 bits + *(res + 1) = hex_table[src_str[0] & 0x0F]; + // high 4 bits + *res = hex_table[(src_str[0] >> 4)]; + std::copy(res, res + 2, dst_str); + } + } +}; +} + +#endif //BE_V_STRING_FUNCTIONS_H \ No newline at end of file diff --git a/be/src/olap/block_column_predicate.cpp b/be/src/olap/block_column_predicate.cpp index f460be5395..725ebda470 100644 --- a/be/src/olap/block_column_predicate.cpp +++ b/be/src/olap/block_column_predicate.cpp @@ -39,6 +39,30 @@ void SingleColumnBlockPredicate::evaluate_or(RowBlockV2 *block, uint16_t selecte _predicate->evaluate_or(&column_block, block->selection_vector(), selected_size, flags); } +void SingleColumnBlockPredicate::evaluate(vectorized::MutableColumns& block, uint16_t* sel, uint16_t* selected_size) const { + auto column_id = _predicate->column_id(); + auto& column = block[column_id]; + _predicate->evaluate(*column, sel, selected_size); +} + +void SingleColumnBlockPredicate::evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const { + auto column_id = _predicate->column_id(); + auto& column = block[column_id]; + _predicate->evaluate_and(*column, sel, selected_size, flags); +} + +void SingleColumnBlockPredicate::evaluate_or(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const { + auto column_id = _predicate->column_id(); + auto& column = block[column_id]; + _predicate->evaluate_or(*column, sel, selected_size, flags); +} + +void SingleColumnBlockPredicate::evaluate_vec(vectorized::MutableColumns& block, uint16_t size, bool* flags) const { + auto column_id = _predicate->column_id(); + auto& column = block[column_id]; + _predicate->evaluate_vec(*column, size, flags); +} + void OrBlockColumnPredicate::evaluate(RowBlockV2* block, uint16_t* selected_size) const { if (num_of_column_predicate() == 1) { _block_column_predicate_vec[0]->evaluate(block, selected_size); @@ -60,12 +84,39 @@ void OrBlockColumnPredicate::evaluate(RowBlockV2* block, uint16_t* selected_size } } +void OrBlockColumnPredicate::evaluate(vectorized::MutableColumns& block, uint16_t* sel, uint16_t* selected_size) const { + if (num_of_column_predicate() == 1) { + _block_column_predicate_vec[0]->evaluate(block, sel, selected_size); + } else { + bool ret_flags[*selected_size]; + memset(ret_flags, false, *selected_size); + for (int i = 0; i < num_of_column_predicate(); ++i) { + auto column_predicate = _block_column_predicate_vec[i]; + column_predicate->evaluate_or(block, sel, *selected_size, ret_flags); + } + + uint16_t new_size = 0; + for (int i = 0; i < *selected_size; ++i) { + if (ret_flags[i]) { + sel[new_size++] = sel[i]; + } + } + *selected_size = new_size; + } +} + void OrBlockColumnPredicate::evaluate_or(RowBlockV2 *block, uint16_t selected_size, bool* flags) const { for (auto block_column_predicate : _block_column_predicate_vec) { block_column_predicate->evaluate_or(block, selected_size, flags); } } +void OrBlockColumnPredicate::evaluate_or(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const { + for (auto block_column_predicate : _block_column_predicate_vec) { + block_column_predicate->evaluate_or(block, sel, selected_size, flags); + } +} + void OrBlockColumnPredicate::evaluate_and(RowBlockV2 *block, uint16_t selected_size, bool* flags) const { if (num_of_column_predicate() == 1) { _block_column_predicate_vec[0]->evaluate_and(block, selected_size, flags); @@ -83,18 +134,47 @@ void OrBlockColumnPredicate::evaluate_and(RowBlockV2 *block, uint16_t selected_s } } +void OrBlockColumnPredicate::evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const { + if (num_of_column_predicate() == 1) { + _block_column_predicate_vec[0]->evaluate_and(block, sel, selected_size, flags); + } else { + bool ret_flags[selected_size]; + memset(ret_flags, false, selected_size); + for (int i = 0; i < num_of_column_predicate(); ++i) { + auto column_predicate = _block_column_predicate_vec[i]; + column_predicate->evaluate_or(block, sel, selected_size, ret_flags); + } + + for (int i = 0; i < selected_size; ++i) { + flags[i] &= ret_flags[i]; + } + } +} + void AndBlockColumnPredicate::evaluate(RowBlockV2* block, uint16_t* selected_size) const { for (auto block_column_predicate : _block_column_predicate_vec) { block_column_predicate->evaluate(block, selected_size); } } +void AndBlockColumnPredicate::evaluate(vectorized::MutableColumns& block, uint16_t* sel, uint16_t* selected_size) const { + for (auto block_column_predicate : _block_column_predicate_vec) { + block_column_predicate->evaluate(block, sel, selected_size); + } +} + void AndBlockColumnPredicate::evaluate_and(RowBlockV2 *block, uint16_t selected_size, bool* flags) const { for (auto block_column_predicate : _block_column_predicate_vec) { block_column_predicate->evaluate_and(block, selected_size, flags); } } +void AndBlockColumnPredicate::evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const { + for (auto block_column_predicate : _block_column_predicate_vec) { + block_column_predicate->evaluate_and(block, sel, selected_size, flags); + } +} + void AndBlockColumnPredicate::evaluate_or(RowBlockV2 *block, uint16_t selected_size, bool* flags) const { if (num_of_column_predicate() == 1) { _block_column_predicate_vec[0]->evaluate_or(block, selected_size, flags); @@ -113,4 +193,38 @@ void AndBlockColumnPredicate::evaluate_or(RowBlockV2 *block, uint16_t selected_s } } +void AndBlockColumnPredicate::evaluate_or(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const { + if (num_of_column_predicate() == 1) { + _block_column_predicate_vec[0]->evaluate_or(block, sel, selected_size, flags); + } else { + bool new_flags[selected_size]; + memset(new_flags, true, selected_size); + + for (auto block_column_predicate : _block_column_predicate_vec) { + block_column_predicate->evaluate_and(block, sel, selected_size, new_flags); + } + + for (uint16_t i = 0; i < selected_size; i++) { + flags[i] |= new_flags[i]; + } + } +} + +// todo(wb) Can the 'and' of multiple bitmaps be vectorized? +void AndBlockColumnPredicate::evaluate_vec(vectorized::MutableColumns& block, uint16_t size, bool* flags) const { + if (num_of_column_predicate() == 1) { + _block_column_predicate_vec[0]->evaluate_vec(block, size, flags); + } else { + bool new_flags[size]; + for (auto block_column_predicate : _block_column_predicate_vec) { + memset(new_flags, true, size); + block_column_predicate->evaluate_vec(block, size, new_flags); + + for (uint16_t j = 0; j < size; j++) { + flags[j] &= new_flags[j] ; + } + } + } +} + } // namespace doris diff --git a/be/src/olap/block_column_predicate.h b/be/src/olap/block_column_predicate.h index 4c0648699a..ef2a4ca038 100644 --- a/be/src/olap/block_column_predicate.h +++ b/be/src/olap/block_column_predicate.h @@ -44,6 +44,12 @@ public: virtual void evaluate_or(RowBlockV2* block, uint16_t selected_size, bool* flags) const = 0; virtual void get_all_column_ids(std::set& column_id_set) const = 0; + + virtual void evaluate(vectorized::MutableColumns& block, uint16_t* sel, uint16_t* selected_size) const {}; + virtual void evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const {}; + virtual void evaluate_or(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const {}; + + virtual void evaluate_vec(vectorized::MutableColumns& block, uint16_t size, bool* flags) const {}; }; class SingleColumnBlockPredicate : public BlockColumnPredicate { @@ -57,6 +63,13 @@ public: void get_all_column_ids(std::set& column_id_set) const override { column_id_set.insert(_predicate->column_id()); }; + + void evaluate(vectorized::MutableColumns& block, uint16_t* sel, uint16_t* selected_size) const override; + void evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const override; + void evaluate_or(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const override; + + void evaluate_vec(vectorized::MutableColumns& block, uint16_t size, bool* flags) const override; + private: const ColumnPredicate* _predicate; }; @@ -98,6 +111,12 @@ public: // 2.Do AND SEMANTICS in flags use 1 result to get proper select flags void evaluate_and(RowBlockV2* block, uint16_t selected_size, bool* flags) const override; void evaluate_or(RowBlockV2* block, uint16_t selected_size, bool* flags) const override; + + void evaluate(vectorized::MutableColumns& block, uint16_t* sel, uint16_t* selected_size) const override; + void evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const override; + void evaluate_or(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const override; + + // note(wb) we didnt't impelment evaluate_vec method here, because storage layer only support AND predicate now; }; class AndBlockColumnPredicate : public MutilColumnBlockPredicate { @@ -109,6 +128,13 @@ public: // 1.AndBlockColumnPredicate need evaluate all child BlockColumnPredicate AND SEMANTICS inside first // 2.Evaluate OR SEMANTICS in flags use 1 result to get proper select flags void evaluate_or(RowBlockV2* block, uint16_t selected_size, bool* flags) const override; + + void evaluate(vectorized::MutableColumns& block, uint16_t* sel, uint16_t* selected_size) const override; + void evaluate_and(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const override; + void evaluate_or(vectorized::MutableColumns& block, uint16_t* sel, uint16_t selected_size, bool* flags) const override; + + void evaluate_vec(vectorized::MutableColumns& block, uint16_t size, bool* flags) const override; + }; } //namespace doris diff --git a/be/src/olap/bloom_filter_predicate.h b/be/src/olap/bloom_filter_predicate.h index b3dcbbb644..ff3201c692 100644 --- a/be/src/olap/bloom_filter_predicate.h +++ b/be/src/olap/bloom_filter_predicate.h @@ -27,6 +27,10 @@ #include "olap/field.h" #include "runtime/string_value.hpp" #include "runtime/vectorized_row_batch.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/columns/predicate_column.h" +#include "vec/utils/util.hpp" namespace doris { @@ -59,12 +63,14 @@ public: return Status::OK(); } + void evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const override; + private: std::shared_ptr _filter; SpecificFilter* _specific_filter; // owned by _filter }; -// blomm filter column predicate do not support in segment v1 +// bloom filter column predicate do not support in segment v1 template void BloomFilterColumnPredicate::evaluate(VectorizedRowBatch* batch) const { uint16_t n = batch->size(); @@ -99,6 +105,38 @@ void BloomFilterColumnPredicate::evaluate(ColumnBlock* block, uint16_t* se *size = new_size; } +template +void BloomFilterColumnPredicate::evaluate(vectorized::IColumn& column, uint16_t* sel, + uint16_t* size) const { + uint16_t new_size = 0; + using T = typename PrimitiveTypeTraits::CppType; + + if (column.is_nullable()) { + auto* nullable_col = vectorized::check_and_get_column(column); + auto& null_map_data = nullable_col->get_null_map_column().get_data(); + auto* pred_col = vectorized::check_and_get_column>( + nullable_col->get_nested_column()); + auto& pred_col_data = pred_col->get_data(); + for (uint16_t i = 0; i < *size; i++) { + uint16_t idx = sel[i]; + sel[new_size] = idx; + const auto* cell_value = reinterpret_cast(&(pred_col_data[idx])); + new_size += (!null_map_data[idx]) && _specific_filter->find_olap_engine(cell_value); + } + } else { + auto* pred_col = + vectorized::check_and_get_column>(column); + auto& pred_col_data = pred_col->get_data(); + for (uint16_t i = 0; i < *size; i++) { + uint16_t idx = sel[i]; + sel[new_size] = idx; + const auto* cell_value = reinterpret_cast(&(pred_col_data[idx])); + new_size += _specific_filter->find_olap_engine(cell_value); + } + } + *size = new_size; +} + class BloomFilterColumnPredicateFactory { public: static ColumnPredicate* create_column_predicate( diff --git a/be/src/olap/collect_iterator.cpp b/be/src/olap/collect_iterator.cpp index 903ed0ac8c..46a7aa4436 100644 --- a/be/src/olap/collect_iterator.cpp +++ b/be/src/olap/collect_iterator.cpp @@ -32,7 +32,7 @@ CollectIterator::~CollectIterator() = default; void CollectIterator::init(TabletReader* reader) { _reader = reader; // when aggregate is enabled or key_type is DUP_KEYS, we don't merge - // multiple data to aggregate for performance in user fetch + // multiple data to aggregate for better performance if (_reader->_reader_type == READER_QUERY && (_reader->_aggregation || _reader->_tablet->keys_type() == KeysType::DUP_KEYS)) { _merge = false; diff --git a/be/src/olap/column_predicate.h b/be/src/olap/column_predicate.h index 21b115e821..6b1aa232a1 100644 --- a/be/src/olap/column_predicate.h +++ b/be/src/olap/column_predicate.h @@ -23,6 +23,7 @@ #include "olap/column_block.h" #include "olap/rowset/segment_v2/bitmap_index_reader.h" #include "olap/selection_vector.h" +#include "vec/columns/column.h" using namespace doris::segment_v2; @@ -54,8 +55,22 @@ public: const std::vector& iterators, uint32_t num_rows, roaring::Roaring* roaring) const = 0; + // evaluate predicate on IColumn + // a short circuit eval way + virtual void evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const {}; + virtual void evaluate_and(vectorized::IColumn& column, uint16_t* sel, uint16_t size, + bool* flags) const {}; + virtual void evaluate_or(vectorized::IColumn& column, uint16_t* sel, uint16_t size, + bool* flags) const {}; + + // used to evaluate pre read column in lazy matertialization + // now only support integer/float + // a vectorized eval way + virtual void evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) const {}; uint32_t column_id() const { return _column_id; } + virtual bool is_in_predicate() { return false; } + protected: uint32_t _column_id; bool _opposite; diff --git a/be/src/olap/comparison_predicate.cpp b/be/src/olap/comparison_predicate.cpp index 7dbe7cb096..a154a04499 100644 --- a/be/src/olap/comparison_predicate.cpp +++ b/be/src/olap/comparison_predicate.cpp @@ -21,6 +21,9 @@ #include "olap/schema.h" #include "runtime/string_value.hpp" #include "runtime/vectorized_row_batch.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/columns/predicate_column.h" namespace doris { @@ -142,6 +145,75 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(LessEqualPredicate, <=) COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(GreaterPredicate, >) COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(GreaterEqualPredicate, >=) +#define COMPARISON_PRED_COLUMN_EVALUATE(CLASS, OP) \ + template \ + void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const { \ + uint16_t new_size = 0; \ + if (column.is_nullable()) { \ + auto* nullable_column = vectorized::check_and_get_column(column);\ + auto& null_bitmap = reinterpret_cast&>(*(nullable_column->get_null_map_column_ptr())).get_data(); \ + auto* nest_column_vector = vectorized::check_and_get_column>(nullable_column->get_nested_column());\ + auto& data_array = nest_column_vector->get_data(); \ + for (uint16_t i = 0; i < *size; i++) { \ + uint16_t idx = sel[i]; \ + sel[new_size] = idx; \ + const type& cell_value = reinterpret_cast(data_array[idx]); \ + bool ret = !null_bitmap[idx] && (cell_value OP _value); \ + new_size += _opposite ? !ret : ret; \ + } \ + *size = new_size; \ + } else {\ + auto& pred_column_ref = reinterpret_cast&>(column);\ + auto& data_array = pred_column_ref.get_data(); \ + for (uint16_t i = 0; i < *size; i++) { \ + uint16_t idx = sel[i]; \ + sel[new_size] = idx; \ + const type& cell_value = reinterpret_cast(data_array[idx]); \ + auto ret = cell_value OP _value; \ + new_size += _opposite ? !ret : ret; \ + } \ + *size = new_size; \ + }\ + } + +COMPARISON_PRED_COLUMN_EVALUATE(EqualPredicate, ==) +COMPARISON_PRED_COLUMN_EVALUATE(NotEqualPredicate, !=) +COMPARISON_PRED_COLUMN_EVALUATE(LessPredicate, <) +COMPARISON_PRED_COLUMN_EVALUATE(LessEqualPredicate, <=) +COMPARISON_PRED_COLUMN_EVALUATE(GreaterPredicate, >) +COMPARISON_PRED_COLUMN_EVALUATE(GreaterEqualPredicate, >=) + +#define COMPARISON_PRED_COLUMN_EVALUATE_VEC(CLASS, OP) \ + template \ + void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) const { \ + if (column.is_nullable()) { \ + auto* nullable_column = vectorized::check_and_get_column(column); \ + auto& data_array = reinterpret_cast&>(nullable_column->get_nested_column()).get_data(); \ + auto& null_bitmap = reinterpret_cast&>(*(nullable_column->get_null_map_column_ptr())).get_data(); \ + for (uint16_t i = 0; i < size; i++) { \ + flags[i] = (data_array[i] OP _value) && (!null_bitmap[i]); \ + } \ + } else { \ + auto& predicate_column = reinterpret_cast&>(column); \ + auto& data_array = predicate_column.get_data(); \ + for (uint16_t i = 0; i < size; i++) { \ + flags[i] = data_array[i] OP _value; \ + } \ + } \ + if (_opposite) { \ + for (uint16_t i = 0; i < size; i++) { \ + flags[i] = !flags[i]; \ + } \ + } \ + } + +COMPARISON_PRED_COLUMN_EVALUATE_VEC(EqualPredicate, ==) +COMPARISON_PRED_COLUMN_EVALUATE_VEC(NotEqualPredicate, !=) +COMPARISON_PRED_COLUMN_EVALUATE_VEC(LessPredicate, <) +COMPARISON_PRED_COLUMN_EVALUATE_VEC(LessEqualPredicate, <=) +COMPARISON_PRED_COLUMN_EVALUATE_VEC(GreaterPredicate, >) +COMPARISON_PRED_COLUMN_EVALUATE_VEC(GreaterEqualPredicate, >=) + #define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(CLASS, OP) \ template \ void CLASS::evaluate_or(ColumnBlock* block, uint16_t* sel, uint16_t size, bool* flags) \ @@ -174,6 +246,20 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(LessEqualPredicate, <=) COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(GreaterPredicate, >) COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_OR(GreaterEqualPredicate, >=) +// todo(wb) support it +#define COMPARISON_PRED_COLUMN_EVALUATE_OR(CLASS, OP) \ + template \ + void CLASS::evaluate_or(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const { \ + \ + } + +COMPARISON_PRED_COLUMN_EVALUATE_OR(EqualPredicate, ==) +COMPARISON_PRED_COLUMN_EVALUATE_OR(NotEqualPredicate, !=) +COMPARISON_PRED_COLUMN_EVALUATE_OR(LessPredicate, <) +COMPARISON_PRED_COLUMN_EVALUATE_OR(LessEqualPredicate, <=) +COMPARISON_PRED_COLUMN_EVALUATE_OR(GreaterPredicate, >) +COMPARISON_PRED_COLUMN_EVALUATE_OR(GreaterEqualPredicate, >=) + #define COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(CLASS, OP) \ template \ void CLASS::evaluate_and(ColumnBlock* block, uint16_t* sel, uint16_t size, bool* flags) \ @@ -206,6 +292,21 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(LessEqualPredicate, <=) COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(GreaterPredicate, >) COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_AND(GreaterEqualPredicate, >=) +//todo(wb) support it +#define COMPARISON_PRED_COLUMN_EVALUATE_AND(CLASS, OP) \ + template \ + void CLASS::evaluate_and(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const { \ + \ + \ + } + +COMPARISON_PRED_COLUMN_EVALUATE_AND(EqualPredicate, ==) +COMPARISON_PRED_COLUMN_EVALUATE_AND(NotEqualPredicate, !=) +COMPARISON_PRED_COLUMN_EVALUATE_AND(LessPredicate, <) +COMPARISON_PRED_COLUMN_EVALUATE_AND(LessEqualPredicate, <=) +COMPARISON_PRED_COLUMN_EVALUATE_AND(GreaterPredicate, >) +COMPARISON_PRED_COLUMN_EVALUATE_AND(GreaterEqualPredicate, >=) + #define BITMAP_COMPARE_EqualPredicate(s, exact_match, seeked_ordinal, iterator, bitmap, roaring) \ do { \ if (!s.is_not_found()) { \ @@ -440,4 +541,66 @@ COMPARISON_PRED_BITMAP_EVALUATE_DECLARATION(LessEqualPredicate) COMPARISON_PRED_BITMAP_EVALUATE_DECLARATION(GreaterPredicate) COMPARISON_PRED_BITMAP_EVALUATE_DECLARATION(GreaterEqualPredicate) +#define COMPARISON_PRED_COLUMN_EVALUATE_DECLARATION(CLASS) \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) \ + const; \ + template void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const; + +COMPARISON_PRED_COLUMN_EVALUATE_DECLARATION(EqualPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_DECLARATION(NotEqualPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_DECLARATION(LessPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_DECLARATION(LessEqualPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_DECLARATION(GreaterPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_DECLARATION(GreaterEqualPredicate) + +#define COMPARISON_PRED_COLUMN_EVALUATE_VEC_DECLARATION(CLASS) \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) \ + const; \ + template void CLASS::evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) const; + +COMPARISON_PRED_COLUMN_EVALUATE_VEC_DECLARATION(EqualPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_VEC_DECLARATION(NotEqualPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_VEC_DECLARATION(LessPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_VEC_DECLARATION(LessEqualPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_VEC_DECLARATION(GreaterPredicate) +COMPARISON_PRED_COLUMN_EVALUATE_VEC_DECLARATION(GreaterEqualPredicate) + } //namespace doris diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h index d0a40490eb..30fd9fde80 100644 --- a/be/src/olap/comparison_predicate.h +++ b/be/src/olap/comparison_predicate.h @@ -40,6 +40,10 @@ class VectorizedRowBatch; virtual Status evaluate(const Schema& schema, \ const std::vector& iterators, \ uint32_t num_rows, roaring::Roaring* roaring) const override; \ + void evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const override; \ + void evaluate_and(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const override; \ + void evaluate_or(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const override; \ + void evaluate_vec(vectorized::IColumn& column, uint16_t size, bool* flags) const override; \ \ private: \ type _value; \ diff --git a/be/src/olap/generic_iterators.cpp b/be/src/olap/generic_iterators.cpp index adac5df916..8a11108c93 100644 --- a/be/src/olap/generic_iterators.cpp +++ b/be/src/olap/generic_iterators.cpp @@ -116,6 +116,11 @@ public: MergeIteratorContext(RowwiseIterator* iter, std::shared_ptr parent) : _iter(iter), _block(iter->schema(), 1024, std::move(parent)) {} + MergeIteratorContext(const MergeIteratorContext&) = delete; + MergeIteratorContext(MergeIteratorContext&&) = delete; + MergeIteratorContext& operator=(const MergeIteratorContext&) = delete; + MergeIteratorContext& operator=(MergeIteratorContext&&) = delete; + ~MergeIteratorContext() { delete _iter; _iter = nullptr; @@ -151,6 +156,7 @@ private: private: RowwiseIterator* _iter; + // used to store data load from iterator RowBlockV2 _block; @@ -181,10 +187,9 @@ Status MergeIteratorContext::advance() { } Status MergeIteratorContext::_load_next_block() { - Status st; do { _block.clear(); - st = _iter->next_batch(&_block); + Status st = _iter->next_batch(&_block); if (!st.ok()) { _valid = false; if (st.is_end_of_file()) { @@ -202,27 +207,29 @@ Status MergeIteratorContext::_load_next_block() { class MergeIterator : public RowwiseIterator { public: // MergeIterator takes the ownership of input iterators - MergeIterator(std::list iters, std::shared_ptr parent, int sequence_id_idx) - : _origin_iters(std::move(iters)), _sequence_id_idx(sequence_id_idx) { + MergeIterator(std::vector iters, std::shared_ptr parent, int sequence_id_idx) + : _origin_iters(std::move(iters)), _sequence_id_idx(sequence_id_idx), _merge_heap(MergeContextComparator(_sequence_id_idx)) { // use for count the mem use of Block use in Merge _mem_tracker = MemTracker::CreateTracker(-1, "MergeIterator", parent, false); } ~MergeIterator() override { - while (!_merge_heap->empty()) { - auto ctx = _merge_heap->top(); - _merge_heap->pop(); + while (!_merge_heap.empty()) { + auto ctx = _merge_heap.top(); + _merge_heap.pop(); delete ctx; } } + Status init(const StorageReadOptions& opts) override; + Status next_batch(RowBlockV2* block) override; const Schema& schema() const override { return *_schema; } private: // It will be released after '_merge_heap' has been built. - std::list _origin_iters; + std::vector _origin_iters; int _sequence_id_idx; @@ -256,9 +263,12 @@ private: int sequence_id_idx; }; - using MergeHeap = std::priority_queue, - MergeContextComparator>; - std::unique_ptr _merge_heap; + + using MergeHeap = std::priority_queue, + MergeContextComparator>; + + MergeHeap _merge_heap; }; Status MergeIterator::init(const StorageReadOptions& opts) { @@ -267,24 +277,24 @@ Status MergeIterator::init(const StorageReadOptions& opts) { } _schema.reset(new Schema((*(_origin_iters.begin()))->schema())); - _merge_heap = std::make_unique(MergeContextComparator(_sequence_id_idx)); for (auto iter : _origin_iters) { std::unique_ptr ctx(new MergeIteratorContext(iter, _mem_tracker)); RETURN_IF_ERROR(ctx->init(opts)); if (!ctx->valid()) { continue; } - _merge_heap->push(ctx.release()); + _merge_heap.push(ctx.release()); } + _origin_iters.clear(); return Status::OK(); } Status MergeIterator::next_batch(RowBlockV2* block) { size_t row_idx = 0; - for (; row_idx < block->capacity() && !_merge_heap->empty(); ++row_idx) { - auto ctx = _merge_heap->top(); - _merge_heap->pop(); + for (; row_idx < block->capacity() && !_merge_heap.empty(); ++row_idx) { + auto ctx = _merge_heap.top(); + _merge_heap.pop(); RowBlockRow dst_row = block->row(row_idx); // copy current row to block @@ -292,7 +302,7 @@ Status MergeIterator::next_batch(RowBlockV2* block) { RETURN_IF_ERROR(ctx->advance()); if (ctx->valid()) { - _merge_heap->push(ctx); + _merge_heap.push(ctx); } else { // Release ctx earlier to reduce resource consumed delete ctx; @@ -313,17 +323,17 @@ public: // Iterators' ownership it transfered to this class. // This class will delete all iterators when destructs // Client should not use iterators any more. - UnionIterator(std::list iters, std::shared_ptr parent) - : _origin_iters(std::move(iters)) { + UnionIterator(std::vector &v, std::shared_ptr parent) + : _origin_iters(v.begin(), v.end()) { _mem_tracker = MemTracker::CreateTracker(-1, "UnionIterator", parent, false); } ~UnionIterator() override { - for (auto iter : _origin_iters) { - delete iter; - } + std::for_each(_origin_iters.begin(), _origin_iters.end(), std::default_delete()); } + Status init(const StorageReadOptions& opts) override; + Status next_batch(RowBlockV2* block) override; const Schema& schema() const override { return *_schema; } @@ -331,7 +341,7 @@ public: private: std::unique_ptr _schema; RowwiseIterator* _cur_iter = nullptr; - std::list _origin_iters; + std::deque _origin_iters; }; Status UnionIterator::init(const StorageReadOptions& opts) { @@ -364,18 +374,18 @@ Status UnionIterator::next_batch(RowBlockV2* block) { return Status::EndOfFile("End of UnionIterator"); } -RowwiseIterator* new_merge_iterator(std::list inputs, std::shared_ptr parent, int sequence_id_idx) { +RowwiseIterator* new_merge_iterator(std::vector inputs, std::shared_ptr parent, int sequence_id_idx) { if (inputs.size() == 1) { return *(inputs.begin()); } return new MergeIterator(std::move(inputs), parent, sequence_id_idx); } -RowwiseIterator* new_union_iterator(std::list inputs, std::shared_ptr parent) { +RowwiseIterator* new_union_iterator(std::vector& inputs, std::shared_ptr parent) { if (inputs.size() == 1) { return *(inputs.begin()); } - return new UnionIterator(std::move(inputs), parent); + return new UnionIterator(inputs, parent); } RowwiseIterator* new_auto_increment_iterator(const Schema& schema, size_t num_rows) { diff --git a/be/src/olap/generic_iterators.h b/be/src/olap/generic_iterators.h index 1d3eccf30a..e8f4528885 100644 --- a/be/src/olap/generic_iterators.h +++ b/be/src/olap/generic_iterators.h @@ -25,14 +25,14 @@ namespace doris { // // Inputs iterators' ownership is taken by created merge iterator. And client // should delete returned iterator after usage. -RowwiseIterator* new_merge_iterator(std::list inputs, std::shared_ptr parent, int sequence_id_idx); +RowwiseIterator* new_merge_iterator(std::vector inputs, std::shared_ptr parent, int sequence_id_idx); // Create a union iterator for input iterators. Union iterator will read // input iterators one by one. // // Inputs iterators' ownership is taken by created union iterator. And client // should delete returned iterator after usage. -RowwiseIterator* new_union_iterator(std::list inputs, std::shared_ptr parent); +RowwiseIterator* new_union_iterator(std::vector& inputs, std::shared_ptr parent); // Create an auto increment iterator which returns num_rows data in format of schema. // This class aims to be used in unit test. diff --git a/be/src/olap/in_list_predicate.cpp b/be/src/olap/in_list_predicate.cpp index c167a17e4c..a17e157c4d 100644 --- a/be/src/olap/in_list_predicate.cpp +++ b/be/src/olap/in_list_predicate.cpp @@ -20,6 +20,8 @@ #include "olap/field.h" #include "runtime/string_value.hpp" #include "runtime/vectorized_row_batch.h" +#include "vec/columns/predicate_column.h" +#include "vec/columns/column_nullable.h" namespace doris { @@ -115,6 +117,43 @@ IN_LIST_PRED_EVALUATE(NotInListPredicate, ==) IN_LIST_PRED_COLUMN_BLOCK_EVALUATE(InListPredicate, !=) IN_LIST_PRED_COLUMN_BLOCK_EVALUATE(NotInListPredicate, ==) +#define IN_LIST_PRED_COLUMN_EVALUATE(CLASS, OP) \ + template \ + void CLASS::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const { \ + uint16_t new_size = 0; \ + if (column.is_nullable()) { \ + auto* nullable_column = \ + vectorized::check_and_get_column(column); \ + auto& null_bitmap = reinterpret_cast&>(*( \ + nullable_column->get_null_map_column_ptr())).get_data(); \ + auto* nest_column_vector = vectorized::check_and_get_column \ + >(nullable_column->get_nested_column()); \ + auto& data_array = nest_column_vector->get_data(); \ + for (uint16_t i = 0; i < *size; i++) { \ + uint16_t idx = sel[i]; \ + sel[new_size] = idx; \ + const type& cell_value = reinterpret_cast(data_array[idx]); \ + bool ret = !null_bitmap[idx] && (_values.find(cell_value) OP _values.end()); \ + new_size += _opposite ? !ret : ret; \ + } \ + *size = new_size; \ + } else { \ + auto& number_column = reinterpret_cast&>(column);\ + auto& data_array = number_column.get_data(); \ + for (uint16_t i = 0; i < *size; i++) { \ + uint16_t idx = sel[i]; \ + sel[new_size] = idx; \ + const type& cell_value = reinterpret_cast(data_array[idx]); \ + auto result = (_values.find(cell_value) OP _values.end()); \ + new_size += _opposite ? !result : result; \ + } \ + } \ + *size = new_size; \ + } + +IN_LIST_PRED_COLUMN_EVALUATE(InListPredicate, !=) +IN_LIST_PRED_COLUMN_EVALUATE(NotInListPredicate, ==) + #define IN_LIST_PRED_COLUMN_BLOCK_EVALUATE_OR(CLASS, OP) \ template \ void CLASS::evaluate_or(ColumnBlock* block, uint16_t* sel, uint16_t size, bool* flags) \ diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h index cf9bf615e5..7cd237b7eb 100644 --- a/be/src/olap/in_list_predicate.h +++ b/be/src/olap/in_list_predicate.h @@ -76,6 +76,8 @@ namespace doris { class VectorizedRowBatch; +// todo(wb) support evaluate_and,evaluate_or + #define IN_LIST_PRED_CLASS_DEFINE(CLASS) \ template \ class CLASS : public ColumnPredicate { \ @@ -90,7 +92,10 @@ class VectorizedRowBatch; virtual Status evaluate(const Schema& schema, \ const std::vector& iterators, \ uint32_t num_rows, roaring::Roaring* bitmap) const override; \ - \ + void evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const override; \ + void evaluate_and(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const override {} \ + void evaluate_or(vectorized::IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const override {} \ + bool is_in_predicate() override { return true; } \ private: \ phmap::flat_hash_set _values; \ }; diff --git a/be/src/olap/iterators.h b/be/src/olap/iterators.h index d514dc0425..4cdfc60578 100644 --- a/be/src/olap/iterators.h +++ b/be/src/olap/iterators.h @@ -23,6 +23,7 @@ #include "olap/olap_common.h" #include "olap/column_predicate.h" #include "olap/block_column_predicate.h" +#include "vec/core/block.h" namespace doris { @@ -80,6 +81,7 @@ public: // REQUIRED (null is not allowed) OlapReaderStatistics* stats = nullptr; bool use_page_cache = false; + int block_row_max = 4096; }; // Used to read data in RowBlockV2 one by one @@ -99,7 +101,9 @@ public: // into input batch with Status::OK() returned // If there is no data to read, will return Status::EndOfFile. // If other error happens, other error code will be returned. - virtual Status next_batch(RowBlockV2* block) = 0; + virtual Status next_batch(RowBlockV2* block) { return Status::NotSupported("to be implemented"); } + + virtual Status next_batch(vectorized::Block* block) { return Status::NotSupported("to be implemented"); } // return schema for this Iterator virtual const Schema& schema() const = 0; diff --git a/be/src/olap/null_predicate.cpp b/be/src/olap/null_predicate.cpp index cc88d261eb..da3eb295bc 100644 --- a/be/src/olap/null_predicate.cpp +++ b/be/src/olap/null_predicate.cpp @@ -20,6 +20,9 @@ #include "olap/field.h" #include "runtime/string_value.hpp" #include "runtime/vectorized_row_batch.h" +#include "vec/columns/column_nullable.h" + +using namespace doris::vectorized; namespace doris { @@ -118,4 +121,44 @@ Status NullPredicate::evaluate(const Schema& schema, return Status::OK(); } +void NullPredicate::evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const { + uint16_t new_size = 0; + if (auto* nullable = check_and_get_column(column)) { + auto& null_map = nullable->get_null_map_data(); + for (uint16_t i = 0; i < *size; ++i) { + uint16_t idx = sel[i]; + sel[new_size] = idx; + new_size += (null_map[idx] == _is_null); + } + *size = new_size; + } else { + if (_is_null) *size = 0; + } +} + +void NullPredicate::evaluate_or(IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const { + if (auto* nullable = check_and_get_column(column)) { + auto& null_map = nullable->get_null_map_data(); + for (uint16_t i = 0; i < size; ++i) { + if (flags[i]) continue; + uint16_t idx = sel[i]; + flags[i] |= (null_map[idx] == _is_null); + } + } else { + if (!_is_null) memset(flags, true, size); + } +} + +void NullPredicate::evaluate_and(IColumn& column, uint16_t* sel, uint16_t size, bool* flags) const { + if (auto* nullable = check_and_get_column(column)) { + auto& null_map = nullable->get_null_map_data(); + for (uint16_t i = 0; i < size; ++i) { + if (flags[i]) continue; + uint16_t idx = sel[i]; + flags[i] &= (null_map[idx] == _is_null); + } + } else { + if (_is_null) memset(flags, false, size); + } +} } //namespace doris diff --git a/be/src/olap/null_predicate.h b/be/src/olap/null_predicate.h index e867d58fe4..681e60bc33 100644 --- a/be/src/olap/null_predicate.h +++ b/be/src/olap/null_predicate.h @@ -43,6 +43,14 @@ public: virtual Status evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, roaring::Roaring* roaring) const override; + void evaluate(vectorized::IColumn& column, uint16_t* sel, uint16_t* size) const override; + + void evaluate_or(vectorized::IColumn& column, uint16_t* sel, uint16_t size, + bool* flags) const override; + + void evaluate_and(vectorized::IColumn& column, uint16_t* sel, uint16_t size, + bool* flags) const override; + private: bool _is_null; //true for null, false for not null }; diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index c2d4b7f3c3..a9ac731333 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -384,7 +384,8 @@ enum OLAPStatus { OLAP_ERR_ROWSET_LOAD_FAILED = -3109, OLAP_ERR_ROWSET_READER_INIT = -3110, OLAP_ERR_ROWSET_READ_FAILED = -3111, - OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION = -3112 + OLAP_ERR_ROWSET_INVALID_STATE_TRANSITION = -3112, + OLAP_ERR_STRING_OVERFLOW_IN_VEC_ENGINE = -3113 }; enum ColumnFamilyIndex { diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp index 763d1e7a24..13e50b80c6 100644 --- a/be/src/olap/reader.cpp +++ b/be/src/olap/reader.cpp @@ -41,6 +41,7 @@ #include "runtime/string_value.hpp" #include "util/date_func.h" #include "util/mem_util.hpp" +#include "vec/olap/vcollect_iterator.h" using std::nothrow; using std::set; @@ -57,8 +58,8 @@ void TabletReader::ReaderParams::check_validation() const { std::string TabletReader::ReaderParams::to_string() const { std::stringstream ss; ss << "tablet=" << tablet->full_name() << " reader_type=" << reader_type - << " aggregation=" << aggregation << " version=" << version << " start_key_include=" << start_key_include - << " end_key_include=" << end_key_include; + << " aggregation=" << aggregation << " version=" << version + << " start_key_include=" << start_key_include << " end_key_include=" << end_key_include; for (const auto& key : start_key) { ss << " keys=" << key; @@ -230,6 +231,7 @@ OLAPStatus TabletReader::_capture_rs_readers(const ReaderParams& read_params, OLAPStatus TabletReader::_init_params(const ReaderParams& read_params) { read_params.check_validation(); + _direct_mode = read_params.direct_mode; _aggregation = read_params.aggregation; _need_agg_finalize = read_params.need_agg_finalize; _reader_type = read_params.reader_type; @@ -699,7 +701,7 @@ ColumnPredicate* TabletReader::_parse_to_predicate(const TCondition& condition, break; } case OLAP_FIELD_TYPE_VARCHAR: - case OLAP_FIELD_TYPE_STRING:{ + case OLAP_FIELD_TYPE_STRING: { phmap::flat_hash_set values; for (auto& cond_val : condition.condition_values) { StringValue value; @@ -806,7 +808,8 @@ void TabletReader::_init_load_bf_columns(const ReaderParams& read_params, Condit return; } FieldType type = _tablet->tablet_schema().column(max_equal_index).type(); - if ((type != OLAP_FIELD_TYPE_VARCHAR && type != OLAP_FIELD_TYPE_STRING)|| max_equal_index + 1 > _tablet->num_short_key_columns()) { + if ((type != OLAP_FIELD_TYPE_VARCHAR && type != OLAP_FIELD_TYPE_STRING) || + max_equal_index + 1 > _tablet->num_short_key_columns()) { load_bf_columns->erase(max_equal_index); } } diff --git a/be/src/olap/reader.h b/be/src/olap/reader.h index 0f20895069..82cd7ffcf4 100644 --- a/be/src/olap/reader.h +++ b/be/src/olap/reader.h @@ -49,6 +49,11 @@ class RowBlock; class CollectIterator; class RuntimeState; +namespace vectorized { +class VCollectIterator; +class Block; +} // namespace vectorized + class TabletReader { struct KeysParam { std::string to_string() const; @@ -64,6 +69,7 @@ public: struct ReaderParams { TabletSharedPtr tablet; ReaderType reader_type = READER_QUERY; + bool direct_mode = false; bool aggregation = false; bool need_agg_finalize = true; // 1. when read column data page: @@ -88,6 +94,9 @@ public: RuntimeProfile* profile = nullptr; RuntimeState* runtime_state = nullptr; + // use only in vec exec engine + std::vector* origin_return_columns = nullptr; + void check_validation() const; std::string to_string() const; @@ -105,7 +114,17 @@ public: // Return OLAP_SUCCESS and set `*eof` to true when no more rows can be read. // Return others when unexpected error happens. virtual OLAPStatus next_row_with_aggregation(RowCursor* row_cursor, MemPool* mem_pool, - ObjectPool* agg_pool, bool* eof) = 0; + ObjectPool* agg_pool, bool* eof) = 0; + + // Read next block with aggregation. + // Return OLAP_SUCCESS and set `*eof` to false when next block is read + // Return OLAP_SUCCESS and set `*eof` to true when no more rows can be read. + // Return others when unexpected error happens. + // TODO: Rethink here we still need mem_pool and agg_pool? + virtual OLAPStatus next_block_with_aggregation(vectorized::Block* block, MemPool* mem_pool, + ObjectPool* agg_pool, bool* eof) { + return OLAP_ERR_READER_INITIALIZE_ERROR; + } uint64_t merged_rows() const { return _merged_rows; } @@ -119,6 +138,7 @@ public: protected: friend class CollectIterator; + friend class vectorized::VCollectIterator; friend class DeleteHandler; OLAPStatus _init_params(const ReaderParams& read_params); @@ -188,8 +208,8 @@ protected: ReaderType _reader_type = READER_QUERY; bool _next_delete_flag = false; bool _filter_delete = false; - bool _has_sequence_col = false; int32_t _sequence_col_idx = -1; + bool _direct_mode = false; CollectIterator _collect_iter; std::vector _key_cids; diff --git a/be/src/olap/row_block2.cpp b/be/src/olap/row_block2.cpp index 20ed4ae029..877f6a21a2 100644 --- a/be/src/olap/row_block2.cpp +++ b/be/src/olap/row_block2.cpp @@ -23,6 +23,11 @@ #include "gutil/strings/substitute.h" #include "olap/row_cursor.h" #include "util/bitmap.h" +#include "vec/columns/column_complex.h" +#include "vec/columns/column_vector.h" +#include "vec/core/block.h" +#include "vec/core/types.h" +#include "vec/runtime/vdatetime_value.h" using strings::Substitute; namespace doris { @@ -90,6 +95,231 @@ Status RowBlockV2::convert_to_row_block(RowCursor* helper, RowBlock* dst) { return Status::OK(); } +Status RowBlockV2::_copy_data_to_column(int cid, doris::vectorized::MutableColumnPtr& origin_column) { + constexpr auto MAX_SIZE_OF_VEC_STRING = 1024l * 1024; + + auto* column = origin_column.get(); + bool nullable_mark_array[_selected_size]; + + bool column_nullable = origin_column->is_nullable(); + bool origin_nullable = _schema.column(cid)->is_nullable(); + if (column_nullable) { + auto nullable_column = assert_cast(origin_column.get()); + auto& null_map = nullable_column->get_null_map_data(); + column = nullable_column->get_nested_column_ptr().get(); + + if (origin_nullable) { + for (uint16_t i = 0; i < _selected_size; ++i) { + uint16_t row_idx = _selection_vector[i]; + null_map.push_back(_column_vector_batches[cid]->is_null_at(row_idx)); + nullable_mark_array[i] = null_map.back(); + } + } else { + null_map.resize_fill(null_map.size() + _selected_size, 0); + memset(nullable_mark_array, false, _selected_size * sizeof(bool)); + } + } else { + memset(nullable_mark_array, false, _selected_size * sizeof(bool)); + } + + auto insert_data_directly = [this, &nullable_mark_array](int cid, auto& column) { + for (uint16_t j = 0; j < _selected_size; ++j) { + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + column->insert_data( + reinterpret_cast(column_block(cid).cell_ptr(row_idx)), 0); + } else { + column->insert_default(); + } + } + }; + + switch (_schema.column(cid)->type()) { + case OLAP_FIELD_TYPE_OBJECT: { + auto column_bitmap = assert_cast(column); + for (uint16_t j = 0; j < _selected_size; ++j) { + column_bitmap->insert_default(); + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + auto slice = reinterpret_cast(column_block(cid).cell_ptr(row_idx)); + + BitmapValue* pvalue = &column_bitmap->get_element(column_bitmap->size() - 1); + + if (slice->size != 0) { + BitmapValue value; + value.deserialize(slice->data); + *pvalue = std::move(value); + } else { + *pvalue = std::move(*reinterpret_cast(slice->data)); + } + } + } + break; + } + case OLAP_FIELD_TYPE_HLL: + case OLAP_FIELD_TYPE_MAP: + case OLAP_FIELD_TYPE_VARCHAR: { + auto column_string = assert_cast(column); + + for (uint16_t j = 0; j < _selected_size; ++j) { + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + auto slice = reinterpret_cast(column_block(cid).cell_ptr(row_idx)); + column_string->insert_data(slice->data, slice->size); + } else { + column_string->insert_default(); + } + } + break; + } + case OLAP_FIELD_TYPE_STRING: { + auto column_string = assert_cast(column); + + for (uint16_t j = 0; j < _selected_size; ++j) { + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + auto slice = reinterpret_cast(column_block(cid).cell_ptr(row_idx)); + if (LIKELY(slice->size <= MAX_SIZE_OF_VEC_STRING)) { + column_string->insert_data(slice->data, slice->size); + } else { + return Status::NotSupported("Not support string len over than 1MB in vec engine."); + } + } else { + column_string->insert_default(); + } + } + break; + } + case OLAP_FIELD_TYPE_CHAR: { + auto column_string = assert_cast(column); + + for (uint16_t j = 0; j < _selected_size; ++j) { + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + auto slice = reinterpret_cast(column_block(cid).cell_ptr(row_idx)); + column_string->insert_data(slice->data, strnlen(slice->data, slice->size)); + } else { + column_string->insert_default(); + } + } + break; + } + case OLAP_FIELD_TYPE_DATE: { + auto column_int = assert_cast*>(column); + + for (uint16_t j = 0; j < _selected_size; ++j) { + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + auto ptr = reinterpret_cast(column_block(cid).cell_ptr(row_idx)); + + uint64_t value = 0; + value = *(unsigned char*)(ptr + 2); + value <<= 8; + value |= *(unsigned char*)(ptr + 1); + value <<= 8; + value |= *(unsigned char*)(ptr); + vectorized::VecDateTimeValue date; + date.from_olap_date(value); + (column_int)->insert_data(reinterpret_cast(&date), 0); + } else + column_int->insert_default(); + } + break; + } + case OLAP_FIELD_TYPE_DATETIME: { + auto column_int = assert_cast*>(column); + + for (uint16_t j = 0; j < _selected_size; ++j) { + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + auto ptr = reinterpret_cast(column_block(cid).cell_ptr(row_idx)); + + uint64_t value = *reinterpret_cast(ptr); + vectorized::VecDateTimeValue data(value); + (column_int)->insert_data(reinterpret_cast(&data), 0); + } else { + column_int->insert_default(); + } + } + break; + } + case OLAP_FIELD_TYPE_DECIMAL: { + auto column_decimal = + assert_cast*>(column); + + for (uint16_t j = 0; j < _selected_size; ++j) { + if (!nullable_mark_array[j]) { + uint16_t row_idx = _selection_vector[j]; + auto ptr = reinterpret_cast(column_block(cid).cell_ptr(row_idx)); + + int64_t int_value = *(int64_t*)(ptr); + int32_t frac_value = *(int32_t*)(ptr + sizeof(int64_t)); + DecimalV2Value data(int_value, frac_value); + column_decimal->insert_data(reinterpret_cast(&data), 0); + } else { + column_decimal->insert_default(); + } + } + break; + } + case OLAP_FIELD_TYPE_INT: { + auto column_int = assert_cast*>(column); + insert_data_directly(cid, column_int); + break; + } + case OLAP_FIELD_TYPE_BOOL: { + auto column_int = assert_cast*>(column); + insert_data_directly(cid, column_int); + break; + } + case OLAP_FIELD_TYPE_TINYINT: { + auto column_int = assert_cast*>(column); + insert_data_directly(cid, column_int); + break; + } + case OLAP_FIELD_TYPE_SMALLINT: { + auto column_int = assert_cast*>(column); + insert_data_directly(cid, column_int); + break; + } + case OLAP_FIELD_TYPE_BIGINT: { + auto column_int = assert_cast*>(column); + insert_data_directly(cid, column_int); + break; + } + case OLAP_FIELD_TYPE_LARGEINT: { + auto column_int = assert_cast*>(column); + insert_data_directly(cid, column_int); + break; + } + case OLAP_FIELD_TYPE_FLOAT: { + auto column_float = assert_cast*>(column); + insert_data_directly(cid, column_float); + break; + } + case OLAP_FIELD_TYPE_DOUBLE: { + auto column_float = assert_cast*>(column); + insert_data_directly(cid, column_float); + break; + } + default: { + DCHECK(false) << "Invalid type in RowBlockV2:" << _schema.column(cid)->type(); + } + } + + return Status::OK(); +} + +Status RowBlockV2::convert_to_vec_block(vectorized::Block* block) { + for (int i = 0; i < _schema.column_ids().size(); ++i) { + auto cid = _schema.column_ids()[i]; + auto column = (*std::move(block->get_by_position(i).column)).assume_mutable(); + RETURN_IF_ERROR(_copy_data_to_column(cid, column)); + } + _pool->clear(); + return Status::OK(); +} + std::string RowBlockRow::debug_string() const { std::stringstream ss; ss << "["; diff --git a/be/src/olap/row_block2.h b/be/src/olap/row_block2.h index ece8d725a0..b98ab954c3 100644 --- a/be/src/olap/row_block2.h +++ b/be/src/olap/row_block2.h @@ -73,6 +73,9 @@ public: // convert RowBlockV2 to RowBlock Status convert_to_row_block(RowCursor* helper, RowBlock* dst); + // convert RowBlockV2 to vectorized::Block + Status convert_to_vec_block(vectorized::Block* block); + // low-level API to access memory for each column block(including data array and nullmap). // `cid` must be one of `schema()->column_ids()`. ColumnBlock column_block(ColumnId cid) const { @@ -106,6 +109,8 @@ public: std::string debug_string(); private: + Status _copy_data_to_column(int cid, vectorized::MutableColumnPtr& mutable_column_ptr); + Schema _schema; size_t _capacity; // _column_vector_batches[cid] == null if cid is not in `_schema`. diff --git a/be/src/olap/rowset/alpha_rowset_reader.h b/be/src/olap/rowset/alpha_rowset_reader.h index 959a90bc88..fd90f37a96 100644 --- a/be/src/olap/rowset/alpha_rowset_reader.h +++ b/be/src/olap/rowset/alpha_rowset_reader.h @@ -65,6 +65,10 @@ public: // It's ok, because we only get ref here, the block's owner is this reader. OLAPStatus next_block(RowBlock** block) override; + OLAPStatus next_block(vectorized::Block *block) override { + return OLAP_ERR_DATA_EOF; + } + bool delete_flag() override; Version version() override; diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp index ada5157e76..4d35f2f31e 100644 --- a/be/src/olap/rowset/beta_rowset_reader.cpp +++ b/be/src/olap/rowset/beta_rowset_reader.cpp @@ -27,6 +27,8 @@ #include "olap/rowset/segment_v2/segment_iterator.h" #include "olap/schema.h" +#include "vec/core/block.h" + namespace doris { BetaRowsetReader::BetaRowsetReader(BetaRowsetSharedPtr rowset, @@ -107,7 +109,8 @@ OLAPStatus BetaRowsetReader::init(RowsetReaderContext* read_context) { } seg_iterators.push_back(std::move(iter)); } - std::list iterators; + + std::vector iterators; for (auto& owned_it : seg_iterators) { // transfer ownership of segment iterator to `_iterator` iterators.push_back(owned_it.release()); @@ -172,4 +175,45 @@ OLAPStatus BetaRowsetReader::next_block(RowBlock** block) { return OLAP_SUCCESS; } +OLAPStatus BetaRowsetReader::next_block(vectorized::Block* block) { + SCOPED_RAW_TIMER(&_stats->block_fetch_ns); + bool is_first = true; + + do { + // read next input block + { + _input_block->clear(); + { + auto s = _iterator->next_batch(_input_block.get()); + if (!s.ok()) { + if (s.is_end_of_file()) { + if (is_first) { + return OLAP_ERR_DATA_EOF; + } else { + break; + } + } else { + LOG(WARNING) << "failed to read next block: " << s.to_string(); + return OLAP_ERR_ROWSET_READ_FAILED; + } + } else if (_input_block->selected_size() == 0) { + continue; + } + } + } + + { + SCOPED_RAW_TIMER(&_stats->block_convert_ns); + auto s = _input_block->convert_to_vec_block(block); + if (UNLIKELY(!s.ok())) { + LOG(WARNING) << "failed to read next block: " << s.to_string(); + return OLAP_ERR_STRING_OVERFLOW_IN_VEC_ENGINE; + } + } + is_first = false; + } while (block->rows() < _context->runtime_state->batch_size()); // here we should keep block.rows() < batch_size + + return OLAP_SUCCESS; +} + } // namespace doris diff --git a/be/src/olap/rowset/beta_rowset_reader.h b/be/src/olap/rowset/beta_rowset_reader.h index 34e7d9cf3d..6074308e48 100644 --- a/be/src/olap/rowset/beta_rowset_reader.h +++ b/be/src/olap/rowset/beta_rowset_reader.h @@ -40,6 +40,7 @@ public: // If parent_tracker is not null, the block we get from next_block() will have the parent_tracker. // It's ok, because we only get ref here, the block's owner is this reader. OLAPStatus next_block(RowBlock** block) override; + OLAPStatus next_block(vectorized::Block* block) override; bool delete_flag() override { return _rowset->delete_flag(); } diff --git a/be/src/olap/rowset/rowset_reader.h b/be/src/olap/rowset/rowset_reader.h index b9e46d1118..53af387b3b 100644 --- a/be/src/olap/rowset/rowset_reader.h +++ b/be/src/olap/rowset/rowset_reader.h @@ -23,9 +23,14 @@ #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_reader_context.h" +#include "vec/core/block.h" namespace doris { +namespace vectorized { +class Block; +} + class RowBlock; class RowsetReader; using RowsetReaderSharedPtr = std::shared_ptr; @@ -33,6 +38,7 @@ using RowsetReaderSharedPtr = std::shared_ptr; class RowsetReader { public: enum RowsetReaderType { ALPHA, BETA }; + virtual ~RowsetReader() {} // reader init @@ -45,6 +51,8 @@ public: // Others when error happens. virtual OLAPStatus next_block(RowBlock** block) = 0; + virtual OLAPStatus next_block(vectorized::Block* block) = 0; + virtual bool delete_flag() = 0; virtual Version version() = 0; diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp index 6fae7eb976..edd64db455 100644 --- a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp +++ b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp @@ -19,9 +19,12 @@ #include "common/logging.h" #include "gutil/strings/substitute.h" // for Substitute -#include "olap/rowset/segment_v2/bitshuffle_page.h" #include "runtime/mem_pool.h" #include "util/slice.h" // for Slice +#include "vec/columns/column_vector.h" +#include "vec/columns/column_string.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/predicate_column.h" namespace doris { namespace segment_v2 { @@ -220,6 +223,8 @@ Status BinaryDictPageDecoder::init() { return Status::OK(); } +BinaryDictPageDecoder::~BinaryDictPageDecoder() {} + Status BinaryDictPageDecoder::seek_to_position_in_page(size_t pos) { return _data_page_decoder->seek_to_position_in_page(pos); } @@ -230,8 +235,64 @@ bool BinaryDictPageDecoder::is_dict_encoding() const { void BinaryDictPageDecoder::set_dict_decoder(PageDecoder* dict_decoder) { _dict_decoder = (BinaryPlainPageDecoder*)dict_decoder; + _bit_shuffle_ptr = reinterpret_cast*>(_data_page_decoder.get()); }; +Status BinaryDictPageDecoder::next_batch(size_t* n, vectorized::MutableColumnPtr &dst) { + if (_encoding_type == PLAIN_ENCODING) { + return _data_page_decoder->next_batch(n, dst); + } + // dictionary encoding + DCHECK(_parsed); + DCHECK(_dict_decoder != nullptr) << "dict decoder pointer is nullptr"; + + if (PREDICT_FALSE(*n == 0 || _bit_shuffle_ptr->_cur_index >= _bit_shuffle_ptr->_num_elements)) { + *n = 0; + return Status::OK(); + } + + size_t max_fetch = std::min(*n, static_cast(_bit_shuffle_ptr->_num_elements - _bit_shuffle_ptr->_cur_index)); + *n = max_fetch; + + const int32_t* data_array = reinterpret_cast(_bit_shuffle_ptr->_chunk.data); + size_t start_index = _bit_shuffle_ptr->_cur_index; + + auto* dst_col_ptr = dst.get(); + if (dst->is_nullable()) { + auto nullable_column = assert_cast(dst.get()); + dst_col_ptr = nullable_column->get_nested_column_ptr().get(); + + // fill null bitmap here, not null; + // todo(wb) using SIMD speed up here + for (int i = 0; i < max_fetch; i++) { + nullable_column->get_null_map_data().push_back(0); + } + } + + if (dst_col_ptr->is_predicate_column()) { + // cast columnptr to columnstringvalue just for avoid virtual function call overhead + auto* string_value_column_ptr = reinterpret_cast(dst_col_ptr); + for (int i = 0; i < max_fetch; i++, start_index++) { + int32_t codeword = data_array[start_index]; + uint32_t start_offset = _start_offset_array[codeword]; + uint32_t str_len = _len_array[codeword]; + string_value_column_ptr->insert_data(&_dict_decoder->_data[start_offset], str_len); + } + } else { + // todo(wb) research whether using batch memcpy to insert columnString can has better performance when data set is big + for (int i = 0; i < max_fetch; i++, start_index++) { + int32_t codeword = data_array[start_index]; + const uint32_t start_offset = _start_offset_array[codeword]; + const uint32_t str_len = _len_array[codeword]; + dst_col_ptr->insert_data(&_dict_decoder->_data[start_offset], str_len); + } + } + _bit_shuffle_ptr->_cur_index += max_fetch; + + return Status::OK(); + +} + Status BinaryDictPageDecoder::next_batch(size_t* n, ColumnBlockView* dst) { if (_encoding_type == PLAIN_ENCODING) { return _data_page_decoder->next_batch(n, dst); diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.h b/be/src/olap/rowset/segment_v2/binary_dict_page.h index 92904662dd..25a70b9a34 100644 --- a/be/src/olap/rowset/segment_v2/binary_dict_page.h +++ b/be/src/olap/rowset/segment_v2/binary_dict_page.h @@ -33,6 +33,7 @@ #include "olap/types.h" #include "runtime/mem_pool.h" #include "runtime/mem_tracker.h" +#include "olap/rowset/segment_v2/bitshuffle_page.h" namespace doris { namespace segment_v2 { @@ -106,6 +107,8 @@ public: Status next_batch(size_t* n, ColumnBlockView* dst) override; + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) override; + size_t count() const override { return _data_page_decoder->count(); } size_t current_index() const override { return _data_page_decoder->current_index(); } @@ -114,15 +117,21 @@ public: void set_dict_decoder(PageDecoder* dict_decoder); + ~BinaryDictPageDecoder(); + private: Slice _data; PageDecoderOptions _options; std::unique_ptr _data_page_decoder; const BinaryPlainPageDecoder* _dict_decoder = nullptr; + BitShufflePageDecoder* _bit_shuffle_ptr = nullptr; bool _parsed; EncodingTypePB _encoding_type; // use as data buf. std::unique_ptr _batch; + + uint32_t* _start_offset_array = nullptr; + uint32_t* _len_array = nullptr; }; } // namespace segment_v2 diff --git a/be/src/olap/rowset/segment_v2/binary_plain_page.h b/be/src/olap/rowset/segment_v2/binary_plain_page.h index 0747df2173..3c55d2b0fb 100644 --- a/be/src/olap/rowset/segment_v2/binary_plain_page.h +++ b/be/src/olap/rowset/segment_v2/binary_plain_page.h @@ -38,6 +38,8 @@ #include "runtime/mem_pool.h" #include "util/coding.h" #include "util/faststring.h" +#include "vec/columns/column_complex.h" +#include "vec/columns/column_nullable.h" namespace doris { namespace segment_v2 { @@ -227,6 +229,61 @@ public: return Status::OK(); } + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) override { + DCHECK(_parsed); + if (PREDICT_FALSE(*n == 0 || _cur_idx >= _num_elems)) { + *n = 0; + return Status::OK(); + } + const size_t max_fetch = std::min(*n, static_cast(_num_elems - _cur_idx)); + + auto* dst_col_ptr = dst.get(); + if (dst->is_nullable()) { + auto nullable_column = assert_cast(dst.get()); + dst_col_ptr = nullable_column->get_nested_column_ptr().get(); + // fill null bitmap here, not null; + for (int i = 0; i < max_fetch; i++) { + nullable_column->get_null_map_data().push_back(0); + } + } + + if (dst_col_ptr->is_bitmap()) { + auto& bitmap_column = reinterpret_cast(*dst_col_ptr); + for (size_t i = 0; i < max_fetch; i++, _cur_idx++) { + const uint32_t start_offset = offset(_cur_idx); + uint32_t len = offset(_cur_idx + 1) - start_offset; + + bitmap_column.insert_default(); + BitmapValue* pvalue = &bitmap_column.get_element(bitmap_column.size() - 1); + if (len != 0) { + BitmapValue value; + value.deserialize(&_data[start_offset]); + *pvalue = std::move(value); + } else { + *pvalue = std::move(*reinterpret_cast(const_cast(&_data[start_offset]))); + } + } + } else if (dst_col_ptr->is_predicate_column()) { + // todo(wb) padding sv here for better comparison performance + for (size_t i = 0; i < max_fetch; i++, _cur_idx++) { + const uint32_t start_offset = offset(_cur_idx); + uint32_t len = offset(_cur_idx + 1) - start_offset; + StringValue sv(const_cast(&_data[start_offset]), len); + dst_col_ptr->insert_data(reinterpret_cast(&sv), 0); + } + } else { + for (size_t i = 0; i < max_fetch; i++, _cur_idx++) { + // todo(wb) need more test case and then improve here + const uint32_t start_offset = offset(_cur_idx); + uint32_t len = offset(_cur_idx + 1) - start_offset; + dst_col_ptr->insert_data(&_data[start_offset], len); + } + } + + *n = max_fetch; + return Status::OK(); + }; + size_t count() const override { DCHECK(_parsed); return _num_elems; @@ -263,6 +320,8 @@ private: // Index of the currently seeked element in the page. uint32_t _cur_idx; + friend class BinaryDictPageDecoder; + friend class FileColumnIterator; }; } // namespace segment_v2 diff --git a/be/src/olap/rowset/segment_v2/binary_prefix_page.h b/be/src/olap/rowset/segment_v2/binary_prefix_page.h index 7e51b82ee5..7d7bcc9a92 100644 --- a/be/src/olap/rowset/segment_v2/binary_prefix_page.h +++ b/be/src/olap/rowset/segment_v2/binary_prefix_page.h @@ -113,6 +113,10 @@ public: Status next_batch(size_t* n, ColumnBlockView* dst) override; + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) override { + return Status::NotSupported("binary prefix page not implement vec op now"); + }; + size_t count() const override { DCHECK(_parsed); return _num_values; diff --git a/be/src/olap/rowset/segment_v2/bitshuffle_page.h b/be/src/olap/rowset/segment_v2/bitshuffle_page.h index cd54b6bdae..83eebae27b 100644 --- a/be/src/olap/rowset/segment_v2/bitshuffle_page.h +++ b/be/src/olap/rowset/segment_v2/bitshuffle_page.h @@ -37,6 +37,8 @@ #include "util/coding.h" #include "util/faststring.h" #include "util/slice.h" +#include "vec/runtime/vdatetime_value.h" +#include "vec/columns/column_nullable.h" namespace doris { namespace segment_v2 { @@ -348,6 +350,71 @@ public: return Status::OK(); } + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) override { + DCHECK(_parsed); + if (PREDICT_FALSE(*n == 0 || _cur_index >= _num_elements)) { + *n = 0; + return Status::OK(); + } + + size_t max_fetch = std::min(*n, static_cast(_num_elements - _cur_index)); + + int begin = _cur_index; + int end = _cur_index + max_fetch; + + auto* dst_col_ptr = dst.get(); + if (dst->is_nullable()) { + auto nullable_column = assert_cast(dst.get()); + dst_col_ptr = nullable_column->get_nested_column_ptr().get(); + + // fill null bitmap here, not null; + for (int j = begin; j < end; j++) { + nullable_column->get_null_map_data().push_back(0); + } + } + + // todo(wb) Try to eliminate type judgment in pagedecoder + if (dst_col_ptr->is_column_decimal()) { // decimal non-predicate column + for (; begin < end; begin++) { + const char* cur_ptr = (const char*)&_chunk.data[begin * SIZE_OF_TYPE]; + int64_t int_value = *(int64_t*)(cur_ptr); + int32_t frac_value = *(int32_t*)(cur_ptr + sizeof(int64_t)); + DecimalV2Value data(int_value, frac_value); + dst_col_ptr->insert_data(reinterpret_cast(&data), 0); + } + } else if (dst_col_ptr->is_date_type()) { + for (; begin < end; begin++) { + const char* cur_ptr = (const char*)&_chunk.data[begin * SIZE_OF_TYPE]; + uint64_t value = 0; + value = *(unsigned char*)(cur_ptr + 2); + value <<= 8; + value |= *(unsigned char*)(cur_ptr + 1); + value <<= 8; + value |= *(unsigned char*)(cur_ptr); + vectorized::VecDateTimeValue date; + date.from_olap_date(value); + dst_col_ptr->insert_data(reinterpret_cast(&date), 0); + } + } else if (dst_col_ptr->is_datetime_type()) { + for (; begin < end; begin++) { + const char* cur_ptr = (const char*)&_chunk.data[begin * SIZE_OF_TYPE]; + uint64_t value = *reinterpret_cast(cur_ptr); + vectorized::VecDateTimeValue date(value); + dst_col_ptr->insert_data(reinterpret_cast(&date), 0); + } + } else { + // todo(wb) batch insert here + for (; begin < end; begin++) { + dst_col_ptr->insert_data((const char*)&_chunk.data[begin * SIZE_OF_TYPE], 0); + } + } + + *n = max_fetch; + _cur_index += max_fetch; + + return Status::OK(); + }; + Status peek_next_batch(size_t* n, ColumnBlockView* dst) override { return next_batch(n, dst); } @@ -393,6 +460,7 @@ private: int _size_of_element; size_t _cur_index; Chunk _chunk; + friend class BinaryDictPageDecoder; }; } // namespace segment_v2 diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index 3bbbeab89e..c08b3280a2 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -30,6 +30,8 @@ #include "util/block_compression.h" #include "util/coding.h" // for get_varint32 #include "util/rle_encoding.h" // for RleDecoder +#include "vec/core/types.h" +#include "vec/runtime/vdatetime_value.h" //for VecDateTime namespace doris { namespace segment_v2 { @@ -578,6 +580,57 @@ Status FileColumnIterator::next_batch(size_t* n, ColumnBlockView* dst, bool* has return Status::OK(); } +Status FileColumnIterator::next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) { + size_t curr_size = dst->byte_size(); + size_t remaining = *n; + *has_null = false; + while (remaining > 0) { + if (!_page->has_remaining()) { + bool eos = false; + RETURN_IF_ERROR(_load_next_page(&eos)); + if (eos) { + break; + } + } + + // number of rows to be read from this page + size_t nrows_in_page = std::min(remaining, _page->remaining()); + size_t nrows_to_read = nrows_in_page; + if (_page->has_null) { + while (nrows_to_read > 0) { + bool is_null = false; + size_t this_run = _page->null_decoder.GetNextRun(&is_null, nrows_to_read); + // we use num_rows only for CHECK + size_t num_rows = this_run; + if (!is_null) { + RETURN_IF_ERROR(_page->data_decoder->next_batch(&num_rows, dst)); + DCHECK_EQ(this_run, num_rows); + } else { + *has_null = true; + // todo(wb) add a DCHECK here to check whether type is column nullable + for (size_t x = 0; x < this_run; x++) { + dst->insert_data(nullptr, 0); // todo(wb) vectorized here + } + } + + nrows_to_read -= this_run; + _page->offset_in_page += this_run; + _current_ordinal += this_run; + } + } else { + RETURN_IF_ERROR(_page->data_decoder->next_batch(&nrows_to_read, dst)); + DCHECK_EQ(nrows_to_read, nrows_in_page); + + _page->offset_in_page += nrows_to_read; + _current_ordinal += nrows_to_read; + } + remaining -= nrows_in_page; + } + *n -= remaining; + _opts.stats->bytes_read += (dst->byte_size() - curr_size) + BitmapSize(*n); + return Status::OK(); +} + Status FileColumnIterator::_load_next_page(bool* eos) { _page_iter.next(); if (!_page_iter.valid()) { @@ -715,5 +768,55 @@ Status DefaultValueColumnIterator::next_batch(size_t* n, ColumnBlockView* dst, b return Status::OK(); } +void DefaultValueColumnIterator::insert_default_data(vectorized::MutableColumnPtr &dst, size_t n) { + vectorized::Int128 int128; + char* data_ptr = (char*)&int128; + size_t data_len = sizeof(int128); + + auto type = _type_info->type(); + if (type == OLAP_FIELD_TYPE_DATE) { + assert(_type_size == sizeof(FieldTypeTraits::CppType)); //uint24_t + std::string str = FieldTypeTraits::to_string(_mem_value); + + vectorized::VecDateTimeValue value; + value.from_date_str(str.c_str(), str.length()); + value.cast_to_date(); + //TODO: here is int128 = int64 + int128 = binary_cast(value); + } else if (type == OLAP_FIELD_TYPE_DATETIME) { + assert(_type_size == sizeof(FieldTypeTraits::CppType)); //int64_t + std::string str = FieldTypeTraits::to_string(_mem_value); + + vectorized::VecDateTimeValue value; + value.from_date_str(str.c_str(), str.length()); + value.to_datetime(); + + int128 = binary_cast(value); + } else if (type == OLAP_FIELD_TYPE_DECIMAL) { + assert(_type_size == sizeof(FieldTypeTraits::CppType)); //decimal12_t + decimal12_t* d = (decimal12_t*)_mem_value; + int128 = DecimalV2Value(d->integer, d->fraction).value(); + } else { + data_ptr = (char*)_mem_value; + data_len = _type_size; + } + + for (size_t i = 0; i < n; ++i) { + dst->insert_data(data_ptr, data_len); + } +} + +Status DefaultValueColumnIterator::next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) { + if (_is_default_value_null) { + *has_null = true; + dst->insert_many_defaults(*n); + } else { + *has_null = false; + insert_default_data(dst, *n); + } + + return Status::OK(); +} + } // namespace segment_v2 } // namespace doris diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h index 0b3588ce75..98afe442d2 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.h +++ b/be/src/olap/rowset/segment_v2/column_reader.h @@ -215,11 +215,20 @@ public: return next_batch(n, dst, &has_null); } + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) { + bool has_null; + return next_batch(n, dst, &has_null); + } + // After one seek, we can call this function many times to read data // into ColumnBlockView. when read string type data, memory will allocated // from MemPool virtual Status next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) = 0; + virtual Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) { + return Status::NotSupported("not implement"); + } + virtual ordinal_t get_current_ordinal() const = 0; virtual Status get_row_ranges_by_zone_map(CondColumn* cond_column, CondColumn* delete_condition, @@ -268,6 +277,8 @@ public: Status next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) override; + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) override; + ordinal_t get_current_ordinal() const override { return _current_ordinal; } // get row ranges by zone map @@ -401,11 +412,20 @@ public: return Status::OK(); } + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) { + bool has_null; + return next_batch(n, dst, &has_null); + } + Status next_batch(size_t* n, ColumnBlockView* dst, bool* has_null) override; + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst, bool* has_null) override; + ordinal_t get_current_ordinal() const override { return _current_rowid; } private: + void insert_default_data(vectorized::MutableColumnPtr &dst, size_t n); + bool _has_default_value; std::string _default_value; bool _is_nullable; diff --git a/be/src/olap/rowset/segment_v2/empty_segment_iterator.cpp b/be/src/olap/rowset/segment_v2/empty_segment_iterator.cpp index 61c0faba69..f367c2d052 100644 --- a/be/src/olap/rowset/segment_v2/empty_segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/empty_segment_iterator.cpp @@ -30,5 +30,9 @@ Status EmptySegmentIterator::next_batch(RowBlockV2* block) { return Status::EndOfFile("no more data in segment"); } +Status EmptySegmentIterator::next_batch(vectorized::Block* block) { + return Status::EndOfFile("no more data in segment"); +} + } // namespace segment_v2 -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/olap/rowset/segment_v2/empty_segment_iterator.h b/be/src/olap/rowset/segment_v2/empty_segment_iterator.h index cb6c48c2df..3e1a4f9bf4 100644 --- a/be/src/olap/rowset/segment_v2/empty_segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/empty_segment_iterator.h @@ -32,10 +32,11 @@ public: Status init(const StorageReadOptions& opts) override { return Status::OK(); } const Schema& schema() const override { return _schema; } Status next_batch(RowBlockV2* row_block) override; + Status next_batch(vectorized::Block* block) override; private: Schema _schema; }; } // namespace segment_v2 -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/olap/rowset/segment_v2/frame_of_reference_page.h b/be/src/olap/rowset/segment_v2/frame_of_reference_page.h index 5d0c0b1e22..df8fe06a1e 100644 --- a/be/src/olap/rowset/segment_v2/frame_of_reference_page.h +++ b/be/src/olap/rowset/segment_v2/frame_of_reference_page.h @@ -161,6 +161,10 @@ public: return Status::OK(); } + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) override { + return Status::NotSupported("frame page not implement vec op now"); + }; + Status peek_next_batch(size_t* n, ColumnBlockView* dst) override { return next_batch(n, dst); } diff --git a/be/src/olap/rowset/segment_v2/page_decoder.h b/be/src/olap/rowset/segment_v2/page_decoder.h index feed37f035..ab5b482e29 100644 --- a/be/src/olap/rowset/segment_v2/page_decoder.h +++ b/be/src/olap/rowset/segment_v2/page_decoder.h @@ -19,6 +19,7 @@ #include "common/status.h" // for Status #include "olap/column_block.h" // for ColumnBlockView +#include "vec/columns/column.h" namespace doris { namespace segment_v2 { @@ -81,6 +82,10 @@ public: // allocated in the column_vector_view's mem_pool. virtual Status next_batch(size_t* n, ColumnBlockView* dst) = 0; + virtual Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) { + return Status::NotSupported("not implement vec op now"); + } + // Same as `next_batch` except for not moving forward the cursor. // When read array's ordinals in `ArrayFileColumnIterator`, we want to read one extra ordinal // but do not want to move forward the cursor. diff --git a/be/src/olap/rowset/segment_v2/plain_page.h b/be/src/olap/rowset/segment_v2/plain_page.h index 555c9af1a1..09e4280474 100644 --- a/be/src/olap/rowset/segment_v2/plain_page.h +++ b/be/src/olap/rowset/segment_v2/plain_page.h @@ -186,6 +186,10 @@ public: Status next_batch(size_t* n, ColumnBlockView* dst) override { return next_batch(n, dst); } + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) override { + return Status::NotSupported("plain page not implement vec op now"); + }; + template inline Status next_batch(size_t* n, ColumnBlockView* dst) { DCHECK(_parsed); diff --git a/be/src/olap/rowset/segment_v2/rle_page.h b/be/src/olap/rowset/segment_v2/rle_page.h index ed20aaf94e..52a9613cf4 100644 --- a/be/src/olap/rowset/segment_v2/rle_page.h +++ b/be/src/olap/rowset/segment_v2/rle_page.h @@ -230,6 +230,29 @@ public: return Status::OK(); } + Status next_batch(size_t* n, vectorized::MutableColumnPtr &dst) override { + DCHECK(_parsed); + if (PREDICT_FALSE(*n == 0 || _cur_index >= _num_elements)) { + *n = 0; + return Status::OK(); + } + + size_t to_fetch = std::min(*n, static_cast(_num_elements - _cur_index)); + size_t remaining = to_fetch; + bool result = false; + CppType value; + while (remaining > 0) { + result = _rle_decoder.Get(&value); + DCHECK(result); + dst->insert_data((char*)(&value), SIZE_OF_TYPE); + remaining--; + } + + _cur_index += to_fetch; + *n = to_fetch; + return Status::OK(); + }; + size_t count() const override { return _num_elements; } size_t current_index() const override { return _cur_index; } diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index cb7e0d56ca..86a89acf41 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -30,6 +30,7 @@ #include "olap/rowset/segment_v2/segment.h" #include "olap/short_key_index.h" #include "util/doris_metrics.h" +#include "olap/in_list_predicate.h" using strings::Substitute; @@ -120,7 +121,7 @@ Status SegmentIterator::init(const StorageReadOptions& opts) { return Status::OK(); } -Status SegmentIterator::_init() { +Status SegmentIterator::_init(bool is_vec) { DorisMetrics::instance()->segment_read_total->increment(1); // get file handle from file descriptor of segment fs::BlockManager* block_mgr = fs::fs_util::block_manager(_segment->_path_desc.storage_medium); @@ -133,7 +134,11 @@ Status SegmentIterator::_init() { RETURN_IF_ERROR(_get_row_ranges_by_keys()); } RETURN_IF_ERROR(_get_row_ranges_by_column_conditions()); - _init_lazy_materialization(); + if (is_vec) { + _vec_init_lazy_materialization(); + } else { + _init_lazy_materialization(); + } _range_iter.reset(new BitmapRangeIterator(_row_bitmap)); return Status::OK(); } @@ -581,5 +586,359 @@ Status SegmentIterator::next_batch(RowBlockV2* block) { return Status::OK(); } +/* ---------------------- for vecterization implementation ---------------------- */ + +// todo(wb) need a UT here +void SegmentIterator::_vec_init_lazy_materialization() { + _is_pred_column.resize(_schema.columns().size(), false); + + std::set pred_column_ids; // including short_cir_pred_col_id_set and vec_pred_col_id_set + _is_all_column_basic_type = true; + bool is_predicate_column_exists = false; + bool is_non_predicate_column_exists = false; + + if (!_col_predicates.empty()) { + is_predicate_column_exists = true; + + std::set short_cir_pred_col_id_set; // using set for distinct cid + std::set vec_pred_col_id_set; + + for (auto predicate : _col_predicates) { + auto cid = predicate->column_id(); + FieldType type = _schema.column(cid)->type(); + _is_pred_column[cid] = true; + pred_column_ids.insert(cid); + + // for date type which can not be executed in a vectorized way, using short circuit execution + if (type == OLAP_FIELD_TYPE_VARCHAR || type == OLAP_FIELD_TYPE_CHAR || type == OLAP_FIELD_TYPE_DECIMAL + || type == OLAP_FIELD_TYPE_DATE || predicate->is_in_predicate()) { + short_cir_pred_col_id_set.insert(cid); + _short_cir_eval_predicate.push_back(predicate); + _is_all_column_basic_type = false; + } else { + vec_pred_col_id_set.insert(predicate->column_id()); + if (_pre_eval_block_predicate == nullptr) { + _pre_eval_block_predicate = new AndBlockColumnPredicate(); + } + reinterpret_cast(_pre_eval_block_predicate)->add_column_predicate(new SingleColumnBlockPredicate(predicate)); + } + } + + std::set del_cond_id_set; + _opts.delete_condition_predicates.get()->get_all_column_ids(del_cond_id_set); + short_cir_pred_col_id_set.insert(del_cond_id_set.begin(), del_cond_id_set.end()); + pred_column_ids.insert(del_cond_id_set.begin(), del_cond_id_set.end()); + + if (_schema.column_ids().size() > pred_column_ids.size()) { + for (auto cid : _schema.column_ids()) { + if (!_is_pred_column[cid]) { + _non_predicate_columns.push_back(cid); + is_non_predicate_column_exists = true; + } + } + } + + _vec_pred_column_ids.assign(vec_pred_col_id_set.cbegin(), vec_pred_col_id_set.cend()); + _short_cir_pred_column_ids.assign(short_cir_pred_col_id_set.cbegin(), short_cir_pred_col_id_set.cend()); + } else { + _is_all_column_basic_type = false; + is_non_predicate_column_exists = true; + for (auto cid : _schema.column_ids()) { + _non_predicate_columns.push_back(cid); + } + } + + // note(wb) in following cases we disable lazy materialization + // case 1: when all column is basic type(is_all_column_basic_type = true) + // because we think `seek and read` cost > read page cost, lazy materialize may cause more `seek and read`, so disable it + // case 2: all column is predicate column + // case 3: all column is not predicate column + // todo(wb) need further research more lazy materialization rule, such as get more info from `statistics` for better decision + if (_is_all_column_basic_type) { + std::set pred_set(_vec_pred_column_ids.begin(), _vec_pred_column_ids.end()); + std::set non_pred_set(_non_predicate_columns.begin(), _non_predicate_columns.end()); + + // when _is_all_column_basic_type = true, _first_read_column_ids should keep the same order with _schema.column_ids which stands for return column order + for (int i = 0; i < _schema.num_column_ids(); i++) { + auto cid = _schema.column_ids()[i]; + if (pred_set.find(cid) != pred_set.end()) { + _first_read_column_ids.push_back(cid); + } else if (non_pred_set.find(cid) != non_pred_set.end()) { + _first_read_column_ids.push_back(cid); + _is_pred_column[cid] = true; // in this case, non-predicate column should also be filtered by sel idx, so we regard it as pred columns + } + } + + } else if (is_predicate_column_exists && !is_non_predicate_column_exists) { + _first_read_column_ids.assign(pred_column_ids.cbegin(), pred_column_ids.cend()); + } else if (!is_predicate_column_exists && is_non_predicate_column_exists) { + for (auto cid : _non_predicate_columns) { + _first_read_column_ids.push_back(cid); + } + } else { + _lazy_materialization_read = true; + _first_read_column_ids.assign(pred_column_ids.cbegin(), pred_column_ids.cend()); + } + + // make _schema_block_id_map + _schema_block_id_map.resize(_schema.columns().size()); + for (int i = 0; i < _schema.num_column_ids(); i++) { + auto cid = _schema.column_ids()[i]; + _schema_block_id_map[cid] = i; + } + +} + +Status SegmentIterator::_read_columns(const std::vector& column_ids, vectorized::MutableColumns& column_block, size_t nrows) { + for (auto cid : column_ids) { + auto& column = column_block[cid]; + size_t rows_read = nrows; + RETURN_IF_ERROR(_column_iterators[cid]->next_batch(&rows_read, column)); + DCHECK_EQ(nrows, rows_read); + } + return Status::OK(); +} + +void SegmentIterator::_init_current_block(vectorized::Block* block, std::vector& current_columns) { + bool is_block_mem_reuse= block->mem_reuse(); + if (is_block_mem_reuse) { + size_t column_to_keep = _schema.num_column_ids(); + for (int i = block->columns() - 1; i >= column_to_keep; i--) { + block->erase(i); + } + block->clear_column_data(); + } else { // pre fill output block here + for (size_t i = 0; i < _schema.num_column_ids(); i++) { + auto cid = _schema.column_ids()[i]; + auto* column_desc = _schema.columns()[cid]; + auto data_type = Schema::get_data_type_ptr(column_desc->type()); + if (column_desc->is_nullable()) { + block->insert({nullptr, std::make_shared(std::move(data_type)), column_desc->name()}); + } else { + block->insert({nullptr, std::move(data_type), column_desc->name()}); + } + } + } + + for (size_t i = 0; i < _schema.num_column_ids(); i++) { + auto cid = _schema.column_ids()[i]; + if (_is_pred_column[cid]) { //todo(wb) maybe we can relase it after output block + current_columns[cid]->clear(); + } else { // non-predicate column + auto &column_desc = _schema.columns()[cid]; + if (is_block_mem_reuse) { + current_columns[cid] = std::move(*block->get_by_position(i).column).mutate(); + } else { + auto data_type = Schema::get_data_type_ptr(column_desc->type()); + if (column_desc->is_nullable()) { + current_columns[cid] = doris::vectorized::ColumnNullable::create( + std::move(data_type->create_column()), doris::vectorized::ColumnUInt8::create()); + } else { + current_columns[cid] = data_type->create_column(); + } + } + if (column_desc->type() == OLAP_FIELD_TYPE_DATE) { + current_columns[cid]->set_date_type(); + } else if (column_desc->type() == OLAP_FIELD_TYPE_DATETIME) { + current_columns[cid]->set_datetime_type(); + } + } + } +} + +void SegmentIterator::_output_non_pred_columns(vectorized::Block* block, bool is_block_mem_reuse) { + for (auto cid : _non_predicate_columns) { + block->replace_by_position(_schema_block_id_map[cid], std::move(_current_return_columns[cid])); + } + } + +void SegmentIterator::_output_column_by_sel_idx(vectorized::Block* block, std::vector columnIds, + uint16_t* sel_rowid_idx, uint16_t select_size, bool is_block_mem_reuse) { + for (auto cid : columnIds) { + auto &column_ptr = _current_return_columns[cid]; + if (is_block_mem_reuse) { + column_ptr->filter_by_selector(sel_rowid_idx, select_size, + &block->get_by_position(_schema_block_id_map[cid]).column); + } else { + block->replace_by_position(_schema_block_id_map[cid], + (*column_ptr).get_ptr()->filter_by_selector(sel_rowid_idx, select_size)); + } + } + } + + +Status SegmentIterator::_read_columns_by_index(uint32_t nrows_read_limit, uint32_t& nrows_read, bool set_block_rowid) { + do { + uint32_t range_from; + uint32_t range_to; + bool has_next_range = + _range_iter->next_range(nrows_read_limit - nrows_read, &range_from, &range_to); + if (!has_next_range) { + break; + } + if (_cur_rowid == 0 || _cur_rowid != range_from) { + _cur_rowid = range_from; + RETURN_IF_ERROR(_seek_columns(_first_read_column_ids, _cur_rowid)); + } + size_t rows_to_read = range_to - range_from; + RETURN_IF_ERROR(_read_columns(_first_read_column_ids, _current_return_columns, rows_to_read)); + _cur_rowid += rows_to_read; + if (set_block_rowid) { + for (uint32_t rid = range_from; rid < range_to; rid++) { + _block_rowids[nrows_read++] = rid; + } + } else { + nrows_read += rows_to_read; + } + } while (nrows_read < nrows_read_limit); + return Status::OK(); +} + +void SegmentIterator::_evaluate_vectorization_predicate(uint16_t* sel_rowid_idx, uint16_t& selected_size) { + uint16_t new_size = 0; + if (_vec_pred_column_ids.size() == 0) { + for (uint32_t i = 0; i < selected_size; ++i) { + sel_rowid_idx[new_size++] = i; + } + return; + } + + uint16_t original_size = selected_size; + bool ret_flags[selected_size]; + memset(ret_flags, 1, selected_size); + _pre_eval_block_predicate->evaluate_vec(_current_return_columns, selected_size, ret_flags); + + for (uint32_t i = 0; i < selected_size; ++i) { + if (ret_flags[i]) { + sel_rowid_idx[new_size++] = i; + } + } + + _opts.stats->rows_vec_cond_filtered += original_size - new_size; + selected_size = new_size; +} + +void SegmentIterator::_evaluate_short_circuit_predicate(uint16_t* vec_sel_rowid_idx, uint16_t* selected_size_ptr) { + if (_short_cir_pred_column_ids.size() == 0) { + return; + } + + for (auto column_predicate : _short_cir_eval_predicate) { + auto column_id = column_predicate->column_id(); + auto& short_cir_column = _current_return_columns[column_id]; + column_predicate->evaluate(*short_cir_column, vec_sel_rowid_idx, selected_size_ptr); + } + + // evaluate delete condition + _opts.delete_condition_predicates->evaluate(_current_return_columns, vec_sel_rowid_idx, selected_size_ptr); +} + +void SegmentIterator::_read_columns_by_rowids(std::vector& read_column_ids, std::vector& rowid_vector, + uint16_t* sel_rowid_idx, size_t select_size, vectorized::MutableColumns* mutable_columns) { + size_t start_idx = 0; + while (start_idx < select_size) { + size_t end_idx = start_idx + 1; + while (end_idx < select_size && (rowid_vector[sel_rowid_idx[end_idx - 1]] == rowid_vector[sel_rowid_idx[end_idx]] - 1)) { + end_idx++; + } + size_t range = end_idx - start_idx; + _seek_columns(read_column_ids, rowid_vector[sel_rowid_idx[start_idx]]); + _read_columns(read_column_ids, *mutable_columns, range); + start_idx += range; + } +} + +Status SegmentIterator::next_batch(vectorized::Block* block) { + bool is_mem_reuse = block->mem_reuse(); + SCOPED_RAW_TIMER(&_opts.stats->block_load_ns); + if (UNLIKELY(!_inited)) { + RETURN_IF_ERROR(_init(true)); + _inited = true; + if (_vec_pred_column_ids.size() > 0 || _short_cir_pred_column_ids.size() > 0) { + _block_rowids.reserve(_opts.block_row_max); + } + _current_return_columns.resize(_schema.columns().size()); + for (size_t i = 0; i < _schema.num_column_ids(); i++) { + auto cid = _schema.column_ids()[i]; + if (_is_pred_column[cid]) { + auto& column_desc = _schema.columns()[cid]; + _current_return_columns[cid] = Schema::get_predicate_column_nullable_ptr(column_desc->type(), column_desc->is_nullable()); + _current_return_columns[cid]->reserve(_opts.block_row_max); + } + } + } + + _init_current_block(block, _current_return_columns); + + uint32_t nrows_read = 0; + uint32_t nrows_read_limit = _opts.block_row_max; + _read_columns_by_index(nrows_read_limit, nrows_read, _col_predicates.size() > 0); + + _opts.stats->blocks_load += 1; + _opts.stats->raw_rows_read += nrows_read; + + if (nrows_read == 0) { + for (int i = 0; i < _schema.num_column_ids(); i++) { + auto cid = _schema.column_ids()[i]; + // todo(wb) abstract make column where + if (!_is_pred_column[cid]) { // non-predicate + block->replace_by_position(i, std::move(_current_return_columns[cid])); + } else { // predicate + if (!is_mem_reuse) { + auto* column_desc = _schema.columns()[cid]; + auto data_type = Schema::get_data_type_ptr(column_desc->type()); + block->replace_by_position(i, data_type->create_column()); + } + } + // not sure whether block is clear before enter segmentIter, so clear it here. + if (is_mem_reuse) { + block->clear_column_data(); + } + } + return Status::EndOfFile("no more data in segment"); + } + + // when no predicate(include delete condition) is provided, output column directly + if (_vec_pred_column_ids.size() == 0 && _short_cir_pred_column_ids.size() == 0) { + _output_non_pred_columns(block, is_mem_reuse); + } else { // need predicate evaluation + uint16_t selected_size = nrows_read; + uint16_t sel_rowid_idx[selected_size]; + + // step 1: evaluate vectorization predicate + _evaluate_vectorization_predicate(sel_rowid_idx, selected_size); + + // When predicate column and no-predicate column are both basic type, lazy materialization is eliminate + // So output block directly after vecorization evaluation + if (_is_all_column_basic_type) { + _output_column_by_sel_idx(block, _first_read_column_ids, sel_rowid_idx, selected_size, is_mem_reuse); + return Status::OK(); + } + + // step 2: evaluate short ciruit predicate + // todo(wb) research whether need to read short predicate after vectorization evaluation + // to reduce cost of read short circuit columns. + // In SSB test, it make no difference; So need more scenarios to test + _evaluate_short_circuit_predicate(sel_rowid_idx, &selected_size); + + // step3: read non_predicate column + if (_non_predicate_columns.size() != 0) { + _read_columns_by_rowids(_non_predicate_columns, _block_rowids, sel_rowid_idx, selected_size, &_current_return_columns); + } + + // step4: output columns + // 4.1 output non-predicate column + _output_non_pred_columns(block, is_mem_reuse); + + // 4.2 output short circuit predicate column + _output_column_by_sel_idx(block, _short_cir_pred_column_ids, sel_rowid_idx, selected_size, is_mem_reuse); + // 4.3 output vectorizatioin predicate column + _output_column_by_sel_idx(block, _vec_pred_column_ids, sel_rowid_idx, selected_size, is_mem_reuse); + + } + + return Status::OK(); +} + } // namespace segment_v2 } // namespace doris diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index 0da6ae4658..c06077e4db 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -50,14 +50,17 @@ public: SegmentIterator(std::shared_ptr segment, const Schema& _schema, std::shared_ptr parent); ~SegmentIterator() override; + Status init(const StorageReadOptions& opts) override; Status next_batch(RowBlockV2* row_block) override; + Status next_batch(vectorized::Block* block) override; + const Schema& schema() const override { return _schema; } bool is_lazy_materialization_read() const override { return _lazy_materialization_read; } uint64_t data_id() const { return _segment->id(); } private: - Status _init(); + Status _init(bool is_vec = false); Status _init_return_column_iterators(); Status _init_bitmap_index_iterators(); @@ -75,6 +78,7 @@ private: Status _apply_bitmap_index(); void _init_lazy_materialization(); + void _vec_init_lazy_materialization(); uint32_t segment_id() const { return _segment->id(); } uint32_t num_rows() const { return _segment->num_rows(); } @@ -84,6 +88,17 @@ private: Status _read_columns(const std::vector& column_ids, RowBlockV2* block, size_t row_offset, size_t nrows); + // for vectorization implementation + Status _read_columns(const std::vector& column_ids, vectorized::MutableColumns& column_block, size_t nrows); + Status _read_columns_by_index(uint32_t nrows_read_limit, uint32_t& nrows_read, bool set_block_rowid); + void _init_current_block(vectorized::Block* block, std::vector& non_pred_vector); + void _evaluate_vectorization_predicate(uint16_t* sel_rowid_idx, uint16_t& selected_size); + void _evaluate_short_circuit_predicate(uint16_t* sel_rowid_idx, uint16_t* selected_size); + void _output_non_pred_columns(vectorized::Block* block, bool is_block_mem_reuse); + void _output_column_by_sel_idx(vectorized::Block* block, std::vector columnids, uint16_t* sel_rowid_idx, uint16_t select_size, bool is_block_mem_reuse); + void _read_columns_by_rowids(std::vector& read_column_ids, std::vector& rowid_vector, + uint16_t* sel_rowid_idx, size_t select_size, vectorized::MutableColumns* mutable_columns); + private: class BitmapRangeIterator; @@ -113,6 +128,21 @@ private: // could be a local variable of next_batch(), kept here to reuse vector memory std::vector _block_rowids; + // fields for vectorization execution + bool _is_all_column_basic_type; + std::vector _vec_pred_column_ids; // keep columnId of columns for vectorized predicate evaluation + std::vector _short_cir_pred_column_ids; // keep columnId of columns for short circuit predicate evaluation + vector _is_pred_column; // columns hold by segmentIter + vectorized::MutableColumns _current_return_columns; + AndBlockColumnPredicate* _pre_eval_block_predicate = nullptr; + std::vector _short_cir_eval_predicate; + // when lazy materialization is enable, segmentIter need to read data at least twice + // first, read predicate columns by various index + // second, read non-predicate columns + // so we need a field to stand for columns first time to read + vector _first_read_column_ids; + vector _schema_block_id_map; // map from schema column id to column idx in Block + // the actual init process is delayed to the first call to next_batch() bool _inited; diff --git a/be/src/olap/schema.cpp b/be/src/olap/schema.cpp index ab00e37773..f5a0900cac 100644 --- a/be/src/olap/schema.cpp +++ b/be/src/olap/schema.cpp @@ -18,6 +18,11 @@ #include "olap/schema.h" #include "olap/row_block2.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_complex.h" +#include "vec/columns/predicate_column.h" +#include "vec/core/types.h" +#include "olap/uint24.h" namespace doris { @@ -103,4 +108,109 @@ Schema::~Schema() { } } +vectorized::DataTypePtr Schema::get_data_type_ptr(FieldType type) { + switch (type) { + case OLAP_FIELD_TYPE_BOOL: + return std::make_shared(); + + case OLAP_FIELD_TYPE_TINYINT: + return std::make_shared(); + + case OLAP_FIELD_TYPE_SMALLINT: + return std::make_shared(); + + case OLAP_FIELD_TYPE_INT: + return std::make_shared(); + + case OLAP_FIELD_TYPE_FLOAT: + return std::make_shared(); + + case OLAP_FIELD_TYPE_BIGINT: + return std::make_shared(); + + case OLAP_FIELD_TYPE_LARGEINT: + return std::make_shared(); + + case OLAP_FIELD_TYPE_DATE: + return std::make_shared(); + + case OLAP_FIELD_TYPE_DATETIME: + return std::make_shared(); + + case OLAP_FIELD_TYPE_DOUBLE: + return std::make_shared(); + + case OLAP_FIELD_TYPE_CHAR: + case OLAP_FIELD_TYPE_VARCHAR: + case OLAP_FIELD_TYPE_HLL: + case OLAP_FIELD_TYPE_STRING: + return std::make_shared(); + + case TYPE_OBJECT: + return std::make_shared(); + + case OLAP_FIELD_TYPE_DECIMAL: + return std::make_shared>(27, 9); + + default: + DCHECK(false); + } + // For llvm complain + return nullptr; +} + +vectorized::IColumn::MutablePtr Schema::get_predicate_column_nullable_ptr(FieldType type, bool is_null) { + vectorized::IColumn::MutablePtr ptr = Schema::get_predicate_column_ptr(type); + if (is_null) { + return doris::vectorized::ColumnNullable::create(std::move(ptr), doris::vectorized::ColumnUInt8::create()); + } + return ptr; +} + +vectorized::IColumn::MutablePtr Schema::get_predicate_column_ptr(FieldType type) { + switch (type) { + case OLAP_FIELD_TYPE_BOOL: + return doris::vectorized::PredicateColumnType::create();; + case OLAP_FIELD_TYPE_TINYINT: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_SMALLINT: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_INT: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_FLOAT: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_DOUBLE: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_BIGINT: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_LARGEINT: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_DATE: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_DATETIME: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_CHAR: + case OLAP_FIELD_TYPE_VARCHAR: + case OLAP_FIELD_TYPE_STRING: + return doris::vectorized::PredicateColumnType::create(); + + case OLAP_FIELD_TYPE_DECIMAL: + return doris::vectorized::PredicateColumnType::create(); + + default: + DCHECK(false); + } + // For llvm complain + return nullptr; +} + } // namespace doris diff --git a/be/src/olap/schema.h b/be/src/olap/schema.h index 39e3c7914c..2596f9780c 100644 --- a/be/src/olap/schema.h +++ b/be/src/olap/schema.h @@ -100,6 +100,14 @@ public: ~Schema(); + static vectorized::DataTypePtr get_data_type_ptr(FieldType type); + + static vectorized::IColumn::MutablePtr get_predicate_column_ptr(FieldType type); + + static vectorized::IColumn::MutablePtr get_predicate_column_nullable_ptr(FieldType type, bool is_null = false); + + const std::vector& columns() const { return _cols; } + const Field* column(ColumnId cid) const { return _cols[cid]; } Field* mutable_column(ColumnId cid) const { return _cols[cid]; } diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index 072a4155a1..5b407d5185 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -18,6 +18,8 @@ #include "olap/tablet_schema.h" #include "tablet_meta.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type.h" namespace doris { @@ -490,6 +492,16 @@ void TabletSchema::init_field_index_for_test() { } } +vectorized::Block TabletSchema::create_block(const std::vector& return_columns) const { + vectorized::Block block; + for (int i = 0; i < return_columns.size(); ++i) { + const auto& col = _cols[return_columns[i]]; + auto data_type = vectorized::IDataType::from_olap_engine(col.type(), col.is_nullable()); + block.insert({data_type->create_column(), data_type, col.name()}); + } + return block; +} + bool operator==(const TabletColumn& a, const TabletColumn& b) { if (a._unique_id != b._unique_id) return false; if (a._col_name != b._col_name) return false; @@ -548,4 +560,5 @@ bool operator!=(const TabletSchema& a, const TabletSchema& b) { return !(a == b); } + } // namespace doris diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h index 0cb894fdfb..ffd8c34bc4 100644 --- a/be/src/olap/tablet_schema.h +++ b/be/src/olap/tablet_schema.h @@ -26,6 +26,9 @@ #include "olap/types.h" namespace doris { +namespace vectorized { +class Block; +} class TabletColumn { public: @@ -141,6 +144,7 @@ public: inline void set_delete_sign_idx(int32_t delete_sign_idx) { _delete_sign_idx = delete_sign_idx; } inline bool has_sequence_col() const { return _sequence_col_idx != -1; } inline int32_t sequence_col_idx() const { return _sequence_col_idx; } + vectorized::Block create_block(const std::vector& return_columns) const; private: // Only for unit test diff --git a/be/src/olap/tuple_reader.cpp b/be/src/olap/tuple_reader.cpp index 9eba1d204f..5c15c2b42f 100644 --- a/be/src/olap/tuple_reader.cpp +++ b/be/src/olap/tuple_reader.cpp @@ -196,7 +196,7 @@ OLAPStatus TupleReader::_unique_key_next_row(RowCursor* row_cursor, MemPool* mem cur_delete_flag = _next_delete_flag; } - // if reader needs to filter delete row and current delete_flag is ture, + // if reader needs to filter delete row and current delete_flag is true, // then continue if (!(cur_delete_flag && _filter_delete)) { break; diff --git a/be/src/runtime/datetime_value.cpp b/be/src/runtime/datetime_value.cpp index 9ec8ccbb6a..b545f5512e 100644 --- a/be/src/runtime/datetime_value.cpp +++ b/be/src/runtime/datetime_value.cpp @@ -1466,6 +1466,8 @@ bool DateTimeValue::from_date_format_str(const char* format, int format_len, con } bool DateTimeValue::date_add_interval(const TimeInterval& interval, TimeUnit unit) { + if (!is_valid_date()) return false; + int sign = interval.is_neg ? -1 : 1; switch (unit) { case MICROSECOND: diff --git a/be/src/runtime/datetime_value.h b/be/src/runtime/datetime_value.h index 096a24d096..4f3a310e48 100644 --- a/be/src/runtime/datetime_value.h +++ b/be/src/runtime/datetime_value.h @@ -30,7 +30,7 @@ #include "udf/udf.h" #include "util/hash_util.hpp" #include "util/timezone_utils.h" - +#include "vec/runtime/vdatetime_value.h" namespace doris { enum TimeUnit { @@ -566,7 +566,9 @@ public: private: // Used to make sure sizeof DateTimeValue friend class UnusedClass; - + friend void doris::vectorized::VecDateTimeValue::convert_vec_dt_to_dt(DateTimeValue* dt); + friend void doris::vectorized::VecDateTimeValue::convert_dt_to_vec_dt(DateTimeValue* dt); + void from_packed_time(int64_t packed_time) { _microsecond = packed_time % (1LL << 24); int64_t ymdhms = packed_time >> 24; diff --git a/be/src/runtime/descriptors.cpp b/be/src/runtime/descriptors.cpp index ab4fa972c2..95d0a79101 100644 --- a/be/src/runtime/descriptors.cpp +++ b/be/src/runtime/descriptors.cpp @@ -25,6 +25,10 @@ #include "gen_cpp/Descriptors_types.h" #include "gen_cpp/descriptors.pb.h" +#include "vec/core/columns_with_type_and_name.h" +#include "vec/columns/column_nullable.h" +#include "vec/data_types/data_type_nullable.h" + namespace doris { using boost::algorithm::join; @@ -80,6 +84,21 @@ void SlotDescriptor::to_protobuf(PSlotDescriptor* pslot) const { pslot->set_is_materialized(_is_materialized); } +vectorized::MutableColumnPtr SlotDescriptor::get_empty_mutable_column() const { + auto data_column = type().get_data_type_ptr()->create_column(); + if (is_nullable()) { + return doris::vectorized::ColumnNullable::create(std::move(data_column), doris::vectorized::ColumnUInt8::create()); + } + return data_column; +} + +vectorized::DataTypePtr SlotDescriptor::get_data_type_ptr() const { + if (is_nullable()) { + return std::make_shared(type().get_data_type_ptr()); + } + return type().get_data_type_ptr(); +} + std::string SlotDescriptor::debug_string() const { std::stringstream out; out << "Slot(id=" << _id << " type=" << _type << " col=" << _col_pos @@ -378,7 +397,7 @@ void RowDescriptor::to_thrift(std::vector* row_tuple_ids) { } void RowDescriptor::to_protobuf( - google::protobuf::RepeatedField* row_tuple_ids) { + google::protobuf::RepeatedField* row_tuple_ids) const { row_tuple_ids->Clear(); for (auto desc : _tuple_desc_map) { row_tuple_ids->Add(desc->id()); diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index d668d424ff..ad432097a3 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -103,6 +103,9 @@ public: std::string debug_string() const; + doris::vectorized::MutableColumnPtr get_empty_mutable_column() const; + + doris::vectorized::DataTypePtr get_data_type_ptr() const; private: friend class DescriptorTbl; friend class TupleDescriptor; @@ -378,7 +381,6 @@ public: int get_row_size() const; int num_materialized_slots() const { - DCHECK(_num_materialized_slots != 0); return _num_materialized_slots; } @@ -405,7 +407,7 @@ public: // Populate row_tuple_ids with our ids. void to_thrift(std::vector* row_tuple_ids); - void to_protobuf(google::protobuf::RepeatedField* row_tuple_ids); + void to_protobuf(google::protobuf::RepeatedField* row_tuple_ids) const; // Return true if the tuple ids of this descriptor are a prefix // of the tuple ids of other_desc. diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 3cc2defa02..0b51e47533 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -93,6 +93,7 @@ public: const std::string& token() const; ExternalScanContextMgr* external_scan_context_mgr() { return _external_scan_context_mgr; } DataStreamMgr* stream_mgr() { return _stream_mgr; } + doris::vectorized::VDataStreamMgr* vstream_mgr() { return _vstream_mgr; } ResultBufferMgr* result_mgr() { return _result_mgr; } ResultQueueMgr* result_queue_mgr() { return _result_queue_mgr; } ClientCache* client_cache() { return _backend_client_cache; } @@ -163,6 +164,7 @@ private: // Leave protected so that subclasses can override ExternalScanContextMgr* _external_scan_context_mgr = nullptr; DataStreamMgr* _stream_mgr = nullptr; + doris::vectorized::VDataStreamMgr* _vstream_mgr = nullptr; ResultBufferMgr* _result_mgr = nullptr; ResultQueueMgr* _result_queue_mgr = nullptr; ClientCache* _backend_client_cache = nullptr; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index a172bb6437..35630d05a6 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -63,6 +63,7 @@ #include "util/parse_util.h" #include "util/pretty_printer.h" #include "util/priority_thread_pool.hpp" +#include "vec/runtime/vdata_stream_mgr.h" namespace doris { @@ -85,6 +86,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { _store_paths = store_paths; _external_scan_context_mgr = new ExternalScanContextMgr(this); _stream_mgr = new DataStreamMgr(); + _vstream_mgr = new doris::vectorized::VDataStreamMgr(); _result_mgr = new ResultBufferMgr(); _result_queue_mgr = new ResultQueueMgr(); _backend_client_cache = new BackendServiceClientCache(config::max_client_cache_size_per_host); diff --git a/be/src/runtime/fold_constant_executor.cpp b/be/src/runtime/fold_constant_executor.cpp index 94d3d9e096..f093c04235 100644 --- a/be/src/runtime/fold_constant_executor.cpp +++ b/be/src/runtime/fold_constant_executor.cpp @@ -28,6 +28,9 @@ #include "common/object_pool.h" #include "common/status.h" +#include "vec/exprs/vexpr.h" +#include "vec/exprs/vexpr_context.h" + #include "gen_cpp/internal_service.pb.h" #include "gen_cpp/PaloInternalService_types.h" @@ -79,7 +82,7 @@ Status FoldConstantExecutor::fold_constant_expr( expr_result.set_success(false); } else { expr_result.set_success(true); - result = _get_result(src, ctx->root()->type().type); + result = _get_result(src, 0, ctx->root()->type().type); } expr_result.set_content(std::move(result)); @@ -96,6 +99,69 @@ Status FoldConstantExecutor::fold_constant_expr( return Status::OK(); } +Status FoldConstantExecutor::fold_constant_vexpr( + const TFoldConstantParams& params, PConstantExprResult* response) { + const auto& expr_map = params.expr_map; + auto expr_result_map = response->mutable_expr_result_map(); + + TQueryGlobals query_globals = params.query_globals; + // init + Status status = _init(query_globals); + if (UNLIKELY(!status.ok())) { + LOG(WARNING) << "Failed to init mem trackers, msg: " << status.get_error_msg(); + return status; + } + + for (const auto& m : expr_map) { + PExprResultMap pexpr_result_map; + for (const auto& n : m.second) { + vectorized::VExprContext* ctx = nullptr; + const TExpr& texpr = n.second; + // create expr tree from TExpr + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(&_pool, texpr, &ctx)); + // prepare and open context + status = _prepare_and_open(ctx); + if (UNLIKELY(!status.ok())) { + LOG(WARNING) << "Failed to init mem trackers, msg: " << status.get_error_msg(); + return status; + } + + vectorized::Block tmp_block; + tmp_block.insert({vectorized::ColumnUInt8::create(1), + std::make_shared(), ""}); + int result_column = -1; + // calc vexpr + RETURN_IF_ERROR(ctx->execute(&tmp_block, &result_column)); + DCHECK(result_column != -1); + PrimitiveType root_type = ctx->root()->type().type; + // covert to thrift type + TPrimitiveType::type t_type = doris::to_thrift(root_type); + + // collect result + PExprResult expr_result; + string result; + const auto& column_ptr = tmp_block.get_by_position(result_column).column; + if (column_ptr->is_null_at(0)) { + expr_result.set_success(false); + } else { + expr_result.set_success(true); + auto string_ref = column_ptr->get_data_at(0); + result = _get_result((void*)string_ref.data, string_ref.size, ctx->root()->type().type); + } + + expr_result.set_content(std::move(result)); + expr_result.mutable_type()->set_type(t_type); + pexpr_result_map.mutable_map()->insert({n.first, expr_result}); + + // close context expr + ctx->close(_runtime_state.get()); + } + expr_result_map->insert({m.first, pexpr_result_map}); + } + + return Status::OK(); +} + Status FoldConstantExecutor::_init(const TQueryGlobals& query_globals) { // init runtime state, runtime profile TPlanFragmentExecParams params; @@ -128,12 +194,14 @@ Status FoldConstantExecutor::_init(const TQueryGlobals& query_globals) { return Status::OK(); } -Status FoldConstantExecutor::_prepare_and_open(ExprContext* ctx) { - ctx->prepare(_runtime_state.get(), RowDescriptor(), _mem_tracker); +template +Status FoldConstantExecutor::_prepare_and_open(Context* ctx) { + RETURN_IF_ERROR(ctx->prepare(_runtime_state.get(), RowDescriptor(), _mem_tracker)); return ctx->open(_runtime_state.get()); } -string FoldConstantExecutor::_get_result(void* src, PrimitiveType slot_type){ +template +string FoldConstantExecutor::_get_result(void* src, size_t size, PrimitiveType slot_type){ switch (slot_type) { case TYPE_BOOLEAN: { bool val = *reinterpret_cast(src); @@ -172,14 +240,24 @@ string FoldConstantExecutor::_get_result(void* src, PrimitiveType slot_type){ case TYPE_STRING: case TYPE_HLL: case TYPE_OBJECT: { + if constexpr (is_vec) { + return std::string((char*)src, size); + } return (reinterpret_cast(src))->to_string(); } case TYPE_DATE: case TYPE_DATETIME: { - const DateTimeValue date_value = *reinterpret_cast(src); - char str[MAX_DTVALUE_STR_LEN]; - date_value.to_string(str); - return str; + if constexpr (is_vec) { + auto date_value = reinterpret_cast(src); + char str[MAX_DTVALUE_STR_LEN]; + date_value->to_string(str); + return str; + } else { + const DateTimeValue date_value = *reinterpret_cast(src); + char str[MAX_DTVALUE_STR_LEN]; + date_value.to_string(str); + return str; + } } case TYPE_DECIMALV2: { return reinterpret_cast(src)->to_string(); diff --git a/be/src/runtime/fold_constant_executor.h b/be/src/runtime/fold_constant_executor.h index 93ee988836..84c52f781d 100644 --- a/be/src/runtime/fold_constant_executor.h +++ b/be/src/runtime/fold_constant_executor.h @@ -36,13 +36,18 @@ class FoldConstantExecutor { public: // fold constant expr Status fold_constant_expr(const TFoldConstantParams& params, PConstantExprResult* response); + + // fold constant vexpr + Status fold_constant_vexpr(const TFoldConstantParams& params, PConstantExprResult* response); private: // init runtime_state and mem_tracker Status _init(const TQueryGlobals& query_globals); // prepare expr - Status _prepare_and_open(ExprContext* ctx); + template + Status _prepare_and_open(Context* ctx); - std::string _get_result(void* src, PrimitiveType slot_type); + template + std::string _get_result(void* src, size_t size, PrimitiveType slot_type); std::unique_ptr _runtime_state; std::shared_ptr _mem_tracker; diff --git a/be/src/runtime/mysql_result_writer.cpp b/be/src/runtime/mysql_result_writer.cpp index 1c2589cc81..eaf1bd7763 100644 --- a/be/src/runtime/mysql_result_writer.cpp +++ b/be/src/runtime/mysql_result_writer.cpp @@ -29,6 +29,13 @@ #include "util/mysql_row_buffer.h" #include "util/types.h" +#include "vec/core/block.h" +#include "vec/columns/column_vector.h" +#include "vec/columns/column_nullable.h" +#include "vec/common/assert_cast.h" +#include "vec/exprs/vexpr.h" +#include "vec/exprs/vexpr_context.h" + namespace doris { MysqlResultWriter::MysqlResultWriter(BufferControlBlock* sinker, @@ -204,7 +211,6 @@ int MysqlResultWriter::_add_row_value(int index, const TypeDescriptor& type, voi } Status MysqlResultWriter::_add_one_row(TupleRow* row) { - SCOPED_TIMER(_convert_tuple_timer); _row_buffer->reset(); int num_columns = _output_expr_ctxs.size(); int buf_ret = 0; diff --git a/be/src/runtime/mysql_result_writer.h b/be/src/runtime/mysql_result_writer.h index 2754e0c868..8b96fec555 100644 --- a/be/src/runtime/mysql_result_writer.h +++ b/be/src/runtime/mysql_result_writer.h @@ -21,6 +21,8 @@ #include "runtime/result_writer.h" #include "runtime/runtime_state.h" +#include "vec/data_types/data_type.h" + namespace doris { class TupleRow; diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index a464211fdb..fe64a4faf3 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -42,6 +42,10 @@ #include "util/uid_util.h" #include "util/logging.h" +#include "vec/core/block.h" +#include "vec/exec/vexchange_node.h" +#include "vec/runtime/vdata_stream_mgr.h" + namespace doris { PlanFragmentExecutor::PlanFragmentExecutor(ExecEnv* exec_env, @@ -159,6 +163,7 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, int num_senders = find_with_default(params.per_exch_num_senders, exch_node->id(), 0); DCHECK_GT(num_senders, 0); if (_runtime_state->enable_vectorized_exec()) { + static_cast(exch_node)->set_num_senders(num_senders); } else { static_cast(exch_node)->set_num_senders(num_senders); } @@ -214,6 +219,7 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request, _row_batch.reset(new RowBatch(_plan->row_desc(), _runtime_state->batch_size(), _runtime_state->instance_mem_tracker().get())); + _block.reset(new doris::vectorized::Block()); // _row_batch->tuple_data_pool()->set_limits(*_runtime_state->mem_trackers()); VLOG_NOTICE << "plan_root=\n" << _plan->debug_string(); _prepared = true; @@ -244,6 +250,7 @@ Status PlanFragmentExecutor::open() { } Status status = Status::OK(); if (_runtime_state->enable_vectorized_exec()) { + status = open_vectorized_internal(); } else { status = open_internal(); } @@ -259,6 +266,82 @@ Status PlanFragmentExecutor::open() { return status; } +Status PlanFragmentExecutor::open_vectorized_internal() { + { + SCOPED_CPU_TIMER(_fragment_cpu_timer); + SCOPED_TIMER(profile()->total_time_counter()); + RETURN_IF_ERROR(_plan->open(_runtime_state.get())); + } + if (_sink == nullptr) { + return Status::OK(); + } + { + SCOPED_CPU_TIMER(_fragment_cpu_timer); + RETURN_IF_ERROR(_sink->open(runtime_state())); + } + doris::vectorized::Block* block = nullptr; + while (true) { + { + SCOPED_CPU_TIMER(_fragment_cpu_timer); + RETURN_IF_ERROR(get_vectorized_internal(&block)); + } + + if (block == NULL) { + break; + } + + SCOPED_TIMER(profile()->total_time_counter()); + SCOPED_CPU_TIMER(_fragment_cpu_timer); + // Collect this plan and sub plan statistics, and send to parent plan. + if (_collect_query_statistics_with_every_batch) { + _collect_query_statistics(); + } + RETURN_IF_ERROR(_sink->send(runtime_state(), block)); + } + { + SCOPED_TIMER(profile()->total_time_counter()); + _collect_query_statistics(); + Status status; + { + std::lock_guard l(_status_lock); + status = _status; + } + status = _sink->close(runtime_state(), status); + RETURN_IF_ERROR(status); + } + // Setting to NULL ensures that the d'tor won't double-close the sink. + _sink.reset(nullptr); + _done = true; + + stop_report_thread(); + send_report(true); + + return Status::OK(); +} +Status PlanFragmentExecutor::get_vectorized_internal(::doris::vectorized::Block** block) { + if (_done) { + *block = nullptr; + return Status::OK(); + } + + auto vexec_node = static_cast(_plan); + while (!_done) { + _block->clear_column_data(vexec_node->row_desc().num_materialized_slots()); + SCOPED_TIMER(profile()->total_time_counter()); + RETURN_IF_ERROR(vexec_node->get_next(_runtime_state.get(), _block.get(), &_done)); + + if (_block->rows() > 0) { + COUNTER_UPDATE(_rows_produced_counter, _block->rows()); + *block = _block.get(); + break; + } + + *block = nullptr; + } + + return Status::OK(); +} + Status PlanFragmentExecutor::open_internal() { { SCOPED_CPU_TIMER(_fragment_cpu_timer); @@ -521,8 +604,14 @@ void PlanFragmentExecutor::cancel() { .query_id(_query_id).instance_id(_runtime_state->fragment_instance_id()); DCHECK(_prepared); _runtime_state->set_is_cancelled(true); - _runtime_state->exec_env()->stream_mgr()->cancel(_runtime_state->fragment_instance_id()); - _runtime_state->exec_env()->result_mgr()->cancel(_runtime_state->fragment_instance_id()); + + // must close stream_mgr to avoid dead lock in Exchange Node + if (_runtime_state->enable_vectorized_exec()) { + _runtime_state->exec_env()->vstream_mgr()->cancel(_runtime_state->fragment_instance_id()); + } else { + _runtime_state->exec_env()->stream_mgr()->cancel(_runtime_state->fragment_instance_id()); + _runtime_state->exec_env()->result_mgr()->cancel(_runtime_state->fragment_instance_id()); + } } void PlanFragmentExecutor::set_abort() { diff --git a/be/src/runtime/plan_fragment_executor.h b/be/src/runtime/plan_fragment_executor.h index 1fc4578240..3cdb6bb249 100644 --- a/be/src/runtime/plan_fragment_executor.h +++ b/be/src/runtime/plan_fragment_executor.h @@ -30,11 +30,11 @@ #include "runtime/runtime_state.h" #include "util/hash_util.hpp" #include "util/time.h" +#include "vec/core/block.h" namespace doris { class QueryFragmentsCtx; -class HdfsFsCache; class ExecNode; class RowDescriptor; class RowBatch; @@ -195,6 +195,7 @@ private: // Created in prepare (if required), owned by this object. std::unique_ptr _sink; std::unique_ptr _row_batch; + std::unique_ptr _block; // Number of rows returned by this fragment RuntimeProfile::Counter* _rows_produced_counter; @@ -234,9 +235,11 @@ private: // have been closed, a final report will have been sent and the report thread will // have been stopped. _sink will be set to nullptr after successful execution. Status open_internal(); + Status open_vectorized_internal(); // Executes get_next() logic and returns resulting status. Status get_next_internal(RowBatch** batch); + Status get_vectorized_internal(::doris::vectorized::Block** block); // Stops report thread, if one is running. Blocks until report thread terminates. // Idempotent. diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h index 36113a55cd..224957f0de 100644 --- a/be/src/runtime/primitive_type.h +++ b/be/src/runtime/primitive_type.h @@ -27,6 +27,13 @@ #include "runtime/decimalv2_value.h" #include "runtime/large_int_value.h" #include "runtime/string_value.h" +#include "udf/udf.h" + +#include "vec/columns/column_decimal.h" +#include "vec/columns/column_string.h" +#include "vec/columns/columns_number.h" +#include "vec/core/types.h" +#include "vec/runtime/vdatetime_value.h" namespace doris { @@ -60,6 +67,51 @@ enum PrimitiveType { TYPE_STRING, /* 23 */ }; +inline PrimitiveType convert_type_to_primitive(FunctionContext::Type type) { + switch (type) { + case FunctionContext::Type::INVALID_TYPE: + return PrimitiveType::INVALID_TYPE; + case FunctionContext::Type::TYPE_DOUBLE: + return PrimitiveType::TYPE_DOUBLE; + case FunctionContext::Type::TYPE_NULL: + return PrimitiveType::TYPE_NULL; + case FunctionContext::Type::TYPE_CHAR: + return PrimitiveType::TYPE_CHAR; + case FunctionContext::Type::TYPE_VARCHAR: + return PrimitiveType::TYPE_VARCHAR; + case FunctionContext::Type::TYPE_STRING: + return PrimitiveType::TYPE_STRING; + case FunctionContext::Type::TYPE_DATETIME: + return PrimitiveType::TYPE_DATETIME; + case FunctionContext::Type::TYPE_DECIMALV2: + return PrimitiveType::TYPE_DECIMALV2; + case FunctionContext::Type::TYPE_BOOLEAN: + return PrimitiveType::TYPE_BOOLEAN; + case FunctionContext::Type::TYPE_ARRAY: + return PrimitiveType::TYPE_ARRAY; + case FunctionContext::Type::TYPE_OBJECT: + return PrimitiveType::TYPE_OBJECT; + case FunctionContext::Type::TYPE_HLL: + return PrimitiveType::TYPE_HLL; + case FunctionContext::Type::TYPE_TINYINT: + return PrimitiveType::TYPE_TINYINT; + case FunctionContext::Type::TYPE_SMALLINT: + return PrimitiveType::TYPE_SMALLINT; + case FunctionContext::Type::TYPE_INT: + return PrimitiveType::TYPE_INT; + case FunctionContext::Type::TYPE_BIGINT: + return PrimitiveType::TYPE_BIGINT; + case FunctionContext::Type::TYPE_LARGEINT: + return PrimitiveType::TYPE_LARGEINT; + case FunctionContext::Type::TYPE_DATE: + return PrimitiveType::TYPE_DATE; + default: + DCHECK(false); + } + + return PrimitiveType::INVALID_TYPE; +} + inline bool is_enumeration_type(PrimitiveType type) { switch (type) { case TYPE_FLOAT: @@ -98,6 +150,10 @@ inline bool is_string_type(PrimitiveType type) { return type == TYPE_CHAR || type == TYPE_VARCHAR || type == TYPE_STRING; } +inline bool has_variable_type(PrimitiveType type) { + return type == TYPE_CHAR || type == TYPE_VARCHAR || type == TYPE_OBJECT || type == TYPE_STRING; +} + // Returns the byte size of 'type' Returns 0 for variable length types. inline int get_byte_size(PrimitiveType type) { switch (type) { @@ -125,9 +181,9 @@ inline int get_byte_size(PrimitiveType type) { case TYPE_DOUBLE: return 8; - case TYPE_LARGEINT: case TYPE_DATETIME: case TYPE_DATE: + case TYPE_LARGEINT: case TYPE_DECIMALV2: return 16; @@ -220,63 +276,78 @@ struct PrimitiveTypeTraits {}; template <> struct PrimitiveTypeTraits { using CppType = bool; + using ColumnType = vectorized::ColumnUInt8; }; template <> struct PrimitiveTypeTraits { using CppType = int8_t; + using ColumnType = vectorized::ColumnInt8; }; template <> struct PrimitiveTypeTraits { using CppType = int16_t; + using ColumnType = vectorized::ColumnInt16; }; template <> struct PrimitiveTypeTraits { using CppType = int32_t; + using ColumnType = vectorized::ColumnInt32; }; template <> struct PrimitiveTypeTraits { using CppType = int64_t; + using ColumnType = vectorized::ColumnInt64; }; template <> struct PrimitiveTypeTraits { using CppType = float; + using ColumnType = vectorized::ColumnFloat32; }; template <> struct PrimitiveTypeTraits { using CppType = double; + using ColumnType = vectorized::ColumnFloat64; }; template <> struct PrimitiveTypeTraits { using CppType = double; + using ColumnType = vectorized::ColumnFloat64; }; template <> struct PrimitiveTypeTraits { - using CppType = DateTimeValue; + using CppType = doris::DateTimeValue; + using ColumnType = vectorized::ColumnVector; }; template <> struct PrimitiveTypeTraits { - using CppType = DateTimeValue; + using CppType = doris::DateTimeValue; + using ColumnType = vectorized::ColumnVector; }; template <> struct PrimitiveTypeTraits { using CppType = DecimalV2Value; + using ColumnType = vectorized::ColumnDecimal; }; template <> struct PrimitiveTypeTraits { using CppType = __int128_t; + using ColumnType = vectorized::ColumnInt128; }; template <> struct PrimitiveTypeTraits { using CppType = StringValue; + using ColumnType = vectorized::ColumnString; }; template <> struct PrimitiveTypeTraits { using CppType = StringValue; + using ColumnType = vectorized::ColumnString; }; template <> struct PrimitiveTypeTraits { using CppType = StringValue; + using ColumnType = vectorized::ColumnString; }; } // namespace doris diff --git a/be/src/runtime/raw_value.h b/be/src/runtime/raw_value.h index e9e270e85b..e16bbf5e13 100644 --- a/be/src/runtime/raw_value.h +++ b/be/src/runtime/raw_value.h @@ -87,6 +87,9 @@ public: // TODO: fix get_hash_value static uint32_t zlib_crc32(const void* value, const TypeDescriptor& type, uint32_t seed); + // Same as the up function, only use in vec exec engine. + static uint32_t zlib_crc32(const void* value, size_t len, const TypeDescriptor& type, uint32_t seed); + // Compares both values. // Return value is < 0 if v1 < v2, 0 if v1 == v2, > 0 if v1 > v2. static int compare(const void* v1, const void* v2, const TypeDescriptor& type); @@ -399,6 +402,59 @@ inline uint32_t RawValue::zlib_crc32(const void* v, const TypeDescriptor& type, } } +// NOTE: this is just for split data, decimal use old doris hash function +// Because crc32 hardware is not equal with zlib crc32 +inline uint32_t RawValue::zlib_crc32(const void* v, size_t len, const TypeDescriptor& type, uint32_t seed) { + // Hash_combine with v = 0 + if (v == nullptr) { + uint32_t value = 0x9e3779b9; + return seed ^ (value + (seed << 6) + (seed >> 2)); + } + + switch (type.type) { + case TYPE_VARCHAR: + case TYPE_HLL: + case TYPE_STRING: + case TYPE_CHAR: { + return HashUtil::zlib_crc_hash(v, len, seed); + } + + case TYPE_BOOLEAN: + case TYPE_TINYINT: + return HashUtil::zlib_crc_hash(v, 1, seed); + case TYPE_SMALLINT: + return HashUtil::zlib_crc_hash(v, 2, seed); + case TYPE_INT: + return HashUtil::zlib_crc_hash(v, 4, seed); + case TYPE_BIGINT: + return HashUtil::zlib_crc_hash(v, 8, seed); + case TYPE_LARGEINT: + return HashUtil::zlib_crc_hash(v, 16, seed); + case TYPE_FLOAT: + return HashUtil::zlib_crc_hash(v, 4, seed); + case TYPE_DOUBLE: + return HashUtil::zlib_crc_hash(v, 8, seed); + case TYPE_DATE: + case TYPE_DATETIME: { + auto* date_val = (const vectorized::VecDateTimeValue*)v; + char buf[64]; + int len = date_val->to_buffer(buf); + return HashUtil::zlib_crc_hash(buf, len, seed); + } + + case TYPE_DECIMALV2: { + const DecimalV2Value* dec_val = (const DecimalV2Value*)v; + int64_t int_val = dec_val->int_value(); + int32_t frac_val = dec_val->frac_value(); + seed = HashUtil::zlib_crc_hash(&int_val, sizeof(int_val), seed); + return HashUtil::zlib_crc_hash(&frac_val, sizeof(frac_val), seed); + } + default: + DCHECK(false) << "invalid type: " << type; + return 0; + } +} + } // namespace doris #endif diff --git a/be/src/runtime/result_sink.cpp b/be/src/runtime/result_sink.cpp index 206b0f0c77..60538caee4 100644 --- a/be/src/runtime/result_sink.cpp +++ b/be/src/runtime/result_sink.cpp @@ -29,6 +29,8 @@ #include "runtime/runtime_state.h" #include "util/uid_util.h" +#include "vec/exprs/vexpr.h" + namespace doris { ResultSink::ResultSink(const RowDescriptor& row_desc, const std::vector& t_output_expr, diff --git a/be/src/runtime/row_batch.cpp b/be/src/runtime/row_batch.cpp index 85ba3c778f..3e7914e2f1 100644 --- a/be/src/runtime/row_batch.cpp +++ b/be/src/runtime/row_batch.cpp @@ -30,8 +30,8 @@ #include "runtime/string_value.h" #include "runtime/tuple_row.h" -//#include "vec/columns/column_vector.h" -//#include "vec/core/block.h" +#include "vec/columns/column_vector.h" +#include "vec/core/block.h" using std::vector; @@ -625,6 +625,57 @@ void RowBatch::transfer_resource_ownership(RowBatch* dest) { reset(); } +vectorized::Block RowBatch::convert_to_vec_block() const { + std::vector columns; + for (const auto tuple_desc : _row_desc.tuple_descriptors()) { + for (const auto slot_desc : tuple_desc->slots()) { + columns.emplace_back(slot_desc->get_empty_mutable_column()); + } + } + + std::vector slot_descs; + std::vector tuple_idx; + int column_numbers = 0; + for (int i = 0; i < _row_desc.tuple_descriptors().size(); ++i) { + auto tuple_desc = _row_desc.tuple_descriptors()[i]; + for (int j = 0; j < tuple_desc->slots().size(); ++j) { + slot_descs.push_back(tuple_desc->slots()[j]); + tuple_idx.push_back(i); + } + column_numbers += tuple_desc->slots().size(); + } + for (int i = 0; i < column_numbers; ++i) { + auto slot_desc = slot_descs[i]; + for (int j = 0; j < _num_rows; ++j) { + TupleRow* src_row = get_row(j); + auto tuple = src_row->get_tuple(tuple_idx[i]); + if (slot_desc->is_nullable() && tuple->is_null(slot_desc->null_indicator_offset())) { + columns[i]->insert_data(nullptr, 0); + } else if (slot_desc->type().is_string_type()) { + auto string_value = + static_cast(tuple->get_slot(slot_desc->tuple_offset())); + columns[i]->insert_data(string_value->ptr, string_value->len); + } else { + columns[i]->insert_data( + static_cast(tuple->get_slot(slot_desc->tuple_offset())), + slot_desc->slot_size()); + } + } + } + + doris::vectorized::ColumnsWithTypeAndName columns_with_type_and_name; + auto n_columns = 0; + for (const auto tuple_desc : _row_desc.tuple_descriptors()) { + for (const auto slot_desc : tuple_desc->slots()) { + columns_with_type_and_name.emplace_back(columns[n_columns++]->get_ptr(), + slot_desc->get_data_type_ptr(), + slot_desc->col_name()); + } + } + + return {columns_with_type_and_name}; +} + size_t RowBatch::get_batch_size(const TRowBatch& batch) { size_t result = batch.tuple_data.size(); result += batch.row_tuples.size() * sizeof(TTupleId); @@ -688,6 +739,7 @@ void RowBatch::deep_copy_to(RowBatch* dst) { } dst->commit_rows(_num_rows); } + // TODO: consider computing size of batches as they are built up size_t RowBatch::total_byte_size() const { size_t result = 0; diff --git a/be/src/runtime/row_batch.h b/be/src/runtime/row_batch.h index 596fa7034c..4333c73618 100644 --- a/be/src/runtime/row_batch.h +++ b/be/src/runtime/row_batch.h @@ -43,6 +43,7 @@ class TupleRow; class TupleDescriptor; class PRowBatch; + // A RowBatch encapsulates a batch of rows, each composed of a number of tuples. // The maximum number of rows is fixed at the time of construction, and the caller // can add rows up to that capacity. @@ -315,7 +316,7 @@ public: // we firstly update dest resource, and then reset current resource void transfer_resource_ownership(RowBatch* dest); - void copy_row(const TupleRow* src, TupleRow* dest) const { + void copy_row(TupleRow* src, TupleRow* dest) { memcpy(dest, src, _num_tuples_per_row * sizeof(Tuple*)); } @@ -362,6 +363,8 @@ public: static size_t get_batch_size(const TRowBatch& batch); static size_t get_batch_size(const PRowBatch& batch); + vectorized::Block convert_to_vec_block() const; + int num_rows() const { return _num_rows; } int capacity() const { return _capacity; } diff --git a/be/src/runtime/string_value.hpp b/be/src/runtime/string_value.hpp index dd29bff24c..c44115d257 100644 --- a/be/src/runtime/string_value.hpp +++ b/be/src/runtime/string_value.hpp @@ -42,22 +42,19 @@ static inline int string_compare(const char* s1, int64_t n1, const char* s2, int int64_t len) { DCHECK_EQ(len, std::min(n1, n2)); #ifdef __SSE4_2__ - if (CpuInfo::is_supported(CpuInfo::SSE4_2)) { - while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) { - __m128i xmm0 = _mm_loadu_si128(reinterpret_cast(s1)); - __m128i xmm1 = _mm_loadu_si128(reinterpret_cast(s2)); - int chars_match = - _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1, - sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE); - if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) { - return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match]; - } - len -= sse_util::CHARS_PER_128_BIT_REGISTER; - s1 += sse_util::CHARS_PER_128_BIT_REGISTER; - s2 += sse_util::CHARS_PER_128_BIT_REGISTER; + while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) { + __m128i xmm0 = _mm_loadu_si128(reinterpret_cast(s1)); + __m128i xmm1 = _mm_loadu_si128(reinterpret_cast(s2)); + int chars_match = + _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1, + sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE); + if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) { + return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match]; } + len -= sse_util::CHARS_PER_128_BIT_REGISTER; + s1 += sse_util::CHARS_PER_128_BIT_REGISTER; + s2 += sse_util::CHARS_PER_128_BIT_REGISTER; } - #endif unsigned char u1, u2; while (len-- > 0) { diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h index e40bc5b0a1..c2adca90fb 100644 --- a/be/src/runtime/types.h +++ b/be/src/runtime/types.h @@ -28,6 +28,12 @@ #include "runtime/collection_value.h" #include "runtime/primitive_type.h" #include "thrift/protocol/TDebugProtocol.h" +#include "vec/data_types/data_type_bitmap.h" +#include "vec/data_types/data_type_date.h" +#include "vec/data_types/data_type_date_time.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/data_types/data_type_number.h" +#include "vec/data_types/data_type_string.h" namespace doris { @@ -272,6 +278,58 @@ struct TypeDescriptor { return -1; } + inline doris::vectorized::DataTypePtr get_data_type_ptr() const { + switch (type) { + case TYPE_BOOLEAN: + return std::make_shared(); + + case TYPE_TINYINT: + return std::make_shared(); + + case TYPE_SMALLINT: + return std::make_shared(); + + case TYPE_INT: + return std::make_shared(); + + case TYPE_FLOAT: + return std::make_shared(); + + case TYPE_BIGINT: + return std::make_shared(); + + case TYPE_LARGEINT: + return std::make_shared(); + case TYPE_DATE: + return std::make_shared(); + case TYPE_DATETIME: + return std::make_shared(); + case TYPE_TIME: + case TYPE_DOUBLE: + return std::make_shared(); + + case TYPE_STRING: + case TYPE_CHAR: + case TYPE_VARCHAR: + case TYPE_HLL: + return std::make_shared(); + case TYPE_OBJECT: + return std::make_shared(); + + case TYPE_DECIMALV2: + return std::make_shared>(27, 9); + // Just Mock A NULL Type in Vec Exec Engine + case TYPE_NULL: + return std::make_shared(); + + case INVALID_TYPE: + default: + DCHECK(false); + } + // For llvm complain + return nullptr; + } + static inline int get_decimal_byte_size(int precision) { DCHECK_GT(precision, 0); if (precision <= MAX_DECIMAL4_PRECISION) { diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index ee6cf7e87a..4106963721 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -36,6 +36,7 @@ #include "util/string_util.h" #include "util/thrift_util.h" #include "util/uid_util.h" +#include "vec/runtime/vdata_stream_mgr.h" namespace doris { @@ -422,7 +423,10 @@ Status PInternalServiceImpl::_fold_constant_expr(const std::string& ser_reque uint32_t len = ser_request.size(); RETURN_IF_ERROR(deserialize_thrift_msg(buf, &len, false, &t_request)); } - return FoldConstantExecutor().fold_constant_expr(t_request, response); + if (!t_request.__isset.vec_exec || !t_request.vec_exec) + return FoldConstantExecutor().fold_constant_expr(t_request, response); + + return FoldConstantExecutor().fold_constant_vexpr(t_request, response); } template @@ -432,6 +436,7 @@ void PInternalServiceImpl::transmit_block(google::protobuf::RpcController* cn google::protobuf::Closure* done) { VLOG_ROW << "transmit data: fragment_instance_id=" << print_id(request->finst_id()) << " node=" << request->node_id(); + _exec_env->vstream_mgr()->transmit_block(request, &done); if (done != nullptr) { done->Run(); } diff --git a/be/src/udf/udf.cpp b/be/src/udf/udf.cpp index 0360f3945c..269bd8891e 100644 --- a/be/src/udf/udf.cpp +++ b/be/src/udf/udf.cpp @@ -131,6 +131,10 @@ void FunctionContextImpl::set_constant_args(const std::vector& constant_cols) { + _constant_cols = constant_cols; +} + bool FunctionContextImpl::check_allocations_empty() { if (_allocations.empty() && _external_bytes_tracked == 0) { return true; @@ -187,6 +191,7 @@ FunctionContext* FunctionContextImpl::clone(MemPool* pool) { create_context(_state, pool, _intermediate_type, _return_type, _arg_types, _varargs_buffer_size, _debug); new_context->_impl->_constant_args = _constant_args; + new_context->_impl->_constant_cols = _constant_cols; new_context->_impl->_fragment_local_fn_state = _fragment_local_fn_state; return new_context; } diff --git a/be/src/udf/udf.h b/be/src/udf/udf.h index 3b9ff5a76d..141219afec 100644 --- a/be/src/udf/udf.h +++ b/be/src/udf/udf.h @@ -28,6 +28,7 @@ // object serves as the interface object between the UDF/UDA and the doris process. namespace doris { class FunctionContextImpl; +class ColumnPtrWrapper; struct StringValue; struct BitmapValue; struct DecimalV2Value; @@ -224,12 +225,16 @@ public: // FunctionContext* argument) is a constant (e.g. 5, "string", 1 + 1). bool is_arg_constant(int arg_idx) const; + bool is_col_constant(int arg_idx) const; + // Returns a pointer to the value of the arg_idx-th input argument (0 indexed, not // including the FunctionContext* argument). Returns nullptr if the argument is not // constant. This function can be used to obtain user-specified constants in a UDF's // Init() or Close() functions. AnyVal* get_constant_arg(int arg_idx) const; + doris::ColumnPtrWrapper* get_constant_col(int arg_idx) const; + // Create a test FunctionContext object. The caller is responsible for calling delete // on it. This context has additional debugging validation enabled. static FunctionContext* create_test_context(); diff --git a/be/src/udf/udf_internal.h b/be/src/udf/udf_internal.h index 2d2c318ed4..085002d8ec 100644 --- a/be/src/udf/udf_internal.h +++ b/be/src/udf/udf_internal.h @@ -32,6 +32,7 @@ namespace doris { class FreePool; class MemPool; class RuntimeState; +class ColumnPtrWrapper; // This class actually implements the interface of FunctionContext. This is split to // hide the details from the external header. @@ -67,6 +68,8 @@ public: void set_constant_args(const std::vector& constant_args); + void set_constant_cols(const std::vector& cols); + uint8_t* varargs_buffer() { return _varargs_buffer; } std::vector* staging_input_vals() { return &_staging_input_vals; } @@ -169,6 +172,8 @@ private: // value of the argument. std::vector _constant_args; + std::vector _constant_cols; + // Used by ScalarFnCall to store the arguments when running without codegen. Allows us // to pass AnyVal* arguments to the scalar function directly, rather than codegening a // call that passes the correct AnyVal subclass pointer type. diff --git a/be/src/udf/udf_ir.cpp b/be/src/udf/udf_ir.cpp index 4eefd175a1..a1ceb02a3c 100644 --- a/be/src/udf/udf_ir.cpp +++ b/be/src/udf/udf_ir.cpp @@ -25,6 +25,13 @@ bool FunctionContext::is_arg_constant(int i) const { return _impl->_constant_args[i] != nullptr; } +bool FunctionContext::is_col_constant(int i) const { + if (i < 0 || i >= _impl->_constant_cols.size()) { + return false; + } + return _impl->_constant_cols[i] != nullptr; +} + AnyVal* FunctionContext::get_constant_arg(int i) const { if (i < 0 || i >= _impl->_constant_args.size()) { return nullptr; @@ -32,6 +39,13 @@ AnyVal* FunctionContext::get_constant_arg(int i) const { return _impl->_constant_args[i]; } +doris::ColumnPtrWrapper* FunctionContext::get_constant_col(int i) const { + if (i < 0 || i >= _impl->_constant_cols.size()) { + return nullptr; + } + return _impl->_constant_cols[i]; +} + int FunctionContext::get_num_args() const { return _impl->_arg_types.size(); } diff --git a/be/src/util/binary_cast.hpp b/be/src/util/binary_cast.hpp index d528d71594..764f5b5f19 100644 --- a/be/src/util/binary_cast.hpp +++ b/be/src/util/binary_cast.hpp @@ -23,7 +23,7 @@ #include "runtime/datetime_value.h" #include "runtime/decimalv2_value.h" #include "util/types.h" - +#include "vec/runtime/vdatetime_value.h" namespace doris { union TypeConverter { uint64_t u64; @@ -56,6 +56,11 @@ union DateTimeInt128Union { ~DateTimeInt128Union() {} }; +union VecDateTimeInt64Union { + doris::vectorized::VecDateTimeValue dt; + __int64_t i64; + ~VecDateTimeInt64Union() {} +}; // similar to reinterpret_cast but won't break strict-aliasing rules template To binary_cast(From from) { @@ -66,11 +71,13 @@ To binary_cast(From from) { constexpr bool from_decv2_to_packed128 = match_v; constexpr bool from_i128_to_dt = match_v; constexpr bool from_dt_to_i128 = match_v; + constexpr bool from_i64_to_vec_dt = match_v; + constexpr bool from_vec_dt_to_i64 = match_v; constexpr bool from_i128_to_decv2 = match_v; constexpr bool from_decv2_to_i128 = match_v; static_assert(from_u64_to_db || from_i64_to_db || from_db_to_i64 || from_db_to_u64 || - from_decv2_to_packed128 || from_i128_to_dt || from_dt_to_i128 || + from_decv2_to_packed128 || from_i128_to_dt || from_dt_to_i128 || from_i64_to_vec_dt || from_vec_dt_to_i64 || from_i128_to_decv2 || from_decv2_to_i128); if constexpr (from_u64_to_db) { @@ -99,6 +106,12 @@ To binary_cast(From from) { } else if constexpr (from_dt_to_i128) { DateTimeInt128Union conv = {.dt = from}; return conv.i128; + } else if constexpr (from_i64_to_vec_dt) { + VecDateTimeInt64Union conv = {.i64 = from}; + return conv.dt; + } else if constexpr (from_vec_dt_to_i64) { + VecDateTimeInt64Union conv = {.dt = from}; + return conv.i64; } else if constexpr (from_i128_to_decv2) { DecimalInt128Union conv; conv.i128 = from; diff --git a/be/src/util/bitmap_value.h b/be/src/util/bitmap_value.h index 81fb10bbd7..bdb124b31a 100644 --- a/be/src/util/bitmap_value.h +++ b/be/src/util/bitmap_value.h @@ -1691,6 +1691,12 @@ public: } return count; } + + void clear() { + _type = EMPTY; + _bitmap.clear(); + _sv = 0; + } // Implement an iterator for convenience friend class BitmapValueIterator; diff --git a/be/src/util/brpc_stub_cache.h b/be/src/util/brpc_stub_cache.h index 53ee3b7a99..21800f3588 100644 --- a/be/src/util/brpc_stub_cache.h +++ b/be/src/util/brpc_stub_cache.h @@ -47,7 +47,7 @@ namespace doris { class BrpcStubCache { public: BrpcStubCache(); - ~BrpcStubCache(); + virtual ~BrpcStubCache(); inline std::shared_ptr get_stub(const butil::EndPoint& endpoint) { auto stub_ptr = _stub_map.find(endpoint); @@ -66,7 +66,7 @@ public: return stub; } - inline std::shared_ptr get_stub(const TNetworkAddress& taddr) { + virtual std::shared_ptr get_stub(const TNetworkAddress& taddr) { butil::EndPoint endpoint; if (str2endpoint(taddr.hostname.c_str(), taddr.port, &endpoint)) { LOG(WARNING) << "unknown endpoint, hostname=" << taddr.hostname diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h index f2632d5a2c..1e9366fd03 100644 --- a/be/src/util/runtime_profile.h +++ b/be/src/util/runtime_profile.h @@ -52,8 +52,9 @@ namespace doris { ScopedTimer MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c) #define CANCEL_SAFE_SCOPED_TIMER(c, is_cancelled) \ ScopedTimer MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c, is_cancelled) -#define SCOPED_RAW_TIMER(c) \ - ScopedRawTimer MACRO_CONCAT(SCOPED_RAW_TIMER, __COUNTER__)(c) +#define SCOPED_RAW_TIMER(c) \ + doris::ScopedRawTimer MACRO_CONCAT(SCOPED_RAW_TIMER, \ + __COUNTER__)(c) #define SCOPED_ATOMIC_TIMER(c) \ ScopedRawTimer> MACRO_CONCAT(SCOPED_ATOMIC_TIMER, \ __COUNTER__)(c) diff --git a/be/src/util/static_asserts.cpp b/be/src/util/static_asserts.cpp index eb50636e32..43833ad121 100644 --- a/be/src/util/static_asserts.cpp +++ b/be/src/util/static_asserts.cpp @@ -17,7 +17,7 @@ #include "runtime/datetime_value.h" #include "runtime/string_value.h" - +#include "vec/runtime/vdatetime_value.h" namespace doris { // This class is unused. It contains static (compile time) asserts. // This is useful to validate struct sizes and other similar things @@ -28,6 +28,7 @@ private: static_assert(offsetof(StringValue, len) == 8); // Datetime value static_assert(sizeof(DateTimeValue) == 16); + static_assert(sizeof(doris::vectorized::VecDateTimeValue) == 8); // static_assert(offsetof(DateTimeValue, _year) == 8); }; diff --git a/be/src/util/string_parser.hpp b/be/src/util/string_parser.hpp index 0354343ed7..cc1110c7a2 100644 --- a/be/src/util/string_parser.hpp +++ b/be/src/util/string_parser.hpp @@ -572,6 +572,26 @@ T StringParser::numeric_limits(bool negative) { return negative ? std::numeric_limits::min() : std::numeric_limits::max(); } +template<> +inline int StringParser::StringParseTraits::max_ascii_len() { + return 3; +} + +template<> +inline int StringParser::StringParseTraits::max_ascii_len() { + return 5; +} + +template<> +inline int StringParser::StringParseTraits::max_ascii_len() { + return 10; +} + +template<> +inline int StringParser::StringParseTraits::max_ascii_len() { + return 20; +} + template<> inline int StringParser::StringParseTraits::max_ascii_len() { return 3; diff --git a/be/src/util/url_coding.cpp b/be/src/util/url_coding.cpp index 862c34ccd7..3c41114fc6 100644 --- a/be/src/util/url_coding.cpp +++ b/be/src/util/url_coding.cpp @@ -194,6 +194,10 @@ int64_t base64_decode(const char* data, size_t length, char* decoded_data) { // run through the whole string, converting as we go while ((ch = *current++) != '\0' && length-- > 0) { + if (ch >= 256 || ch < 0) { + return -1; + } + if (ch == base64_pad) { if (*current != '=' && (i % 4) == 1) { return -1; diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt new file mode 100644 index 0000000000..f73739122d --- /dev/null +++ b/be/src/vec/CMakeLists.txt @@ -0,0 +1,159 @@ +# 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. +# where to put generated libraries +set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/vec") +# where to put generated binaries +set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/vec") + +set(VEC_FILES + aggregate_functions/aggregate_function_avg.cpp + aggregate_functions/aggregate_function_count.cpp + aggregate_functions/aggregate_function_distinct.cpp + aggregate_functions/aggregate_function_sum.cpp + aggregate_functions/aggregate_function_min_max.cpp + aggregate_functions/aggregate_function_null.cpp + aggregate_functions/aggregate_function_uniq.cpp + aggregate_functions/aggregate_function_hll_union_agg.cpp + aggregate_functions/aggregate_function_bitmap.cpp + aggregate_functions/aggregate_function_reader.cpp + aggregate_functions/aggregate_function_window.cpp + aggregate_functions/aggregate_function_stddev.cpp + aggregate_functions/aggregate_function_simple_factory.cpp + columns/collator.cpp + columns/column.cpp + columns/column_const.cpp + columns/column_decimal.cpp + columns/column_nullable.cpp + columns/column_string.cpp + columns/column_vector.cpp + columns/columns_common.cpp + common/demangle.cpp + common/exception.cpp + common/pod_array.cpp + common/string_utils/string_utils.cpp + core/block.cpp + core/block_info.cpp + core/column_with_type_and_name.cpp + core/field.cpp + core/field.cpp + core/sort_block.cpp + core/materialize_block.cpp + data_types/data_type.cpp + data_types/data_type_bitmap.cpp + data_types/data_type_nothing.cpp + data_types/data_type_nothing.cpp + data_types/data_type_nullable.cpp + data_types/data_type_nullable.cpp + data_types/data_type_number_base.cpp + data_types/data_type_string.cpp + data_types/data_type_decimal.cpp + data_types/get_least_supertype.cpp + data_types/nested_utils.cpp + data_types/data_type_date.cpp + data_types/data_type_date_time.cpp + exec/vaggregation_node.cpp + exec/ves_http_scan_node.cpp + exec/ves_http_scanner.cpp + exec/volap_scan_node.cpp + exec/vsort_node.cpp + exec/vsort_exec_exprs.cpp + exec/volap_scanner.cpp + exec/vexchange_node.cpp + exec/vset_operation_node.cpp + exec/vunion_node.cpp + exec/vintersect_node.cpp + exec/vexcept_node.cpp + exec/vselect_node.cpp + exec/vblocking_join_node.cpp + exec/vcross_join_node.cpp + exec/vodbc_scan_node.cpp + exec/vmysql_scan_node.cpp + exec/vschema_scan_node.cpp + exec/vempty_set_node.cpp + exec/vanalytic_eval_node.cpp + exec/vassert_num_rows_node.cpp + exec/vrepeat_node.cpp + exec/join/vhash_join_node.cpp + exprs/vectorized_agg_fn.cpp + exprs/vectorized_fn_call.cpp + exprs/vexpr.cpp + exprs/vexpr_context.cpp + exprs/vliteral.cpp + exprs/vin_predicate.cpp + exprs/vslot_ref.cpp + exprs/vcast_expr.cpp + exprs/vcase_expr.cpp + exprs/vinfo_func.cpp + functions/math.cpp + functions/function_bitmap.cpp + functions/comparison.cpp + functions/comparison_less.cpp + functions/comparison_equals.cpp + functions/comparison_greater.cpp + functions/function.cpp + functions/function_helpers.cpp + functions/function_hash.cpp + functions/functions_logical.cpp + functions/function_case.cpp + functions/function_cast.cpp + functions/function_conv.cpp + functions/function_string.cpp + functions/function_timestamp.cpp + functions/function_utility.cpp + functions/comparison_equal_for_null.cpp + functions/function_json.cpp + functions/hll_cardinality.cpp + functions/hll_empty.cpp + functions/hll_hash.cpp + functions/plus.cpp + functions/modulo.cpp + functions/multiply.cpp + functions/minus.cpp + functions/int_div.cpp + functions/divide.cpp + functions/function_bit.cpp + functions/is_null.cpp + functions/is_not_null.cpp + functions/in.cpp + functions/like.cpp + functions/to_time_function.cpp + functions/time_of_function.cpp + functions/if.cpp + functions/function_ifnull.cpp + functions/nullif.cpp + functions/random.cpp + functions/function_coalesce.cpp + functions/function_date_or_datetime_computation.cpp + functions/function_date_or_datetime_to_string.cpp + functions/function_datetime_string_to_string.cpp + functions/function_grouping.cpp + olap/vgeneric_iterators.cpp + olap/vcollect_iterator.cpp + olap/block_reader.cpp + sink/mysql_result_writer.cpp + sink/result_sink.cpp + sink/vdata_stream_sender.cpp + sink/vtabet_sink.cpp + runtime/vdatetime_value.cpp + runtime/vdata_stream_recvr.cpp + runtime/vdata_stream_mgr.cpp + runtime/vpartition_info.cpp + runtime/vsorted_run_merger.cpp) + +add_library(Vec STATIC + ${VEC_FILES} +) diff --git a/be/src/vec/aggregate_functions/aggregate_function.h b/be/src/vec/aggregate_functions/aggregate_function.h new file mode 100644 index 0000000000..4c2ef36d7b --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function.h @@ -0,0 +1,242 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/IAggregateFunction.h +// and modified by Doris + +#pragma once + +#include +#include +#include +#include +#include +#include + +#include "vec/common/exception.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/core/field.h" +#include "vec/core/types.h" + +namespace doris::vectorized { + +class Arena; +class IColumn; +class IDataType; + +using DataTypePtr = std::shared_ptr; +using DataTypes = std::vector; + +using AggregateDataPtr = char*; +using ConstAggregateDataPtr = const char*; + +/** Aggregate functions interface. + * Instances of classes with this interface do not contain the data itself for aggregation, + * but contain only metadata (description) of the aggregate function, + * as well as methods for creating, deleting and working with data. + * The data resulting from the aggregation (intermediate computing states) is stored in other objects + * (which can be created in some memory pool), + * and IAggregateFunction is the external interface for manipulating them. + */ +class IAggregateFunction { +public: + IAggregateFunction(const DataTypes& argument_types_, const Array& parameters_) + : argument_types(argument_types_), parameters(parameters_) {} + + /// Get main function name. + virtual String get_name() const = 0; + + /// Get the result type. + virtual DataTypePtr get_return_type() const = 0; + + virtual ~IAggregateFunction() {} + + /** Create empty data for aggregation with `placement new` at the specified location. + * You will have to destroy them using the `destroy` method. + */ + virtual void create(AggregateDataPtr __restrict place) const = 0; + + /// Delete data for aggregation. + virtual void destroy(AggregateDataPtr __restrict place) const noexcept = 0; + + /// Reset aggregation state + virtual void reset(AggregateDataPtr place) const = 0; + + /// It is not necessary to delete data. + virtual bool has_trivial_destructor() const = 0; + + /// Get `sizeof` of structure with data. + virtual size_t size_of_data() const = 0; + + /// How the data structure should be aligned. NOTE: Currently not used (structures with aggregation state are put without alignment). + virtual size_t align_of_data() const = 0; + + /** Adds a value into aggregation data on which place points to. + * columns points to columns containing arguments of aggregation function. + * row_num is number of row which should be added. + * Additional parameter arena should be used instead of standard memory allocator if the addition requires memory allocation. + */ + virtual void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num, + Arena* arena) const = 0; + + /// Merges state (on which place points to) with other state of current aggregation function. + virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena* arena) const = 0; + + /// Serializes state (to transmit it over the network, for example). + virtual void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const = 0; + + /// Deserializes state. This function is called only for empty (just created) states. + virtual void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena* arena) const = 0; + + /// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()). + virtual bool allocates_memory_in_arena() const { return false; } + + /// Inserts results into a column. + virtual void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const = 0; + + /** Returns true for aggregate functions of type -State. + * They are executed as other aggregate functions, but not finalized (return an aggregation state that can be combined with another). + */ + virtual bool is_state() const { return false; } + + /// if return false, during insert_result_into function, you colud get nullable result column, + /// so could insert to null value by yourself, rather than by AggregateFunctionNullBase; + /// because you maybe be calculate a invalid value, but want to use null replace it; + virtual bool insert_to_null_default() const { return true; } + + /** The inner loop that uses the function pointer is better than using the virtual function. + * The reason is that in the case of virtual functions GCC 5.1.2 generates code, + * which, at each iteration of the loop, reloads the function address (the offset value in the virtual function table) from memory to the register. + * This gives a performance drop on simple queries around 12%. + * After the appearance of better compilers, the code can be removed. + */ + using AddFunc = void (*)(const IAggregateFunction*, AggregateDataPtr, const IColumn**, size_t, + Arena*); + virtual AddFunc get_address_of_add_function() const = 0; + + /** Contains a loop with calls to "add" function. You can collect arguments into array "places" + * and do a single call to "add_batch" for devirtualization and inlining. + */ + virtual void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset, + const IColumn** columns, Arena* arena) const = 0; + + /** The same for single place. + */ + virtual void add_batch_single_place(size_t batch_size, AggregateDataPtr place, + const IColumn** columns, Arena* arena) const = 0; + + // only used at agg reader + virtual void add_batch_range(size_t batch_begin, size_t batch_end, AggregateDataPtr place, + const IColumn** columns, Arena* arena, bool has_null = false) = 0; + + // only used at window function + virtual void add_range_single_place(int64_t partition_start, int64_t partition_end, + int64_t frame_start, int64_t frame_end, + AggregateDataPtr place, const IColumn** columns, + Arena* arena) const = 0; + + /** This is used for runtime code generation to determine, which header files to include in generated source. + * Always implement it as + * const char * get_header_file_path() const override { return __FILE__; } + */ + virtual const char* get_header_file_path() const = 0; + + const DataTypes& get_argument_types() const { return argument_types; } + const Array& get_parameters() const { return parameters; } + +protected: + DataTypes argument_types; + Array parameters; +}; + +/// Implement method to obtain an address of 'add' function. +template +class IAggregateFunctionHelper : public IAggregateFunction { +private: + static void add_free(const IAggregateFunction* that, AggregateDataPtr place, + const IColumn** columns, size_t row_num, Arena* arena) { + static_cast(*that).add(place, columns, row_num, arena); + } + +public: + IAggregateFunctionHelper(const DataTypes& argument_types_, const Array& parameters_) + : IAggregateFunction(argument_types_, parameters_) {} + + AddFunc get_address_of_add_function() const override { return &add_free; } + + void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset, + const IColumn** columns, Arena* arena) const override { + for (size_t i = 0; i < batch_size; ++i) + static_cast(this)->add(places[i] + place_offset, columns, i, arena); + } + + void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, + Arena* arena) const override { + for (size_t i = 0; i < batch_size; ++i) + static_cast(this)->add(place, columns, i, arena); + } + //now this is use for sum/count/avg/min/max win function, other win function should override this function in class + void add_range_single_place(int64_t partition_start, int64_t partition_end, int64_t frame_start, + int64_t frame_end, AggregateDataPtr place, const IColumn** columns, + Arena* arena) const override { + frame_start = std::max(frame_start, partition_start); + frame_end = std::min(frame_end, partition_end); + for (int64_t i = frame_start; i < frame_end; ++i) { + static_cast(this)->add(place, columns, i, arena); + } + } + + void add_batch_range(size_t batch_begin, size_t batch_end, AggregateDataPtr place, + const IColumn** columns, Arena* arena, bool has_null) override { + for (size_t i = batch_begin; i <= batch_end; ++i) + static_cast(this)->add(place, columns, i, arena); + } +}; + +/// Implements several methods for manipulation with data. T - type of structure with data for aggregation. +template +class IAggregateFunctionDataHelper : public IAggregateFunctionHelper { +protected: + using Data = T; + + static Data& data(AggregateDataPtr __restrict place) { return *reinterpret_cast(place); } + static const Data& data(ConstAggregateDataPtr __restrict place) { + return *reinterpret_cast(place); + } + +public: + IAggregateFunctionDataHelper(const DataTypes& argument_types_, const Array& parameters_) + : IAggregateFunctionHelper(argument_types_, parameters_) {} + + void create(AggregateDataPtr __restrict place) const override { new (place) Data; } + + void destroy(AggregateDataPtr __restrict place) const noexcept override { data(place).~Data(); } + + bool has_trivial_destructor() const override { return std::is_trivially_destructible_v; } + + size_t size_of_data() const override { return sizeof(Data); } + + /// NOTE: Currently not used (structures with aggregation state are put without alignment). + size_t align_of_data() const override { return alignof(Data); } + + void reset(AggregateDataPtr place) const override {} +}; + +using AggregateFunctionPtr = std::shared_ptr; + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg.cpp b/be/src/vec/aggregate_functions/aggregate_function_avg.cpp new file mode 100644 index 0000000000..bb7605bedf --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_avg.cpp @@ -0,0 +1,68 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionAvg.cpp +// and modified by Doris + +#include "vec/aggregate_functions/aggregate_function_avg.h" + +#include "common/logging.h" +#include "vec/aggregate_functions/aggregate_function_simple_factory.h" +#include "vec/aggregate_functions/factory_helpers.h" +#include "vec/aggregate_functions/helpers.h" + +namespace doris::vectorized { + +namespace { + +template +struct Avg { + using FieldType = std::conditional_t, Decimal128, NearestFieldType>; + using Function = AggregateFunctionAvg>; +}; + +template +using AggregateFuncAvg = typename Avg::Function; + +AggregateFunctionPtr create_aggregate_function_avg(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + assert_no_parameters(name, parameters); + assert_unary(name, argument_types); + + AggregateFunctionPtr res; + DataTypePtr data_type = argument_types[0]; + if (is_decimal(data_type)) + res.reset( + create_with_decimal_type(*data_type, *data_type, argument_types)); + else + res.reset(create_with_numeric_type(*data_type, argument_types)); + + if (!res) { + LOG(WARNING) << fmt::format("Illegal type {} of argument for aggregate function {}", + argument_types[0]->get_name(), name); + } + return res; +} + +} // namespace + +void register_aggregate_function_avg(AggregateFunctionSimpleFactory& factory) { + factory.register_function("avg", create_aggregate_function_avg); +} +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg.h b/be/src/vec/aggregate_functions/aggregate_function_avg.h new file mode 100644 index 0000000000..18584ee91a --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_avg.h @@ -0,0 +1,128 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionAvg.h +// and modified by Doris + +#pragma once + +#include "common/status.h" +#include "vec/aggregate_functions/aggregate_function.h" +#include "vec/columns/columns_number.h" +#include "vec/data_types/data_type_decimal.h" +#include "vec/data_types/data_type_number.h" +#include "vec/io/io_helper.h" + +namespace doris::vectorized { + +template +struct AggregateFunctionAvgData { + T sum = 0; + UInt64 count = 0; + + template + ResultT result() const { + if constexpr (std::is_floating_point_v) + if constexpr (std::numeric_limits::is_iec559) + return static_cast(sum) / count; /// allow division by zero + + if (!count) + throw Exception("AggregateFunctionAvg with zero values", TStatusCode::VEC_LOGIC_ERROR); + return static_cast(sum) / count; + } + + void write(BufferWritable& buf) const { + write_binary(sum, buf); + write_binary(count, buf); + } + + void read(BufferReadable& buf) { + read_binary(sum, buf); + read_binary(count, buf); + } +}; + +/// Calculates arithmetic mean of numbers. +template +class AggregateFunctionAvg final + : public IAggregateFunctionDataHelper> { +public: + using ResultType = std::conditional_t, Decimal128, Float64>; + using ResultDataType = std::conditional_t, DataTypeDecimal, + DataTypeNumber>; + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, + ColumnVector>; + + /// ctor for native types + AggregateFunctionAvg(const DataTypes& argument_types_) + : IAggregateFunctionDataHelper>(argument_types_, + {}), + scale(0) {} + + /// ctor for Decimals + AggregateFunctionAvg(const IDataType& data_type, const DataTypes& argument_types_) + : IAggregateFunctionDataHelper>(argument_types_, + {}), + scale(get_decimal_scale(data_type)) {} + + String get_name() const override { return "avg"; } + + DataTypePtr get_return_type() const override { + if constexpr (IsDecimalNumber) + return std::make_shared(ResultDataType::max_precision(), scale); + else + return std::make_shared(); + } + + void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num, + Arena*) const override { + const auto& column = static_cast(*columns[0]); + this->data(place).sum += column.get_data()[row_num]; + ++this->data(place).count; + } + + void reset(AggregateDataPtr place) const override { + this->data(place).sum = 0; + this->data(place).count = 0; + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + this->data(place).sum += this->data(rhs).sum; + this->data(place).count += this->data(rhs).count; + } + + void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { + this->data(place).write(buf); + } + + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + this->data(place).read(buf); + } + + void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + auto& column = static_cast(to); + column.get_data().push_back(this->data(place).template result()); + } + + const char* get_header_file_path() const override { return __FILE__; } + +private: + UInt32 scale; +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.cpp b/be/src/vec/aggregate_functions/aggregate_function_bitmap.cpp new file mode 100644 index 0000000000..d110b09e1e --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.cpp @@ -0,0 +1,87 @@ +// 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/aggregate_functions/aggregate_function_bitmap.h" + +#include "vec/aggregate_functions/aggregate_function_simple_factory.h" + +namespace doris::vectorized { + +template class AggregateFunctionTemplate> +static IAggregateFunction* createWithIntDataType(const DataTypes& argument_type) { + auto type = argument_type[0].get(); + if (type->is_nullable()) { + type = assert_cast(type)->get_nested_type().get(); + } + WhichDataType which(type); + if (which.idx == TypeIndex::Int8) + return new AggregateFunctionTemplate>(argument_type); + if (which.idx == TypeIndex::Int16) + return new AggregateFunctionTemplate>(argument_type); + if (which.idx == TypeIndex::Int32) + return new AggregateFunctionTemplate>(argument_type); + if (which.idx == TypeIndex::Int64) + return new AggregateFunctionTemplate>(argument_type); + return nullptr; +} + +AggregateFunctionPtr create_aggregate_function_bitmap_union(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + return std::make_shared>( + argument_types); +} + +AggregateFunctionPtr create_aggregate_function_bitmap_intersect(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + return std::make_shared>( + argument_types); +} +template +AggregateFunctionPtr create_aggregate_function_bitmap_union_count(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + return std::make_shared>(argument_types); +} + +template +AggregateFunctionPtr create_aggregate_function_bitmap_union_int(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + return std::shared_ptr( + createWithIntDataType(argument_types)); +} + +void register_aggregate_function_bitmap(AggregateFunctionSimpleFactory& factory) { + factory.register_function("bitmap_union", create_aggregate_function_bitmap_union); + factory.register_function("bitmap_intersect", create_aggregate_function_bitmap_intersect); + factory.register_function("bitmap_union_count", + create_aggregate_function_bitmap_union_count); + factory.register_function("bitmap_union_count", + create_aggregate_function_bitmap_union_count, true); + + factory.register_function("bitmap_union_int", + create_aggregate_function_bitmap_union_int); + factory.register_function("bitmap_union_int", create_aggregate_function_bitmap_union_int, + true); +} +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h new file mode 100644 index 0000000000..a2e43e5392 --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h @@ -0,0 +1,175 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once +#include +#include + +#include "vec/aggregate_functions/aggregate_function.h" +#include "vec/columns/column_complex.h" +#include "vec/columns/column_nullable.h" +#include "vec/common/assert_cast.h" +#include "vec/data_types/data_type_bitmap.h" +#include "vec/data_types/data_type_nullable.h" +#include "vec/data_types/data_type_number.h" +#include "vec/io/io_helper.h" + +namespace doris::vectorized { + +struct AggregateFunctionBitmapUnionOp { + static constexpr auto name = "bitmap_union"; + + template + static void add(BitmapValue& res, const T& data) { + res.add(data); + } + + static void add(BitmapValue& res, const BitmapValue& data) { res |= data; } + + static void merge(BitmapValue& res, const BitmapValue& data) { res |= data; } +}; + +struct AggregateFunctionBitmapIntersectOp { + static constexpr auto name = "bitmap_intersect"; + static void add(BitmapValue& res, const BitmapValue& data) { res &= data; } + + static void merge(BitmapValue& res, const BitmapValue& data) { res &= data; } +}; + +template +struct AggregateFunctionBitmapData { + BitmapValue value; + + template + void add(const T& data) { + Op::add(value, data); + } + + void merge(const BitmapValue& data) { Op::merge(value, data); } + + void write(BufferWritable& buf) const { DataTypeBitMap::serialize_as_stream(value, buf); } + + void read(BufferReadable& buf) { DataTypeBitMap::deserialize_as_stream(value, buf); } + + BitmapValue& get() { return value; } +}; + +template +class AggregateFunctionBitmapOp final + : public IAggregateFunctionDataHelper, + AggregateFunctionBitmapOp> { +public: + using ResultDataType = BitmapValue; + using ColVecType = ColumnBitmap; + using ColVecResult = ColumnBitmap; + + String get_name() const override { return Op::name; } + + AggregateFunctionBitmapOp(const DataTypes& argument_types_) + : IAggregateFunctionDataHelper, + AggregateFunctionBitmapOp>(argument_types_, {}) {} + + DataTypePtr get_return_type() const override { return std::make_shared(); } + + void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num, + Arena*) const override { + const auto& column = static_cast(*columns[0]); + this->data(place).add(column.get_data()[row_num]); + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + this->data(place).merge( + const_cast&>(this->data(rhs)).get()); + } + + void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { + this->data(place).write(buf); + } + + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + this->data(place).read(buf); + } + + void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + auto& column = static_cast(to); + column.get_data().push_back( + const_cast&>(this->data(place)).get()); + } + + const char* get_header_file_path() const override { return __FILE__; } +}; + +template +class AggregateFunctionBitmapCount final + : public IAggregateFunctionDataHelper< + AggregateFunctionBitmapData, + AggregateFunctionBitmapCount> { +public: + // using ColVecType = ColumnBitmap; + using ColVecResult = ColumnVector; + using AggFunctionData = AggregateFunctionBitmapData; + + AggregateFunctionBitmapCount(const DataTypes& argument_types_) + : IAggregateFunctionDataHelper< + AggregateFunctionBitmapData, + AggregateFunctionBitmapCount>(argument_types_, {}) {} + + String get_name() const override { return "count"; } + DataTypePtr get_return_type() const override { return std::make_shared(); } + + void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num, + Arena*) const override { + if constexpr (nullable) { + auto& nullable_column = assert_cast(*columns[0]); + if (!nullable_column.is_null_at(row_num)) { + const auto& column = + static_cast(nullable_column.get_nested_column()); + this->data(place).add(column.get_data()[row_num]); + } + } else { + const auto& column = static_cast(*columns[0]); + this->data(place).add(column.get_data()[row_num]); + } + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + this->data(place).merge(const_cast(this->data(rhs)).get()); + } + + void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { + this->data(place).write(buf); + } + + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + this->data(place).read(buf); + } + + void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + auto& value_data = const_cast(this->data(place)).get(); + auto& column = static_cast(to); + column.get_data().push_back(value_data.cardinality()); + } + + const char* get_header_file_path() const override { return __FILE__; } +}; + +AggregateFunctionPtr create_aggregate_function_bitmap_union(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable); + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/aggregate_functions/aggregate_function_combinator.h b/be/src/vec/aggregate_functions/aggregate_function_combinator.h new file mode 100644 index 0000000000..77d465b236 --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_combinator.h @@ -0,0 +1,81 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/IAggregateFunctionCombinator.h +// and modified by Doris + +#pragma once + +#include +#include + +#include + +namespace doris::vectorized { + +/** Aggregate function combinator allows to take one aggregate function + * and transform it to another aggregate function. + * + * In SQL language they are used as suffixes for existing aggregate functions. + * + * Example: -If combinator takes an aggregate function and transforms it + * to aggregate function with additional argument at end (condition), + * that will pass values to original aggregate function when the condition is true. + * + * More examples: + * + * sum(x) - calculate sum of x + * sumIf(x, cond) - calculate sum of x for rows where condition is true. + * sumArray(arr) - calculate sum of all elements of arrays. + * + * PS. Please don't mess it with so called "combiner" - totally unrelated notion from Hadoop world. + * "combining" - merging the states of aggregate functions - is supported naturally in ClickHouse. + */ + +class IAggregateFunctionCombinator { +public: + virtual String get_name() const = 0; + + virtual bool is_for_internal_usage_only() const { return false; } + + /** From the arguments for combined function (ex: UInt64, UInt8 for sumIf), + * get the arguments for nested function (ex: UInt64 for sum). + * If arguments are not suitable for combined function, throw an exception. + */ + virtual DataTypes transform_arguments(const DataTypes& arguments) const { return arguments; } + + /** From the parameters for combined function, + * get the parameters for nested function. + * If arguments are not suitable for combined function, throw an exception. + */ + virtual Array transform_parameters(const Array& parameters) const { return parameters; } + + /** Create combined aggregate function (ex: sumIf) + * from nested function (ex: sum) + * and arguments for combined agggregate function (ex: UInt64, UInt8 for sumIf). + * It's assumed that function transform_arguments was called before this function and 'arguments' are validated. + */ + virtual AggregateFunctionPtr transform_aggregate_function( + const AggregateFunctionPtr& nested_function, const DataTypes& arguments, + const Array& params, const bool result_is_nullable) const = 0; + + virtual ~IAggregateFunctionCombinator() {} +}; + +using AggregateFunctionCombinatorPtr = std::shared_ptr; + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_count.cpp b/be/src/vec/aggregate_functions/aggregate_function_count.cpp new file mode 100644 index 0000000000..aa5d0061eb --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_count.cpp @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionCount.cpp +// and modified by Doris + +#include "vec/aggregate_functions/aggregate_function_count.h" + +#include "vec/aggregate_functions/aggregate_function_simple_factory.h" +#include "vec/aggregate_functions/factory_helpers.h" + +namespace doris::vectorized { + +AggregateFunctionPtr create_aggregate_function_count(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + assert_no_parameters(name, parameters); + assert_arity_at_most<1>(name, argument_types); + + return std::make_shared(argument_types); +} + +AggregateFunctionPtr create_aggregate_function_count_not_null_unary(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + assert_arity_at_most<1>(name, argument_types); + + return std::make_shared(argument_types); +} + +void register_aggregate_function_count(AggregateFunctionSimpleFactory& factory) { + factory.register_function("count", create_aggregate_function_count); + factory.register_function("count", create_aggregate_function_count_not_null_unary, true); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_count.h b/be/src/vec/aggregate_functions/aggregate_function_count.h new file mode 100644 index 0000000000..fd096dc6be --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_count.h @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionCount.h +// and modified by Doris + +#pragma once + +#include + +#include "common/logging.h" +#include "vec/aggregate_functions/aggregate_function.h" +#include "vec/columns/column_nullable.h" +#include "vec/common/assert_cast.h" +#include "vec/data_types/data_type_number.h" +#include "vec/io/io_helper.h" + +namespace doris::vectorized { + +struct AggregateFunctionCountData { + UInt64 count = 0; +}; + +/// Simply count number of calls. +class AggregateFunctionCount final + : public IAggregateFunctionDataHelper { +public: + AggregateFunctionCount(const DataTypes& argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}) {} + + String get_name() const override { return "count"; } + + DataTypePtr get_return_type() const override { return std::make_shared(); } + + void add(AggregateDataPtr __restrict place, const IColumn**, size_t, Arena*) const override { + ++data(place).count; + } + + void reset(AggregateDataPtr place) const override { + this->data(place).count = 0; + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + data(place).count += data(rhs).count; + } + + void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { + write_var_uint(data(place).count, buf); + } + + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + read_var_uint(data(place).count, buf); + } + + void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + assert_cast(to).get_data().push_back(data(place).count); + } + + const char* get_header_file_path() const override { return __FILE__; } +}; + +/// Simply count number of not-NULL values. +class AggregateFunctionCountNotNullUnary final + : public IAggregateFunctionDataHelper { +public: + AggregateFunctionCountNotNullUnary(const DataTypes& argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}) {} + + String get_name() const override { return "count"; } + + DataTypePtr get_return_type() const override { return std::make_shared(); } + + void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num, + Arena*) const override { + data(place).count += !assert_cast(*columns[0]).is_null_at(row_num); + } + + void reset(AggregateDataPtr place) const override { + data(place).count = 0; + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + data(place).count += data(rhs).count; + } + + void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { + write_var_uint(data(place).count, buf); + } + + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + read_var_uint(data(place).count, buf); + } + + void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + if (to.is_nullable()) { + auto& null_column = assert_cast(to); + null_column.get_null_map_data().push_back(0); + assert_cast(null_column.get_nested_column()).get_data().push_back(data(place).count); + } else { + assert_cast(to).get_data().push_back(data(place).count); + } + } + + const char* get_header_file_path() const override { return __FILE__; } +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_distinct.cpp b/be/src/vec/aggregate_functions/aggregate_function_distinct.cpp new file mode 100644 index 0000000000..af64a3a15d --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_distinct.cpp @@ -0,0 +1,96 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionDistinct.cpp +// and modified by Doris + +#include +#include + +#include "vec/aggregate_functions/aggregate_function_combinator.h" +#include "vec/aggregate_functions/aggregate_function_distinct.h" +#include "vec/aggregate_functions/aggregate_function_simple_factory.h" +#include "vec/aggregate_functions/helpers.h" +#include "vec/common/typeid_cast.h" +#include "vec/data_types/data_type_nullable.h" + +namespace doris::vectorized { + +class AggregateFunctionCombinatorDistinct final : public IAggregateFunctionCombinator { +public: + String get_name() const override { return "Distinct"; } + + DataTypes transform_arguments(const DataTypes& arguments) const override { + if (arguments.empty()) { + LOG(FATAL) + << "Incorrect number of arguments for aggregate function with Distinct suffix"; + } + return arguments; + } + + AggregateFunctionPtr transform_aggregate_function(const AggregateFunctionPtr& nested_function, + const DataTypes& arguments, + const Array& params, + const bool result_is_nullable) const override { + DCHECK(nested_function != nullptr); + if (nested_function == nullptr) return nullptr; + + AggregateFunctionPtr res; + if (arguments.size() == 1) { + res.reset(create_with_numeric_type( + *arguments[0], nested_function, arguments)); + + if (res) return res; + + if (arguments[0]->is_value_unambiguously_represented_in_contiguous_memory_region()) + return std::make_shared>>(nested_function, + arguments); + else + return std::make_shared>>(nested_function, + arguments); + } + + return std::make_shared< + AggregateFunctionDistinct>( + nested_function, arguments); + } +}; + +const std::string DISTINCT_FUNCTION_PREFIX = "multi_distinct_"; + +void register_aggregate_function_combinator_distinct(AggregateFunctionSimpleFactory& factory) { + AggregateFunctionCreator creator = [&](const std::string& name, const DataTypes& types, + const Array& params, const bool result_is_nullable) { + // 1. we should get not nullable types; + DataTypes nested_types(types.size()); + std::transform(types.begin(), types.end(), nested_types.begin(), + [](const auto& e) { return remove_nullable(e); }); + auto function_combinator = std::make_shared(); + auto transform_arguments = function_combinator->transform_arguments(nested_types); + if (!boost::algorithm::starts_with(name, DISTINCT_FUNCTION_PREFIX)) { + return AggregateFunctionPtr(); + } + auto nested_function_name = name.substr(DISTINCT_FUNCTION_PREFIX.size()); + auto nested_function = factory.get(nested_function_name, transform_arguments, params); + return function_combinator->transform_aggregate_function(nested_function, types, params, result_is_nullable); + }; + factory.register_distinct_function_combinator(creator, DISTINCT_FUNCTION_PREFIX); +} +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_distinct.h b/be/src/vec/aggregate_functions/aggregate_function_distinct.h new file mode 100644 index 0000000000..6502b48275 --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_distinct.h @@ -0,0 +1,224 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionDistinct.h +// and modified by Doris + +#pragma once + +#include "vec/aggregate_functions/aggregate_function.h" +#include "vec/aggregate_functions/key_holder_helpers.h" +#include "vec/common/aggregation_common.h" +#include "vec/common/assert_cast.h" +#include "vec/common/field_visitors.h" +#include "vec/common/hash_table/hash_set.h" +#include "vec/common/hash_table/hash_table.h" +#include "vec/common/sip_hash.h" +#include "vec/io/io_helper.h" + +namespace doris::vectorized { + +template +struct AggregateFunctionDistinctSingleNumericData { + /// When creating, the hash table must be small. + using Set = HashSetWithStackMemory, 4>; + using Self = AggregateFunctionDistinctSingleNumericData; + Set set; + + void add(const IColumn** columns, size_t /* columns_num */, size_t row_num, Arena*) { + const auto& vec = assert_cast&>(*columns[0]).get_data(); + set.insert(vec[row_num]); + } + + void merge(const Self& rhs, Arena*) { set.merge(rhs.set); } + + void serialize(BufferWritable& buf) const { set.write(buf); } + + void deserialize(BufferReadable& buf, Arena*) { set.read(buf); } + + MutableColumns get_arguments(const DataTypes& argument_types) const { + MutableColumns argument_columns; + argument_columns.emplace_back(argument_types[0]->create_column()); + for (const auto& elem : set) argument_columns[0]->insert(elem.get_value()); + + return argument_columns; + } +}; + +struct AggregateFunctionDistinctGenericData { + /// When creating, the hash table must be small. + using Set = HashSetWithSavedHashWithStackMemory; + using Self = AggregateFunctionDistinctGenericData; + Set set; + + void merge(const Self& rhs, Arena* arena) { + Set::LookupResult it; + bool inserted; + for (const auto& elem : rhs.set) + set.emplace(ArenaKeyHolder{elem.get_value(), *arena}, it, inserted); + } + + void serialize(BufferWritable& buf) const { + write_var_uint(set.size(), buf); + for (const auto& elem : set) + write_string_binary(elem.get_value(), buf); + } + + void deserialize(BufferReadable& buf, Arena* arena) { + size_t size; + read_var_uint(size, buf); + + StringRef ref; + for (size_t i = 0; i < size; ++i) { + read_string_binary(ref, buf); + set.insert(ref); + } + } +}; + +template +struct AggregateFunctionDistinctSingleGenericData : public AggregateFunctionDistinctGenericData { + void add(const IColumn** columns, size_t /* columns_num */, size_t row_num, Arena* arena) { + Set::LookupResult it; + bool inserted; + auto key_holder = get_key_holder(*columns[0], row_num, *arena); + set.emplace(key_holder, it, inserted); + } + + MutableColumns get_arguments(const DataTypes& argument_types) const { + MutableColumns argument_columns; + argument_columns.emplace_back(argument_types[0]->create_column()); + for (const auto& elem : set) + deserialize_and_insert(elem.get_value(), *argument_columns[0]); + + return argument_columns; + } +}; + +struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDistinctGenericData { + void add(const IColumn** columns, size_t columns_num, size_t row_num, Arena* arena) { + const char* begin = nullptr; + StringRef value(begin, 0); + for (size_t i = 0; i < columns_num; ++i) { + auto cur_ref = columns[i]->serialize_value_into_arena(row_num, *arena, begin); + value.data = cur_ref.data - value.size; + value.size += cur_ref.size; + } + + Set::LookupResult it; + bool inserted; + auto key_holder = SerializedKeyHolder{value, *arena}; + set.emplace(key_holder, it, inserted); + } + + MutableColumns get_arguments(const DataTypes& argument_types) const { + MutableColumns argument_columns(argument_types.size()); + for (size_t i = 0; i < argument_types.size(); ++i) + argument_columns[i] = argument_types[i]->create_column(); + + for (const auto& elem : set) { + const char* begin = elem.get_value().data; + for (auto& column : argument_columns) + begin = column->deserialize_and_insert_from_arena(begin); + } + + return argument_columns; + } +}; + +/** Adaptor for aggregate functions. + * Adding -Distinct suffix to aggregate function +**/ +template +class AggregateFunctionDistinct + : public IAggregateFunctionDataHelper> { +private: + static constexpr auto prefix_size = sizeof(Data); + AggregateFunctionPtr nested_func; + size_t arguments_num; + + AggregateDataPtr get_nested_place(AggregateDataPtr __restrict place) const noexcept { + return place + prefix_size; + } + + ConstAggregateDataPtr get_nested_place(ConstAggregateDataPtr __restrict place) const noexcept { + return place + prefix_size; + } + +public: + AggregateFunctionDistinct(AggregateFunctionPtr nested_func_, const DataTypes& arguments) + : IAggregateFunctionDataHelper( + arguments, nested_func_->get_parameters()), + nested_func(nested_func_), + arguments_num(arguments.size()) {} + + void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num, + Arena* arena) const override { + this->data(place).add(columns, arguments_num, row_num, arena); + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, + Arena* arena) const override { + this->data(place).merge(this->data(rhs), arena); + } + + void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena* arena) const override { + this->data(place).deserialize(buf, arena); + } + + // void insert_result_into(AggregateDataPtr place, IColumn & to, Arena * arena) const override + void insert_result_into(ConstAggregateDataPtr targetplace, IColumn& to) const override { + auto place = const_cast(targetplace); + auto arguments = this->data(place).get_arguments(this->argument_types); + ColumnRawPtrs arguments_raw(arguments.size()); + for (size_t i = 0; i < arguments.size(); ++i) arguments_raw[i] = arguments[i].get(); + + assert(!arguments.empty()); + // nested_func->add_batch_single_place(arguments[0]->size(), get_nested_place(place), arguments_raw.data(), arena); + // nested_func->insert_result_into(get_nested_place(place), to, arena); + + nested_func->add_batch_single_place(arguments[0]->size(), get_nested_place(place), + arguments_raw.data(), nullptr); + nested_func->insert_result_into(get_nested_place(place), to); + } + + size_t size_of_data() const override { return prefix_size + nested_func->size_of_data(); } + + void create(AggregateDataPtr __restrict place) const override { + new (place) Data; + nested_func->create(get_nested_place(place)); + } + + void destroy(AggregateDataPtr __restrict place) const noexcept override { + this->data(place).~Data(); + nested_func->destroy(get_nested_place(place)); + } + + String get_name() const override { return nested_func->get_name() + "Distinct"; } + + DataTypePtr get_return_type() const override { return nested_func->get_return_type(); } + + bool allocates_memory_in_arena() const override { return true; } + + const char* get_header_file_path() const override { return __FILE__; } +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.cpp b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.cpp new file mode 100644 index 0000000000..3b2aba0552 --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.cpp @@ -0,0 +1,51 @@ +// 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/aggregate_functions/aggregate_function_hll_union_agg.h" + +#include "vec/aggregate_functions/aggregate_function_simple_factory.h" +#include "vec/aggregate_functions/factory_helpers.h" + +namespace doris::vectorized { + +AggregateFunctionPtr create_aggregate_function_HLL_union_agg(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + assert_no_parameters(name, parameters); + assert_arity_at_most<1>(name, argument_types); + + return std::make_shared(argument_types); +} + +AggregateFunctionPtr create_aggregate_function_HLL_union(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable) { + assert_no_parameters(name, parameters); + assert_arity_at_most<1>(name, argument_types); + + return std::make_shared(argument_types); +} + +void register_aggregate_function_HLL_union_agg(AggregateFunctionSimpleFactory& factory) { + factory.register_function("hll_union_agg", create_aggregate_function_HLL_union_agg); + factory.register_function("hll_union", create_aggregate_function_HLL_union); + factory.register_function("hll_raw_agg", create_aggregate_function_HLL_union); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h new file mode 100644 index 0000000000..f71a1f5596 --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h @@ -0,0 +1,133 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include + +#include "exprs/hll_function.h" +#include "olap/hll.h" +#include "util/slice.h" +#include "vec/aggregate_functions/aggregate_function.h" +#include "vec/columns/column_string.h" +#include "vec/columns/column_vector.h" +#include "vec/data_types/data_type_number.h" +#include "vec/data_types/data_type_string.h" +#include "vec/io/io_helper.h" + +namespace doris::vectorized { + +struct AggregateFunctionHLLData { + HyperLogLog dst_hll {}; + + void add(const StringRef& src) { dst_hll.merge(HyperLogLog(Slice(src.data, src.size))); } + + void merge(const AggregateFunctionHLLData& rhs) { dst_hll.merge(rhs.dst_hll); } + + void write(BufferWritable& buf) const { + std::string result(dst_hll.max_serialized_size(), '0'); + int size = dst_hll.serialize((uint8_t*)result.c_str()); + result.resize(size); + write_binary(result, buf); + } + + void read(BufferReadable& buf) { + StringRef ref; + read_binary(ref, buf); + dst_hll.deserialize(Slice(ref.data, ref.size)); + } + + Int64 get_cardinality() const { return dst_hll.estimate_cardinality(); } + + std::string get() const { + std::string result(dst_hll.max_serialized_size(), '0'); + int size = dst_hll.serialize((uint8_t*)result.c_str()); + result.resize(size); + + return result; + } +}; + +class AggregateFunctionHLLUnionAgg + : public IAggregateFunctionDataHelper { +public: + virtual String get_name() const override { return "hll_union_agg"; } + + AggregateFunctionHLLUnionAgg(const DataTypes& argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}) {} + + AggregateFunctionHLLUnionAgg(const IDataType& data_type, const DataTypes& argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}) {} + + virtual DataTypePtr get_return_type() const override { + return std::make_shared(); + } + + void add(AggregateDataPtr __restrict place, const IColumn** columns, size_t row_num, + Arena*) const override { + const auto& column = static_cast(*columns[0]); + this->data(place).add(column.get_data_at(row_num)); + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena*) const override { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { + this->data(place).write(buf); + } + + void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, Arena*) const override { + this->data(place).read(buf); + } + + virtual void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + auto& column = static_cast&>(to); + column.get_data().push_back(this->data(place).get_cardinality()); + } + + const char* get_header_file_path() const override { return __FILE__; } +}; + +class AggregateFunctionHLLUnion final : public AggregateFunctionHLLUnionAgg { +public: + String get_name() const override { return "hll_union"; } + + AggregateFunctionHLLUnion(const DataTypes& argument_types_) + : AggregateFunctionHLLUnionAgg {argument_types_} {} + + AggregateFunctionHLLUnion(const IDataType& data_type, const DataTypes& argument_types_) + : AggregateFunctionHLLUnionAgg(data_type, argument_types_) {} + + DataTypePtr get_return_type() const override { return std::make_shared(); } + + void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { + auto& column = static_cast(to); + auto result = this->data(place).get(); + column.insert_data(result.c_str(), result.length()); + } +}; + +AggregateFunctionPtr create_aggregate_function_HLL_union(const std::string& name, + const DataTypes& argument_types, + const Array& parameters, + const bool result_is_nullable); + +} // namespace doris::vectorized diff --git a/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp b/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp new file mode 100644 index 0000000000..813674a312 --- /dev/null +++ b/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp @@ -0,0 +1,85 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// This file is copied from +// https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionMinMaxAny.cpp +// and modified by Doris + +#include "vec/aggregate_functions/aggregate_function_min_max.h" + +#include "vec/aggregate_functions/aggregate_function_simple_factory.h" +#include "vec/aggregate_functions/factory_helpers.h" +#include "vec/aggregate_functions/helpers.h" + +namespace doris::vectorized { + +/// min, max +template