diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index fe4337ad8c7f45..152651a8119948 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -178,12 +178,12 @@ set_target_properties(brpc PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib64/ add_library(rocksdb STATIC IMPORTED) set_target_properties(rocksdb PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/librocksdb.a) -add_library(librdkafka STATIC IMPORTED) -set_target_properties(librdkafka PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/librdkafka.a) - add_library(librdkafka_cpp STATIC IMPORTED) set_target_properties(librdkafka_cpp PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/librdkafka++.a) +add_library(librdkafka STATIC IMPORTED) +set_target_properties(librdkafka PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/librdkafka.a) + find_program(THRIFT_COMPILER thrift ${CMAKE_SOURCE_DIR}/bin) # llvm-config @@ -440,9 +440,10 @@ set(DORIS_LINK_LIBS # Set thirdparty libraries set(DORIS_DEPENDENCIES + ${WL_START_GROUP} rocksdb - librdkafka librdkafka_cpp + librdkafka lzo snappy ${Boost_LIBRARIES} @@ -456,7 +457,6 @@ set(DORIS_DEPENDENCIES libevent mysql curl - ${WL_START_GROUP} ${LIBZ} ${LIBBZ2} gflags @@ -464,8 +464,8 @@ set(DORIS_DEPENDENCIES protobuf openssl crypto - ${WL_START_GROUP} leveldb + ${WL_END_GROUP} ) # Add all external dependencies. They should come after the palo libs. diff --git a/be/src/common/config.h b/be/src/common/config.h index f723ec36da9d83..633defec052533 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -396,6 +396,9 @@ namespace config { // If set to true, metric calculator will run CONF_Bool(enable_metric_calculator, "false"); + + // max consumer num in one data consumer group, for routine load + CONF_Int32(max_consumer_num_per_group, "3"); } // namespace config } // namespace doris diff --git a/be/src/common/status.cpp b/be/src/common/status.cpp index c66a108cfed537..64a9b865c098ee 100644 --- a/be/src/common/status.cpp +++ b/be/src/common/status.cpp @@ -34,6 +34,9 @@ const Status Status::MEM_LIMIT_EXCEEDED( const Status Status::THRIFT_RPC_ERROR( TStatusCode::THRIFT_RPC_ERROR, "Thrift RPC failed", true); +const Status Status::TIMEOUT( + TStatusCode::TIMEOUT, "timeout", true); + Status::ErrorDetail::ErrorDetail(const TStatus& status) : error_code(status.status_code), error_msgs(status.error_msgs) { diff --git a/be/src/common/status.h b/be/src/common/status.h index 040baf8c5ef207..958e30f5e286b3 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -54,6 +54,7 @@ class Status { static const Status CANCELLED; static const Status MEM_LIMIT_EXCEEDED; static const Status THRIFT_RPC_ERROR; + static const Status TIMEOUT; // copy c'tor makes copy of error detail so Status can be returned by value Status(const Status& status) : _error_detail( diff --git a/be/src/common/utils.h b/be/src/common/utils.h new file mode 100644 index 00000000000000..8376fe38a68e67 --- /dev/null +++ b/be/src/common/utils.h @@ -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. + +#pragma once + +#include + +namespace doris { + +struct AuthInfo { + std::string user; + std::string passwd; + std::string cluster; + std::string user_ip; + // -1 as unset + int64_t auth_code = -1; +}; + +template +void set_request_auth(T* req, const AuthInfo& auth) { + if (auth.auth_code != -1) { + // if auth_code is set, no need to set other info + req->__set_auth_code(auth.auth_code); + // user name and passwd is unused, but they are required field. + // so they have to be set. + req->user = ""; + req->passwd = ""; + } else { + req->user = auth.user; + req->passwd = auth.passwd; + if (!auth.cluster.empty()) { + req->__set_cluster(auth.cluster); + } + req->__set_user_ip(auth.user_ip); + } +} + +} diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt index ac101769ef104c..554ecc13afdbac 100644 --- a/be/src/exec/CMakeLists.txt +++ b/be/src/exec/CMakeLists.txt @@ -63,6 +63,13 @@ set(EXEC_FILES csv_scan_node.cpp csv_scanner.cpp es_scan_node.cpp + es_http_scan_node.cpp + es_http_scanner.cpp + es/es_predicate.cpp + es/es_scan_reader.cpp + es/es_scroll_query.cpp + es/es_scroll_parser.cpp + es/es_query_builder.cpp spill_sort_node.cc union_node.cpp union_node_ir.cpp @@ -75,7 +82,7 @@ set(EXEC_FILES schema_scanner/schema_columns_scanner.cpp schema_scanner/schema_charsets_scanner.cpp schema_scanner/schema_collations_scanner.cpp - schema_scanner/frontend_helper.cpp + schema_scanner/schema_helper.cpp partitioned_hash_table.cc partitioned_hash_table_ir.cc partitioned_aggregation_node.cc @@ -101,6 +108,8 @@ add_library(Exec STATIC ) # TODO: why is this test disabled? +#ADD_BE_TEST(es/es_query_builder_test) +#ADD_BE_TEST(es/es_scan_reader_test) #ADD_BE_TEST(new_olap_scan_node_test) #ADD_BE_TEST(pre_aggregation_node_test) #ADD_BE_TEST(hash_table_test) diff --git a/be/src/exec/broker_scan_node.cpp b/be/src/exec/broker_scan_node.cpp index cd0b3b746ec210..28b27aa4122530 100644 --- a/be/src/exec/broker_scan_node.cpp +++ b/be/src/exec/broker_scan_node.cpp @@ -342,9 +342,9 @@ Status BrokerScanNode::scanner_scan( char* new_tuple = reinterpret_cast(tuple); new_tuple += _tuple_desc->byte_size(); tuple = reinterpret_cast(new_tuple); - counter->num_rows_returned++; + // counter->num_rows_returned++; } else { - counter->num_rows_filtered++; + counter->num_rows_unselected++; } } @@ -407,8 +407,10 @@ void BrokerScanNode::scanner_worker(int start_idx, int length) { } // Update stats - _runtime_state->update_num_rows_load_success(counter.num_rows_returned); + // _runtime_state->update_num_rows_load_success(counter.num_rows_returned); + _runtime_state->update_num_rows_load_total(counter.num_rows_total); _runtime_state->update_num_rows_load_filtered(counter.num_rows_filtered); + _runtime_state->update_num_rows_load_unselected(counter.num_rows_unselected); // scanner is going to finish { diff --git a/be/src/exec/broker_scanner.cpp b/be/src/exec/broker_scanner.cpp index c60e35665ac3ea..e56c7033beb146 100644 --- a/be/src/exec/broker_scanner.cpp +++ b/be/src/exec/broker_scanner.cpp @@ -24,8 +24,8 @@ #include "runtime/exec_env.h" #include "runtime/mem_tracker.h" #include "runtime/raw_value.h" -#include "runtime/load_stream_mgr.h" -#include "runtime/stream_load_pipe.h" +#include "runtime/stream_load/load_stream_mgr.h" +#include "runtime/stream_load/stream_load_pipe.h" #include "runtime/tuple.h" #include "exprs/expr.h" #include "exec/text_converter.h" @@ -178,6 +178,7 @@ Status BrokerScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) { { COUNTER_UPDATE(_rows_read_counter, 1); SCOPED_TIMER(_materialize_timer); + _counter->num_rows_total++; if (convert_one_row(Slice(ptr, size), tuple, tuple_pool)) { break; } @@ -237,6 +238,7 @@ Status BrokerScanner::open_file_reader() { case TFileType::FILE_STREAM: { _stream_load_pipe = _state->exec_env()->load_stream_mgr()->get(range.load_id); if (_stream_load_pipe == nullptr) { + VLOG(3) << "unknown stream load id: " << UniqueId(range.load_id); return Status("unknown stream load id"); } _cur_file_reader = _stream_load_pipe.get(); @@ -582,6 +584,7 @@ bool BrokerScanner::fill_dest_tuple(const Slice& line, Tuple* dest_tuple, MemPoo if (!slot_desc->is_materialized()) { continue; } + ExprContext* ctx = _dest_expr_ctx[ctx_idx++]; void* value = ctx->get_value(_src_tuple_row); if (value == nullptr) { diff --git a/be/src/exec/broker_scanner.h b/be/src/exec/broker_scanner.h index 26a84c7ca3a54f..c18b7fc200f567 100644 --- a/be/src/exec/broker_scanner.h +++ b/be/src/exec/broker_scanner.h @@ -49,11 +49,17 @@ class RuntimeProfile; class StreamLoadPipe; struct BrokerScanCounter { - BrokerScanCounter() : num_rows_returned(0), num_rows_filtered(0) { + BrokerScanCounter() : + num_rows_total(0), + // num_rows_returned(0), + num_rows_filtered(0), + num_rows_unselected(0) { } - int64_t num_rows_returned; - int64_t num_rows_filtered; + int64_t num_rows_total; // total read rows (read from source) + // int64_t num_rows_returned; // qualified rows (match the dest schema) + int64_t num_rows_filtered; // unqualified rows (unmatch the dest schema, or no partition) + int64_t num_rows_unselected; // rows filterd by predicates }; // Broker scanner convert the data read from broker to doris's tuple. diff --git a/be/src/exec/es/es_predicate.cpp b/be/src/exec/es/es_predicate.cpp new file mode 100644 index 00000000000000..a7f0bc1f453929 --- /dev/null +++ b/be/src/exec/es/es_predicate.cpp @@ -0,0 +1,414 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es/es_predicate.h" + +#include +#include +#include +#include +#include + +#include "common/status.h" +#include "common/logging.h" +#include "exec/es/es_query_builder.h" +#include "exprs/expr.h" +#include "exprs/expr_context.h" +#include "exprs/in_predicate.h" + +#include "gen_cpp/PlanNodes_types.h" +#include "olap/olap_common.h" +#include "olap/utils.h" +#include "runtime/client_cache.h" +#include "runtime/runtime_state.h" +#include "runtime/row_batch.h" +#include "runtime/datetime_value.h" +#include "runtime/large_int_value.h" +#include "runtime/string_value.h" +#include "runtime/tuple_row.h" + +#include "service/backend_options.h" +#include "util/debug_util.h" +#include "util/runtime_profile.h" + +namespace doris { + +using namespace std; + +std::string ExtLiteral::value_to_string() { + std::stringstream ss; + switch (_type) { + case TYPE_TINYINT: + ss << std::to_string(get_byte()); + break; + case TYPE_SMALLINT: + ss << std::to_string(get_short()); + break; + case TYPE_INT: + ss << std::to_string(get_int()); + break; + case TYPE_BIGINT: + ss << std::to_string(get_long()); + break; + case TYPE_FLOAT: + ss << std::to_string(get_float()); + break; + case TYPE_DOUBLE: + ss << std::to_string(get_double()); + break; + case TYPE_CHAR: + case TYPE_VARCHAR: + ss << get_string(); + break; + case TYPE_DATE: + case TYPE_DATETIME: + ss << get_date_string(); + break; + case TYPE_BOOLEAN: + ss << std::to_string(get_bool()); + break; + case TYPE_DECIMAL: + ss << get_decimal_string(); + break; + case TYPE_DECIMALV2: + ss << get_decimalv2_string(); + break; + case TYPE_LARGEINT: + ss << get_largeint_string(); + break; + default: + DCHECK(false); + break; + } + return ss.str(); +} + +ExtLiteral::~ExtLiteral(){ +} + +int8_t ExtLiteral::get_byte() { + DCHECK(_type == TYPE_TINYINT); + return *(reinterpret_cast(_value)); +} + +int16_t ExtLiteral::get_short() { + DCHECK(_type == TYPE_SMALLINT); + return *(reinterpret_cast(_value)); +} + +int32_t ExtLiteral::get_int() { + DCHECK(_type == TYPE_INT); + return *(reinterpret_cast(_value)); +} + +int64_t ExtLiteral::get_long() { + DCHECK(_type == TYPE_BIGINT); + return *(reinterpret_cast(_value)); +} + +float ExtLiteral::get_float() { + DCHECK(_type == TYPE_FLOAT); + return *(reinterpret_cast(_value)); +} + +double ExtLiteral::get_double() { + DCHECK(_type == TYPE_DOUBLE); + return *(reinterpret_cast(_value)); +} + +std::string ExtLiteral::get_string() { + DCHECK(_type == TYPE_VARCHAR || _type == TYPE_CHAR); + return (reinterpret_cast(_value))->to_string(); +} + +std::string ExtLiteral::get_date_string() { + DCHECK(_type == TYPE_DATE || _type == TYPE_DATETIME); + DateTimeValue date_value = *reinterpret_cast(_value); + if (_type == TYPE_DATE) { + date_value.cast_to_date(); + } + + char str[MAX_DTVALUE_STR_LEN]; + date_value.to_string(str); + return std::string(str, strlen(str)); +} + +bool ExtLiteral::get_bool() { + DCHECK(_type == TYPE_BOOLEAN); + return *(reinterpret_cast(_value)); +} + +std::string ExtLiteral::get_decimal_string() { + DCHECK(_type == TYPE_DECIMAL); + return reinterpret_cast(_value)->to_string(); +} + +std::string ExtLiteral::get_decimalv2_string() { + DCHECK(_type == TYPE_DECIMALV2); + return reinterpret_cast(_value)->to_string(); +} + +std::string ExtLiteral::get_largeint_string() { + DCHECK(_type == TYPE_LARGEINT); + return LargeIntValue::to_string(*reinterpret_cast<__int128*>(_value)); +} + +EsPredicate::EsPredicate(ExprContext* context, + const TupleDescriptor* tuple_desc) : + _context(context), + _disjuncts_num(0), + _tuple_desc(tuple_desc), + _es_query_status(Status::OK) { +} + +EsPredicate::~EsPredicate() { + for(int i=0; i < _disjuncts.size(); i++) { + delete _disjuncts[i]; + } + _disjuncts.clear(); +} + +Status EsPredicate::build_disjuncts_list() { + return build_disjuncts_list(_context->root()); +} + +// make sure to build by build_disjuncts_list +const vector& EsPredicate::get_predicate_list(){ + return _disjuncts; +} + +static bool ignore_cast(const SlotDescriptor* slot, const Expr* expr) { + if (slot->type().is_date_type() && expr->type().is_date_type()) { + return true; + } + if (slot->type().is_string_type() && expr->type().is_string_type()) { + return true; + } + return false; +} + +static bool is_literal_node(const Expr* expr) { + switch (expr->node_type()) { + case TExprNodeType::BOOL_LITERAL: + case TExprNodeType::INT_LITERAL: + case TExprNodeType::LARGE_INT_LITERAL: + case TExprNodeType::FLOAT_LITERAL: + case TExprNodeType::DECIMAL_LITERAL: + case TExprNodeType::STRING_LITERAL: + case TExprNodeType::DATE_LITERAL: + return true; + default: + return false; + } +} + +Status EsPredicate::build_disjuncts_list(const Expr* conjunct) { + if (TExprNodeType::BINARY_PRED == conjunct->node_type()) { + if (conjunct->children().size() != 2) { + return Status("build disjuncts failed: number of childs is not 2"); + } + + SlotRef* slot_ref = nullptr; + TExprOpcode::type op; + Expr* expr = nullptr; + if (TExprNodeType::SLOT_REF == conjunct->get_child(0)->node_type()) { + expr = conjunct->get_child(1); + slot_ref = (SlotRef*)(conjunct->get_child(0)); + op = conjunct->op(); + } else if (TExprNodeType::SLOT_REF == conjunct->get_child(1)->node_type()) { + expr = conjunct->get_child(0); + slot_ref = (SlotRef*)(conjunct->get_child(1)); + op = conjunct->op(); + } else { + return Status("build disjuncts failed: no SLOT_REF child"); + } + + const SlotDescriptor* slot_desc = get_slot_desc(slot_ref); + if (slot_desc == nullptr) { + return Status("build disjuncts failed: slot_desc is null"); + } + + if (!is_literal_node(expr)) { + return Status("build disjuncts failed: expr is not literal type"); + } + + ExtLiteral literal(expr->type().type, _context->get_value(expr, NULL)); + ExtPredicate* predicate = new ExtBinaryPredicate( + TExprNodeType::BINARY_PRED, + slot_desc->col_name(), + slot_desc->type(), + op, + literal); + + _disjuncts.push_back(predicate); + return Status::OK; + } + + if (is_match_func(conjunct)) { + Expr* expr = conjunct->get_child(1); + ExtLiteral literal(expr->type().type, _context->get_value(expr, NULL)); + vector query_conditions; + query_conditions.emplace_back(literal); + vector cols; //TODO + ExtPredicate* predicate = new ExtFunction( + TExprNodeType::FUNCTION_CALL, + conjunct->fn().name.function_name, + cols, + query_conditions); + if (_es_query_status.ok()) { + _es_query_status + = BooleanQueryBuilder::check_es_query(*(ExtFunction *)predicate); + if (!_es_query_status.ok()) { + delete predicate; + return _es_query_status; + } + } + _disjuncts.push_back(predicate); + + return Status::OK; + } + + if (TExprNodeType::FUNCTION_CALL == conjunct->node_type()) { + std::string fname = conjunct->fn().name.function_name; + if (fname != "like") { + return Status("build disjuncts failed: function name is not like"); + } + + SlotRef* slot_ref = nullptr; + Expr* expr = nullptr; + if (TExprNodeType::SLOT_REF == conjunct->get_child(0)->node_type()) { + expr = conjunct->get_child(1); + slot_ref = (SlotRef*)(conjunct->get_child(0)); + } else if (TExprNodeType::SLOT_REF == conjunct->get_child(1)->node_type()) { + expr = conjunct->get_child(0); + slot_ref = (SlotRef*)(conjunct->get_child(1)); + } else { + return Status("build disjuncts failed: no SLOT_REF child"); + } + + const SlotDescriptor* slot_desc = get_slot_desc(slot_ref); + if (slot_desc == nullptr) { + return Status("build disjuncts failed: slot_desc is null"); + } + + PrimitiveType type = expr->type().type; + if (type != TYPE_VARCHAR && type != TYPE_CHAR) { + return Status("build disjuncts failed: like value is not a string"); + } + + ExtLiteral literal(type, _context->get_value(expr, NULL)); + ExtPredicate* predicate = new ExtLikePredicate( + TExprNodeType::LIKE_PRED, + slot_desc->col_name(), + slot_desc->type(), + literal); + + _disjuncts.push_back(predicate); + return Status::OK; + } + + if (TExprNodeType::IN_PRED == conjunct->node_type()) { + // the op code maybe FILTER_NEW_IN, it means there is function in list + // like col_a in (abs(1)) + if (TExprOpcode::FILTER_IN != conjunct->op() + && TExprOpcode::FILTER_NOT_IN != conjunct->op()) { + return Status("build disjuncts failed: " + "opcode in IN_PRED is neither FILTER_IN nor FILTER_NOT_IN"); + } + + vector in_pred_values; + const InPredicate* pred = dynamic_cast(conjunct); + const Expr* expr = Expr::expr_without_cast(pred->get_child(0)); + if (expr->node_type() != TExprNodeType::SLOT_REF) { + return Status("build disjuncts failed: node type is not slot ref"); + } + + const SlotDescriptor* slot_desc = get_slot_desc((const SlotRef *)expr); + if (slot_desc == nullptr) { + return Status("build disjuncts failed: slot_desc is null"); + } + + if (pred->get_child(0)->type().type != slot_desc->type().type) { + if (!ignore_cast(slot_desc, pred->get_child(0))) { + return Status("build disjuncts failed"); + } + } + + HybirdSetBase::IteratorBase* iter = pred->hybird_set()->begin(); + while (iter->has_next()) { + if (nullptr == iter->get_value()) { + return Status("build disjuncts failed: hybird set has a null value"); + } + + ExtLiteral literal(slot_desc->type().type, const_cast(iter->get_value())); + in_pred_values.emplace_back(literal); + iter->next(); + } + + ExtPredicate* predicate = new ExtInPredicate( + TExprNodeType::IN_PRED, + pred->is_not_in(), + slot_desc->col_name(), + slot_desc->type(), + in_pred_values); + _disjuncts.push_back(predicate); + + return Status::OK; + } + + if (TExprNodeType::COMPOUND_PRED == conjunct->node_type()) { + if (TExprOpcode::COMPOUND_OR != conjunct->op()) { + return Status("build disjuncts failed: op is not COMPOUND_OR"); + } + Status status = build_disjuncts_list(conjunct->get_child(0)); + if (!status.ok()) { + return status; + } + status = build_disjuncts_list(conjunct->get_child(1)); + if (!status.ok()) { + return status; + } + + return Status::OK; + } + + // if go to here, report error + std::stringstream ss; + ss << "build disjuncts failed: node type " << conjunct->node_type() << " is not supported"; + return Status(ss.str()); +} + +bool EsPredicate::is_match_func(const Expr* conjunct) { + if (TExprNodeType::FUNCTION_CALL == conjunct->node_type() + && conjunct->fn().name.function_name == "esquery") { + return true; + } + return false; +} + +const SlotDescriptor* EsPredicate::get_slot_desc(const SlotRef* slotRef) { + const SlotDescriptor* slot_desc = nullptr; + for (SlotDescriptor* slot : _tuple_desc->slots()) { + if (slot->id() == slotRef->slot_id()) { + slot_desc = slot; + break; + } + } + return slot_desc; +} + +} diff --git a/be/src/exec/es/es_predicate.h b/be/src/exec/es/es_predicate.h new file mode 100644 index 00000000000000..6b24754e6710a5 --- /dev/null +++ b/be/src/exec/es/es_predicate.h @@ -0,0 +1,200 @@ +// 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. + +#ifndef BE_EXEC_ES_PREDICATE_H +#define BE_EXEC_ES_PREDICATE_H + +#include +#include + +#include "exprs/slot_ref.h" +#include "gen_cpp/Exprs_types.h" +#include "gen_cpp/Opcodes_types.h" +#include "gen_cpp/PaloExternalDataSourceService_types.h" +#include "runtime/descriptors.h" +#include "runtime/tuple.h" +#include "runtime/primitive_type.h" + +namespace doris { + +class Status; +class ExprContext; +class ExtBinaryPredicate; + +class ExtLiteral { +public: + ExtLiteral(PrimitiveType type, void *value) : + _type(type), + _value(value) { + _str = value_to_string(); + } + ~ExtLiteral(); + const std::string& to_string() const { + return _str; + } + +private: + int8_t get_byte(); + int16_t get_short(); + int32_t get_int(); + int64_t get_long(); + float get_float(); + double get_double(); + std::string get_string(); + std::string get_date_string(); + bool get_bool(); + std::string get_decimal_string(); + std::string get_decimalv2_string(); + std::string get_largeint_string(); + + std::string value_to_string(); + + PrimitiveType _type; + void* _value; + std::string _str; +}; + +struct ExtColumnDesc { + ExtColumnDesc(const std::string& name, const TypeDescriptor& type) : + name(name), + type(type) { + } + + std::string name; + TypeDescriptor type; +}; + +struct ExtPredicate { + ExtPredicate(TExprNodeType::type node_type) : node_type(node_type) { + } + + TExprNodeType::type node_type; +}; + +struct ExtBinaryPredicate : public ExtPredicate { + ExtBinaryPredicate( + TExprNodeType::type node_type, + const std::string& name, + const TypeDescriptor& type, + TExprOpcode::type op, + const ExtLiteral& value) : + ExtPredicate(node_type), + col(name, type), + op(op), + value(value) { + } + + ExtColumnDesc col; + TExprOpcode::type op; + ExtLiteral value; +}; + +struct ExtInPredicate : public ExtPredicate { + ExtInPredicate( + TExprNodeType::type node_type, + bool is_not_in, + const std::string& name, + const TypeDescriptor& type, + const std::vector& values) : + ExtPredicate(node_type), + is_not_in(is_not_in), + col(name, type), + values(values) { + } + + bool is_not_in; + ExtColumnDesc col; + std::vector values; +}; + +struct ExtLikePredicate : public ExtPredicate { + ExtLikePredicate( + TExprNodeType::type node_type, + const std::string& name, + const TypeDescriptor& type, + ExtLiteral value) : + ExtPredicate(node_type), + col(name, type), + value(value) { + } + + ExtColumnDesc col; + ExtLiteral value; +}; + +struct ExtIsNullPredicate : public ExtPredicate { + ExtIsNullPredicate( + TExprNodeType::type node_type, + const std::string& name, + const TypeDescriptor& type, + bool is_not_null, + ExtLiteral value) : + ExtPredicate(node_type), + col(name, type), + is_not_null(is_not_null) { + } + + ExtColumnDesc col; + bool is_not_null; +}; + +struct ExtFunction : public ExtPredicate { + ExtFunction(TExprNodeType::type node_type, + const std::string& func_name, + std::vector cols, + std::vector values) : + ExtPredicate(node_type), + func_name(func_name), + cols(cols), + values(values) { + } + + const std::string& func_name; + std::vector cols; + const std::vector values; +}; + +class EsPredicate { +public: + EsPredicate(ExprContext* context, const TupleDescriptor* tuple_desc); + ~EsPredicate(); + const std::vector& get_predicate_list(); + Status build_disjuncts_list(); + // public for tests + EsPredicate(const std::vector& all_predicates) { + _disjuncts = all_predicates; + }; + + Status get_es_query_status() { + return _es_query_status; + } + +private: + Status build_disjuncts_list(const Expr* conjunct); + bool is_match_func(const Expr* conjunct); + const SlotDescriptor* get_slot_desc(const SlotRef* slotRef); + + ExprContext* _context; + int _disjuncts_num; + const TupleDescriptor* _tuple_desc; + std::vector _disjuncts; + Status _es_query_status; +}; + +} + +#endif diff --git a/be/src/exec/es/es_query_builder.cpp b/be/src/exec/es/es_query_builder.cpp new file mode 100644 index 00000000000000..8fc260c6715601 --- /dev/null +++ b/be/src/exec/es/es_query_builder.cpp @@ -0,0 +1,375 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es/es_query_builder.h" + +#include +#include "rapidjson/rapidjson.h" +#include "rapidjson/stringbuffer.h" +#include "rapidjson/writer.h" +#include "common/logging.h" + +namespace doris { + +ESQueryBuilder::ESQueryBuilder(const std::string& es_query_str) : _es_query_str(es_query_str) { + +} +ESQueryBuilder::ESQueryBuilder(const ExtFunction& es_query) { + auto first = es_query.values.front(); + _es_query_str = first.to_string(); +} + +// note: call this function must invoke BooleanQueryBuilder::check_es_query to check validation +void ESQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) { + rapidjson::Document scratch_document; + scratch_document.Parse(_es_query_str.c_str()); + rapidjson::Document::AllocatorType& allocator = document->GetAllocator(); + rapidjson::Value query_key; + rapidjson::Value query_value; + //{ "term": { "dv": "2" } } + rapidjson::Value::ConstMemberIterator first = scratch_document.MemberBegin(); + // deep copy, reference http://rapidjson.org/md_doc_tutorial.html#DeepCopyValue + query_key.CopyFrom(first->name, allocator); + // if we found one key, then end loop as QueryDSL only support one `query` root + query_value.CopyFrom(first->value, allocator); + // Move Semantics, reference http://rapidjson.org/md_doc_tutorial.html#MoveSemantics + query->AddMember(query_key, query_value, allocator); +} + +TermQueryBuilder::TermQueryBuilder(const std::string& field, const std::string& term) : _field(field), _term(term) { + +} + +TermQueryBuilder::TermQueryBuilder(const ExtBinaryPredicate& binary_predicate) { + _field = binary_predicate.col.name; + _term = binary_predicate.value.to_string(); +} + +void TermQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) { + rapidjson::Document::AllocatorType& allocator = document->GetAllocator(); + rapidjson::Value term_node(rapidjson::kObjectType); + term_node.SetObject(); + rapidjson::Value field_value(_field.c_str(), allocator); + rapidjson::Value term_value(_term.c_str(), allocator); + term_node.AddMember(field_value, term_value, allocator); + query->AddMember("term", term_node, allocator); +} + +RangeQueryBuilder::RangeQueryBuilder(const ExtBinaryPredicate& range_predicate) { + _field = range_predicate.col.name; + _value = range_predicate.value.to_string(); + _op = range_predicate.op; +} + +void RangeQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) { + rapidjson::Document::AllocatorType& allocator = document->GetAllocator(); + rapidjson::Value field_value(_field.c_str(), allocator); + rapidjson::Value value(_value.c_str(), allocator); + rapidjson::Value op_node(rapidjson::kObjectType); + op_node.SetObject(); + switch (_op) { + case TExprOpcode::LT: + op_node.AddMember("lt", value, allocator); + break; + case TExprOpcode::LE: + op_node.AddMember("le", value, allocator); + break; + case TExprOpcode::GT: + op_node.AddMember("gt", value, allocator); + break; + case TExprOpcode::GE: + op_node.AddMember("ge", value, allocator); + break; + default: + break; + } + rapidjson::Value field_node(rapidjson::kObjectType); + field_node.SetObject(); + field_node.AddMember(field_value, op_node, allocator); + query->AddMember("range", field_node, allocator); +} + +void WildCardQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) { + rapidjson::Document::AllocatorType& allocator = document->GetAllocator(); + rapidjson::Value term_node(rapidjson::kObjectType); + term_node.SetObject(); + rapidjson::Value field_value(_field.c_str(), allocator); + rapidjson::Value term_value(_like_value.c_str(), allocator); + term_node.AddMember(field_value, term_value, allocator); + query->AddMember("wildcard", term_node, allocator); +} +WildCardQueryBuilder::WildCardQueryBuilder(const ExtLikePredicate& like_predicate) { + _like_value = like_predicate.value.to_string(); + std::replace(_like_value.begin(), _like_value.end(), '_', '?'); + std::replace(_like_value.begin(), _like_value.end(), '%', '*'); + _field = like_predicate.col.name; +} + +void TermsInSetQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) { + rapidjson::Document::AllocatorType& allocator = document->GetAllocator(); + rapidjson::Value terms_node(rapidjson::kObjectType); + rapidjson::Value values_node(rapidjson::kArrayType); + for (auto& value : _values) { + rapidjson::Value value_value(value.c_str(), allocator); + values_node.PushBack(value_value, allocator); + } + rapidjson::Value field_value(_field.c_str(), allocator); + terms_node.AddMember(field_value, values_node, allocator); + query->AddMember("terms", terms_node, allocator); +} + +TermsInSetQueryBuilder::TermsInSetQueryBuilder(const ExtInPredicate& in_predicate) { + _field = in_predicate.col.name; + for (auto& value : in_predicate.values) { + _values.push_back(value.to_string()); + } +} + +void MatchAllQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) { + rapidjson::Document::AllocatorType& allocator = document->GetAllocator(); + rapidjson::Value match_all_node(rapidjson::kObjectType); + match_all_node.SetObject(); + query->AddMember("match_all", match_all_node, allocator); +} + +BooleanQueryBuilder::BooleanQueryBuilder() { + +} +BooleanQueryBuilder::~BooleanQueryBuilder() { + for (auto clause : _must_clauses) { + delete clause; + clause = nullptr; + } + for (auto clause : _must_not_clauses) { + delete clause; + clause = nullptr; + } + for (auto clause : _filter_clauses) { + delete clause; + clause = nullptr; + } + for (auto clause : _should_clauses) { + delete clause; + clause = nullptr; + } +} + +BooleanQueryBuilder::BooleanQueryBuilder(const std::vector& predicates) { + for (auto predicate : predicates) { + switch (predicate->node_type) { + case TExprNodeType::BINARY_PRED: { + ExtBinaryPredicate* binary_predicate = (ExtBinaryPredicate*)predicate; + switch (binary_predicate->op) { + case TExprOpcode::EQ: { + TermQueryBuilder* term_query = new TermQueryBuilder(*binary_predicate); + _should_clauses.push_back(term_query); + break; + } + case TExprOpcode::NE:{ // process NE + TermQueryBuilder* term_query = new TermQueryBuilder(*binary_predicate); + BooleanQueryBuilder* bool_query = new BooleanQueryBuilder(); + bool_query->must_not(term_query); + _should_clauses.push_back(bool_query); + break; + } + case TExprOpcode::LT: + case TExprOpcode::LE: + case TExprOpcode::GT: + case TExprOpcode::GE: { + RangeQueryBuilder* range_query = new RangeQueryBuilder(*binary_predicate); + _should_clauses.push_back(range_query); + break; + } + default: + break; + } + break; + } + case TExprNodeType::IN_PRED: { + ExtInPredicate* in_predicate = (ExtInPredicate *)predicate; + bool is_not_in = in_predicate->is_not_in; + if (is_not_in) { // process not in predicate + TermsInSetQueryBuilder* terms_predicate = new TermsInSetQueryBuilder(*in_predicate); + BooleanQueryBuilder* bool_query = new BooleanQueryBuilder(); + bool_query->must_not(terms_predicate); + _should_clauses.push_back(bool_query); + } else { // process in predicate + TermsInSetQueryBuilder* terms_query= new TermsInSetQueryBuilder(*in_predicate); + _should_clauses.push_back(terms_query); + } + break; + } + case TExprNodeType::LIKE_PRED: { + ExtLikePredicate* like_predicate = (ExtLikePredicate *)predicate; + WildCardQueryBuilder* wild_card_query = new WildCardQueryBuilder(*like_predicate); + _should_clauses.push_back(wild_card_query); + break; + } + case TExprNodeType::FUNCTION_CALL: { + ExtFunction* function_predicate = (ExtFunction *)predicate; + if ("esquery" == function_predicate->func_name ) { + ESQueryBuilder* es_query = new ESQueryBuilder(*function_predicate); + _should_clauses.push_back(es_query); + }; + break; + } + default: + break; + } + } +} + +void BooleanQueryBuilder::to_json(rapidjson::Document* document, rapidjson::Value* query) { + rapidjson::Document::AllocatorType &allocator = document->GetAllocator(); + rapidjson::Value root_node_object(rapidjson::kObjectType); + if (_filter_clauses.size() > 0) { + rapidjson::Value filter_node(rapidjson::kArrayType); + for (auto must_clause : _filter_clauses) { + rapidjson::Value must_clause_query(rapidjson::kObjectType); + must_clause_query.SetObject(); + must_clause->to_json(document, &must_clause_query); + filter_node.PushBack(must_clause_query, allocator); + } + root_node_object.AddMember("filter", filter_node, allocator); + } + + if (_should_clauses.size() > 0) { + rapidjson::Value should_node(rapidjson::kArrayType); + for (auto should_clause : _should_clauses) { + rapidjson::Value should_clause_query(rapidjson::kObjectType); + should_clause_query.SetObject(); + should_clause->to_json(document, &should_clause_query); + should_node.PushBack(should_clause_query, allocator); + } + root_node_object.AddMember("should", should_node, allocator); + } + + if (_must_not_clauses.size() > 0) { + rapidjson::Value must_not_node(rapidjson::kArrayType); + for (auto must_not_clause : _must_not_clauses) { + rapidjson::Value must_not_clause_query(rapidjson::kObjectType); + must_not_clause_query.SetObject(); + must_not_clause->to_json(document, &must_not_clause_query); + must_not_node.PushBack(must_not_clause_query, allocator); + } + root_node_object.AddMember("must_not", must_not_node, allocator); + } + query->AddMember("bool", root_node_object, allocator); +} + +void BooleanQueryBuilder::should(QueryBuilder* filter) { + _should_clauses.push_back(filter); +} +void BooleanQueryBuilder::filter(QueryBuilder* filter) { + _filter_clauses.push_back(filter); +} +void BooleanQueryBuilder::must(QueryBuilder* filter) { + _filter_clauses.push_back(filter); +} +void BooleanQueryBuilder::must_not(QueryBuilder* filter) { + _must_not_clauses.push_back(filter); +} + +Status BooleanQueryBuilder::check_es_query(const ExtFunction& extFunction) { + const std::string& esquery_str = extFunction.values.front().to_string(); + rapidjson::Document scratch_document; + scratch_document.Parse(esquery_str.c_str()); + rapidjson::Document::AllocatorType& allocator = scratch_document.GetAllocator(); + rapidjson::Value query_key; + // { "term": { "dv": "2" } } + if (!scratch_document.HasParseError()) { + if (!scratch_document.IsObject()) { + return Status(TStatusCode::ES_REQUEST_ERROR, "esquery must be a object"); + } + rapidjson::SizeType object_count = scratch_document.MemberCount(); + if (object_count != 1) { + return Status(TStatusCode::ES_REQUEST_ERROR, "esquery must only one root"); + } + // deep copy, reference http://rapidjson.org/md_doc_tutorial.html#DeepCopyValue + rapidjson::Value::ConstMemberIterator first = scratch_document.MemberBegin(); + query_key.CopyFrom(first->name, allocator); + if (!query_key.IsString()) { + // if we found one key, then end loop as QueryDSL only support one `query` root + return Status(TStatusCode::ES_REQUEST_ERROR, "esquery root key must be string"); + } + } else { + return Status(TStatusCode::ES_REQUEST_ERROR, "malformed esquery json"); + } + return Status::OK; +} + +void BooleanQueryBuilder::validate(const std::vector& espredicates, std::vector* result) { + int conjunct_size = espredicates.size(); + result->reserve(conjunct_size); + for (auto espredicate : espredicates) { + bool flag = true; + for (auto predicate : espredicate->get_predicate_list()) { + switch (predicate->node_type) { + case TExprNodeType::BINARY_PRED: { + ExtBinaryPredicate* binary_predicate = (ExtBinaryPredicate*)predicate; + TExprOpcode::type op = binary_predicate->op; + if (op != TExprOpcode::EQ && op != TExprOpcode::NE + && op != TExprOpcode::LT && op != TExprOpcode::LE + && op != TExprOpcode::GT && op != TExprOpcode::GE) { + flag = false; + } + break; + } + case TExprNodeType::LIKE_PRED: + case TExprNodeType::IN_PRED: { + break; + } + case TExprNodeType::FUNCTION_CALL: { + ExtFunction* function_predicate = (ExtFunction *)predicate; + if ("esquery" == function_predicate->func_name ) { + Status st = check_es_query(*function_predicate); + if (!st.ok()) { + flag = false; + } + } else { + flag = false; + } + break; + } + default: { + flag = false; + break; + } + } + if (!flag) { + break; + } + } + result->push_back(flag); + } +} + +void BooleanQueryBuilder::to_query(const std::vector& predicates, rapidjson::Document* root, rapidjson::Value* query) { + if (predicates.size() == 0) { + MatchAllQueryBuilder match_all_query; + match_all_query.to_json(root, query); + return; + } + root->SetObject(); + BooleanQueryBuilder bool_query; + for (auto es_predicate : predicates) { + vector or_predicates = es_predicate->get_predicate_list(); + BooleanQueryBuilder* inner_bool_query = new BooleanQueryBuilder(or_predicates); + bool_query.must(inner_bool_query); + } + bool_query.to_json(root, query);} +} diff --git a/be/src/exec/es/es_query_builder.h b/be/src/exec/es/es_query_builder.h new file mode 100644 index 00000000000000..e7c5e563356069 --- /dev/null +++ b/be/src/exec/es/es_query_builder.h @@ -0,0 +1,130 @@ +// 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 "rapidjson/document.h" +#include "exec/es/es_predicate.h" +#include "common/status.h" + +namespace doris { + +class QueryBuilder { + +public: + virtual void to_json(rapidjson::Document* document, rapidjson::Value* query) = 0; + virtual ~QueryBuilder() { + }; +}; + +// process esquery(fieldA, json dsl) function +class ESQueryBuilder : public QueryBuilder { +public: + ESQueryBuilder(const std::string& es_query_str); + ESQueryBuilder(const ExtFunction& es_query); + void to_json(rapidjson::Document* document, rapidjson::Value* query) override; +private: + std::string _es_query_str; +}; + +// process field = value +class TermQueryBuilder : public QueryBuilder { + +public: + TermQueryBuilder(const std::string& field, const std::string& term); + TermQueryBuilder(const ExtBinaryPredicate& binary_predicate); + void to_json(rapidjson::Document* document, rapidjson::Value* query) override; + +private: + std::string _field; + std::string _term; +}; + +// process range predicate field >= value or field < value etc. +class RangeQueryBuilder : public QueryBuilder { + +public: + RangeQueryBuilder(const ExtBinaryPredicate& range_predicate); + void to_json(rapidjson::Document* document, rapidjson::Value* query) override; +private: + std::string _field; + std::string _value; + TExprOpcode::type _op; +}; + +// process in predicate : field in [value1, value2] +class TermsInSetQueryBuilder : public QueryBuilder { + +public: + TermsInSetQueryBuilder(const ExtInPredicate& in_predicate); + void to_json(rapidjson::Document* document, rapidjson::Value* query) override; +private: + std::string _field; + std::vector _values; +}; + +// process like predicate : field like "a%b%c_" +class WildCardQueryBuilder : public QueryBuilder { + +public: + WildCardQueryBuilder(const ExtLikePredicate& like_predicate); + void to_json(rapidjson::Document* document, rapidjson::Value* query) override; + +private: + std::string _like_value; + std::string _field; +}; + +// no predicates: all doccument match +class MatchAllQueryBuilder : public QueryBuilder { + +public: + void to_json(rapidjson::Document* document, rapidjson::Value* query) override; +}; + +// proccess bool compound query, and play the role of a bridge for transferring predicates to es native query +class BooleanQueryBuilder : public QueryBuilder { + +public: + BooleanQueryBuilder(const std::vector& predicates); + BooleanQueryBuilder(); + ~BooleanQueryBuilder(); + // class method for transfer predicate to es query value, invoker should enclose this value with `query` + static void to_query(const std::vector& predicates, rapidjson::Document* root, rapidjson::Value* query); + // validate esquery syntax + static Status check_es_query(const ExtFunction& extFunction); + // decide which predicate can process + static void validate(const std::vector& espredicates, std::vector* result); + +private: + // add child query + void should(QueryBuilder* filter); + void filter(QueryBuilder* filter); + void must(QueryBuilder* filter); + void must_not(QueryBuilder* filter); + void to_json(rapidjson::Document* document, rapidjson::Value* query) override; + + std::vector _must_clauses; + std::vector _must_not_clauses; + std::vector _filter_clauses; + std::vector _should_clauses; +}; + +} diff --git a/be/src/exec/es/es_scan_reader.cpp b/be/src/exec/es/es_scan_reader.cpp new file mode 100644 index 00000000000000..e2f17a0908d1a7 --- /dev/null +++ b/be/src/exec/es/es_scan_reader.cpp @@ -0,0 +1,157 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es/es_scan_reader.h" + +#include +#include +#include + +#include "common/logging.h" +#include "common/status.h" +#include "exec/es/es_scroll_query.h" + +namespace doris { +const std::string REUQEST_SCROLL_FILTER_PATH = "filter_path=_scroll_id,hits.hits._source,hits.total,_id,hits.hits._source.fields"; +const std::string REQUEST_SCROLL_PATH = "_scroll"; +const std::string REQUEST_PREFERENCE_PREFIX = "&preference=_shards:"; +const std::string REQUEST_SEARCH_SCROLL_PATH = "/_search/scroll"; +const std::string REQUEST_SEPARATOR = "/"; +const std::string REQUEST_SCROLL_TIME = "5m"; + +ESScanReader::ESScanReader(const std::string& target, const std::map& props) { + _target = target; + _index = props.at(KEY_INDEX); + _type = props.at(KEY_TYPE); + if (props.find(KEY_USER_NAME) != props.end()) { + _user_name = props.at(KEY_USER_NAME); + } + if (props.find(KEY_PASS_WORD) != props.end()){ + _passwd = props.at(KEY_PASS_WORD); + } + if (props.find(KEY_SHARD) != props.end()) { + _shards = props.at(KEY_SHARD); + } + if (props.find(KEY_QUERY) != props.end()) { + _query = props.at(KEY_QUERY); + } + std::string batch_size_str = props.at(KEY_BATCH_SIZE); + _batch_size = atoi(batch_size_str.c_str()); + _init_scroll_url = _target + REQUEST_SEPARATOR + _index + REQUEST_SEPARATOR + _type + "/_search?scroll=" + REQUEST_SCROLL_TIME + REQUEST_PREFERENCE_PREFIX + _shards + "&" + REUQEST_SCROLL_FILTER_PATH; + _next_scroll_url = _target + REQUEST_SEARCH_SCROLL_PATH + "?" + REUQEST_SCROLL_FILTER_PATH; + _eos = false; +} + +ESScanReader::~ESScanReader() { +} + +Status ESScanReader::open() { + _is_first = true; + RETURN_IF_ERROR(_network_client.init(_init_scroll_url)); + _network_client.set_basic_auth(_user_name, _passwd); + _network_client.set_content_type("application/json"); + // phase open, we cached the first response for `get_next` phase + Status status = _network_client.execute_post_request(_query, &_cached_response); + if (!status.ok() || _network_client.get_http_status() != 200) { + std::stringstream ss; + ss << "Failed to connect to ES server, errmsg is: " << status.get_error_msg(); + LOG(WARNING) << ss.str(); + return Status(ss.str()); + } + VLOG(1) << "open _cached response: " << _cached_response; + return Status::OK; +} + +Status ESScanReader::get_next(bool* scan_eos, std::unique_ptr& scroll_parser) { + std::string response; + // if is first scroll request, should return the cached response + *scan_eos = true; + if (_eos) { + return Status::OK; + } + + if (_is_first) { + response = _cached_response; + _is_first = false; + } else { + RETURN_IF_ERROR(_network_client.init(_next_scroll_url)); + _network_client.set_basic_auth(_user_name, _passwd); + _network_client.set_content_type("application/json"); + _network_client.set_timeout_ms(5 * 1000); + RETURN_IF_ERROR(_network_client.execute_post_request( + ESScrollQueryBuilder::build_next_scroll_body(_scroll_id, REQUEST_SCROLL_TIME), &response)); + long status = _network_client.get_http_status(); + if (status == 404) { + LOG(WARNING) << "request scroll search failure 404[" + << ", response: " << (response.empty() ? "empty response" : response); + return Status("No search context found for " + _scroll_id); + } + if (status != 200) { + LOG(WARNING) << "request scroll search failure[" + << "http status" << status + << ", response: " << (response.empty() ? "empty response" : response); + if (status == 404) { + return Status("No search context found for " + _scroll_id); + } + return Status("request scroll search failure: " + (response.empty() ? "empty response" : response)); + } + } + + scroll_parser.reset(new ScrollParser()); + Status status = scroll_parser->parse(response); + if (!status.ok()){ + _eos = true; + LOG(WARNING) << status.get_error_msg(); + return status; + } + + _scroll_id = scroll_parser->get_scroll_id(); + if (scroll_parser->get_total() == 0) { + _eos = true; + return Status::OK; + } + + if (scroll_parser->get_size() < _batch_size) { + _eos = true; + } else { + _eos = false; + } + + *scan_eos = false; + return Status::OK; +} + +Status ESScanReader::close() { + if (_scroll_id.empty()) { + return Status::OK; + } + + std::string scratch_target = _target + REQUEST_SEARCH_SCROLL_PATH; + RETURN_IF_ERROR(_network_client.init(scratch_target)); + _network_client.set_basic_auth(_user_name, _passwd); + _network_client.set_method(DELETE); + _network_client.set_content_type("application/json"); + _network_client.set_timeout_ms(5 * 1000); + std::string response; + RETURN_IF_ERROR(_network_client.execute_delete_request(ESScrollQueryBuilder::build_clear_scroll_body(_scroll_id), &response)); + if (_network_client.get_http_status() == 200) { + return Status::OK; + } else { + return Status("es_scan_reader delete scroll context failure"); + } +} +} diff --git a/be/src/exec/es/es_scan_reader.h b/be/src/exec/es/es_scan_reader.h new file mode 100644 index 00000000000000..b03701c11e772c --- /dev/null +++ b/be/src/exec/es/es_scan_reader.h @@ -0,0 +1,74 @@ +// 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 "exec/es/es_scroll_parser.h" +#include "http/http_client.h" + +using std::string; + +namespace doris { + +class Status; + +class ESScanReader { + +public: + static constexpr const char* KEY_USER_NAME = "user"; + static constexpr const char* KEY_PASS_WORD = "password"; + static constexpr const char* KEY_HOST_PORT = "host_port"; + static constexpr const char* KEY_INDEX = "index"; + static constexpr const char* KEY_TYPE = "type"; + static constexpr const char* KEY_SHARD = "shard_id"; + static constexpr const char* KEY_QUERY = "query"; + static constexpr const char* KEY_BATCH_SIZE = "batch_size"; + ESScanReader(const std::string& target, const std::map& props); + ~ESScanReader(); + + // launch the first scroll request, this method will cache the first scroll response, and return the this cached response when invoke get_next + Status open(); + // invoke get_next to get next batch documents from elasticsearch + Status get_next(bool *eos, std::unique_ptr& parser); + // clear scroll context from elasticsearch + Status close(); + +private: + std::string _target; + std::string _user_name; + std::string _passwd; + std::string _scroll_id; + HttpClient _network_client; + std::string _index; + std::string _type; + // push down filter + std::string _query; + // elaticsearch shards to fetch document + std::string _shards; + // distinguish the first scroll phase and the following scroll + bool _is_first; + std::string _init_scroll_url; + std::string _next_scroll_url; + bool _eos; + int _batch_size; + + std::string _cached_response; +}; +} + diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp new file mode 100644 index 00000000000000..e1ee317f777cf5 --- /dev/null +++ b/be/src/exec/es/es_scroll_parser.cpp @@ -0,0 +1,353 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es/es_scroll_parser.h" + +#include +#include +#include + +#include "common/logging.h" +#include "common/status.h" +#include "runtime/mem_pool.h" +#include "runtime/mem_tracker.h" +#include "util/string_parser.hpp" + +namespace doris { + +static const char* FIELD_SCROLL_ID = "_scroll_id"; +static const char* FIELD_HITS = "hits"; +static const char* FIELD_INNER_HITS = "hits"; +static const char* FIELD_SOURCE = "_source"; +static const char* FIELD_TOTAL = "total"; + +static const string ERROR_INVALID_COL_DATA = "Data source returned inconsistent column data. " + "Expected value of type $0 based on column metadata. This likely indicates a " + "problem with the data source library."; +static const string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate " + "$1 bytes for $2."; +static const string ERROR_COL_DATA_IS_ARRAY = "Data source returned an array for the type $0" + "based on column metadata."; + +#define RETURN_ERROR_IF_COL_IS_ARRAY(col, type) \ + do { \ + if (col.IsArray()) { \ + return Status(strings::Substitute(ERROR_COL_DATA_IS_ARRAY, type_to_string(type))); \ + } \ + } while (false) + + +#define RETURN_ERROR_IF_COL_IS_NOT_STRING(col, type) \ + do { \ + if (!col.IsString()) { \ + return Status(strings::Substitute(ERROR_INVALID_COL_DATA, type_to_string(type))); \ + } \ + } while (false) + + +#define RETURN_ERROR_IF_PARSING_FAILED(result, type) \ + do { \ + if (result != StringParser::PARSE_SUCCESS) { \ + return Status(strings::Substitute(ERROR_INVALID_COL_DATA, type_to_string(type))); \ + } \ + } while (false) + +template +static Status get_int_value(const rapidjson::Value &col, PrimitiveType type, void* slot) { + if (col.IsNumber()) { + *reinterpret_cast(slot) = (T)(sizeof(T) < 8 ? col.GetInt() : col.GetInt64()); + 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, type); + + if (sizeof(T) < 16) { + *reinterpret_cast(slot) = v; + } else { + DCHECK(sizeof(T) == 16); + memcpy(slot, &v, sizeof(v)); + } + + return Status::OK; +} + +template +static Status get_float_value(const rapidjson::Value &col, PrimitiveType type, void* slot) { + DCHECK(sizeof(T) == 4 || sizeof(T) == 8); + if (col.IsNumber()) { + *reinterpret_cast(slot) = (T)(sizeof(T) == 4 ? col.GetFloat() : col.GetDouble()); + 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, type); + *reinterpret_cast(slot) = v; + + return Status::OK; +} + +ScrollParser::ScrollParser() : + _scroll_id(""), + _total(0), + _size(0), + _line_index(0) { +} + +ScrollParser::~ScrollParser() { +} + +Status ScrollParser::parse(const std::string& scroll_result) { + _document_node.Parse(scroll_result.c_str()); + if (_document_node.HasParseError()) { + std::stringstream ss; + ss << "Parsing json error, json is: " << scroll_result; + return Status(ss.str()); + } + + if (!_document_node.HasMember(FIELD_SCROLL_ID)) { + return Status("Document has not a scroll id field"); + } + + const rapidjson::Value &scroll_node = _document_node[FIELD_SCROLL_ID]; + _scroll_id = scroll_node.GetString(); + // { hits: { total : 2, "hits" : [ {}, {}, {} ]}} + const rapidjson::Value &outer_hits_node = _document_node[FIELD_HITS]; + const rapidjson::Value &field_total = outer_hits_node[FIELD_TOTAL]; + _total = field_total.GetInt(); + if (_total == 0) { + return Status::OK; + } + + VLOG(1) << "es_scan_reader total hits: " << _total << " documents"; + const rapidjson::Value &inner_hits_node = outer_hits_node[FIELD_INNER_HITS]; + if (!inner_hits_node.IsArray()) { + return Status("inner hits node is not an array"); + } + + rapidjson::Document::AllocatorType& a = _document_node.GetAllocator(); + _inner_hits_node.CopyFrom(inner_hits_node, a); + _size = _inner_hits_node.Size(); + + return Status::OK; +} + +int ScrollParser::get_size() { + return _size; +} + +const std::string& ScrollParser::get_scroll_id() { + return _scroll_id; +} + +int ScrollParser::get_total() { + return _total; +} + +Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, + Tuple* tuple, MemPool* tuple_pool, bool* line_eof) { + *line_eof = true; + if (_size <= 0 || _line_index >= _size) { + return Status::OK; + } + + const rapidjson::Value& obj = _inner_hits_node[_line_index++]; + const rapidjson::Value& line = obj[FIELD_SOURCE]; + if (!line.IsObject()) { + return Status("Parse inner hits failed"); + } + + tuple->init(tuple_desc->byte_size()); + for (int i = 0; i < tuple_desc->slots().size(); ++i) { + const SlotDescriptor* slot_desc = tuple_desc->slots()[i]; + + if (!slot_desc->is_materialized()) { + continue; + } + + const char* col_name = slot_desc->col_name().c_str(); + rapidjson::Value::ConstMemberIterator itr = line.FindMember(col_name); + if (itr == line.MemberEnd()) { + tuple->set_null(slot_desc->null_indicator_offset()); + continue; + } + + tuple->set_not_null(slot_desc->null_indicator_offset()); + const rapidjson::Value &col = line[col_name]; + + void* slot = tuple->get_slot(slot_desc->tuple_offset()); + PrimitiveType type = slot_desc->type().type; + switch (type) { + case TYPE_CHAR: + case TYPE_VARCHAR: { + 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(); + char* buffer = reinterpret_cast(tuple_pool->try_allocate_unaligned(val_size)); + if (UNLIKELY(buffer == NULL)) { + string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", + val_size, "string slot"); + return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, val_size); + } + memcpy(buffer, val.data(), val_size); + reinterpret_cast(slot)->ptr = buffer; + reinterpret_cast(slot)->len = val_size; + break; + } + + case TYPE_TINYINT: { + Status status = get_int_value(col, type, slot); + if (!status.ok()) { + return status; + } + break; + } + + case TYPE_SMALLINT: { + Status status = get_int_value(col, type, slot); + if (!status.ok()) { + return status; + } + break; + } + + case TYPE_INT: { + Status status = get_int_value(col, type, slot); + if (!status.ok()) { + return status; + } + break; + } + + case TYPE_BIGINT: { + Status status = get_int_value(col, type, slot); + if (!status.ok()) { + return status; + } + break; + } + + case TYPE_LARGEINT: { + Status status = get_int_value<__int128>(col, type, slot); + if (!status.ok()) { + return status; + } + break; + } + + case TYPE_DOUBLE: { + Status status = get_float_value(col, type, slot); + if (!status.ok()) { + return status; + } + break; + } + + case TYPE_FLOAT: { + Status status = get_float_value(col, type, slot); + if (!status.ok()) { + return status; + } + break; + } + + case TYPE_BOOLEAN: { + if (col.IsBool()) { + *reinterpret_cast(slot) = col.GetBool(); + break; + } + + if (col.IsNumber()) { + *reinterpret_cast(slot) = col.GetInt(); + break; + } + + 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(); + StringParser::ParseResult result; + bool b = + StringParser::string_to_bool(val.c_str(), val_size, &result); + RETURN_ERROR_IF_PARSING_FAILED(result, type); + *reinterpret_cast(slot) = b; + break; + } + + case TYPE_DATE: + case TYPE_DATETIME: { + if (col.IsNumber()) { + if (!reinterpret_cast(slot)->from_unixtime(col.GetInt64())) { + return Status(strings::Substitute(ERROR_INVALID_COL_DATA, type_to_string(type))); + } + + if (type == TYPE_DATE) { + reinterpret_cast(slot)->cast_to_date(); + } else { + reinterpret_cast(slot)->set_type(TIME_DATETIME); + } + break; + } + + RETURN_ERROR_IF_COL_IS_ARRAY(col, type); + RETURN_ERROR_IF_COL_IS_NOT_STRING(col, type); + + DateTimeValue* ts_slot = reinterpret_cast(slot); + const std::string& val = col.GetString(); + size_t val_size = col.GetStringLength(); + if (!ts_slot->from_date_str(val.c_str(), val_size)) { + return Status(strings::Substitute(ERROR_INVALID_COL_DATA, type_to_string(type))); + } + + if (ts_slot->year() < 1900) { + return Status(strings::Substitute(ERROR_INVALID_COL_DATA, type_to_string(type))); + } + + if (type == TYPE_DATE) { + ts_slot->cast_to_date(); + } else { + ts_slot->to_datetime(); + } + break; + } + + default: { + DCHECK(false); + break; + } + } + } + + *line_eof = false; + return Status::OK; +} +} diff --git a/be/src/runtime/kafka_consumer_pipe.cpp b/be/src/exec/es/es_scroll_parser.h similarity index 55% rename from be/src/runtime/kafka_consumer_pipe.cpp rename to be/src/exec/es/es_scroll_parser.h index 10b7fd83edb9d7..5af75a85eec2aa 100644 --- a/be/src/runtime/kafka_consumer_pipe.cpp +++ b/be/src/exec/es/es_scroll_parser.h @@ -15,9 +15,40 @@ // specific language governing permissions and limitations // under the License. -#include "runtime/kafka_consumer_pipe.h" +#pragma once + +#include + +#include "rapidjson/document.h" +#include "runtime/descriptors.h" +#include "runtime/tuple.h" namespace doris { +class Status; + +class ScrollParser { + +public: + ScrollParser(); + ~ScrollParser(); + + Status parse(const std::string& scroll_result); + Status fill_tuple(const TupleDescriptor* _tuple_desc, Tuple* tuple, + MemPool* mem_pool, bool* line_eof); + + const std::string& get_scroll_id(); + int get_total(); + int get_size(); + +private: + + std::string _scroll_id; + int _total; + int _size; + rapidjson::SizeType _line_index; -} // end namespace doris + rapidjson::Document _document_node; + rapidjson::Value _inner_hits_node; +}; +} diff --git a/be/src/exec/es/es_scroll_query.cpp b/be/src/exec/es/es_scroll_query.cpp new file mode 100644 index 00000000000000..1c405136e749d3 --- /dev/null +++ b/be/src/exec/es/es_scroll_query.cpp @@ -0,0 +1,102 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es/es_scroll_query.h" + +#include +#include + +#include "common/logging.h" +#include "exec/es/es_query_builder.h" +#include "exec/es/es_scan_reader.h" +#include "rapidjson/document.h" +#include "rapidjson/stringbuffer.h" +#include "rapidjson/writer.h" + +namespace doris { + +ESScrollQueryBuilder::ESScrollQueryBuilder() { + +} + +ESScrollQueryBuilder::~ESScrollQueryBuilder() { + +} + +std::string ESScrollQueryBuilder::build_next_scroll_body(const std::string& scroll_id, const std::string& scroll) { + rapidjson::Document scroll_dsl; + rapidjson::Document::AllocatorType &allocator = scroll_dsl.GetAllocator(); + scroll_dsl.SetObject(); + rapidjson::Value scroll_id_value(scroll_id.c_str(), allocator); + scroll_dsl.AddMember("scroll_id", scroll_id_value, allocator); + rapidjson::Value scroll_value(scroll.c_str(), allocator); + scroll_dsl.AddMember("scroll", scroll_value, allocator); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + scroll_dsl.Accept(writer); + return buffer.GetString(); +} +std::string ESScrollQueryBuilder::build_clear_scroll_body(const std::string& scroll_id) { + rapidjson::Document delete_scroll_dsl; + rapidjson::Document::AllocatorType &allocator = delete_scroll_dsl.GetAllocator(); + delete_scroll_dsl.SetObject(); + rapidjson::Value scroll_id_value(scroll_id.c_str(), allocator); + delete_scroll_dsl.AddMember("scroll_id", scroll_id_value, allocator); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + delete_scroll_dsl.Accept(writer); + return buffer.GetString(); +} + +std::string ESScrollQueryBuilder::build(const std::map& properties, + const std::vector& fields, + std::vector& predicates) { + rapidjson::Document es_query_dsl; + rapidjson::Document::AllocatorType &allocator = es_query_dsl.GetAllocator(); + es_query_dsl.SetObject(); + // generate the filter caluse + rapidjson::Document scratch_document; + rapidjson::Value query_node(rapidjson::kObjectType); + query_node.SetObject(); + BooleanQueryBuilder::to_query(predicates, &scratch_document, &query_node); + // note: add `query` for this value.... + es_query_dsl.AddMember("query", query_node, allocator); + // just filter the selected fields for reducing the network cost + if (fields.size() > 0) { + rapidjson::Value source_node(rapidjson::kArrayType); + for (auto iter = fields.begin(); iter != fields.end(); iter++) { + rapidjson::Value field(iter->c_str(), allocator); + source_node.PushBack(field, allocator); + } + es_query_dsl.AddMember("_source", source_node, allocator); + } + int size = atoi(properties.at(ESScanReader::KEY_BATCH_SIZE).c_str()); + rapidjson::Value sort_node(rapidjson::kArrayType); + // use the scroll-scan mode for scan index documents + rapidjson::Value field("_doc", allocator); + sort_node.PushBack(field, allocator); + es_query_dsl.AddMember("sort", sort_node, allocator); + // number of docuements returned + es_query_dsl.AddMember("size", size, allocator); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + es_query_dsl.Accept(writer); + std::string es_query_dsl_json = buffer.GetString(); + return es_query_dsl_json; + +} +} diff --git a/be/src/exec/es/es_scroll_query.h b/be/src/exec/es/es_scroll_query.h new file mode 100644 index 00000000000000..0f6c20457ad713 --- /dev/null +++ b/be/src/exec/es/es_scroll_query.h @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include + +#include "exec/es/es_predicate.h" + +namespace doris { + +class ESScrollQueryBuilder { + +public: + ESScrollQueryBuilder(); + ~ESScrollQueryBuilder(); + // build the query DSL for elasticsearch + static std::string build_next_scroll_body(const std::string& scroll_id, const std::string& scroll); + static std::string build_clear_scroll_body(const std::string& scroll_id); + // @note: predicates should processed before pass it to this method, + // tie breaker for predicate wheather can push down es can reference the push-down filters + static std::string build(const std::map& properties, + const std::vector& fields, std::vector& predicates); +}; +} diff --git a/be/src/exec/es_http_scan_node.cpp b/be/src/exec/es_http_scan_node.cpp new file mode 100644 index 00000000000000..5467b4966d4d20 --- /dev/null +++ b/be/src/exec/es_http_scan_node.cpp @@ -0,0 +1,448 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es_http_scan_node.h" + +#include +#include + +#include "common/object_pool.h" +#include "exec/es/es_predicate.h" +#include "exec/es/es_query_builder.h" +#include "exec/es/es_scan_reader.h" +#include "exec/es/es_scroll_query.h" +#include "exprs/expr.h" +#include "runtime/runtime_state.h" +#include "runtime/row_batch.h" +#include "runtime/dpp_sink_internal.h" +#include "service/backend_options.h" +#include "util/runtime_profile.h" + +namespace doris { + +EsHttpScanNode::EsHttpScanNode( + ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : + ScanNode(pool, tnode, descs), + _tuple_id(tnode.es_scan_node.tuple_id), + _runtime_state(nullptr), + _tuple_desc(nullptr), + _num_running_scanners(0), + _scan_finished(false), + _eos(false), + _max_buffered_batches(1024), + _wait_scanner_timer(nullptr) { +} + +EsHttpScanNode::~EsHttpScanNode() { +} + +Status EsHttpScanNode::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(ScanNode::init(tnode)); + + // use TEsScanNode + _properties = tnode.es_scan_node.properties; + return Status::OK; +} + +Status EsHttpScanNode::prepare(RuntimeState* state) { + VLOG_QUERY << "EsHttpScanNode prepare"; + RETURN_IF_ERROR(ScanNode::prepare(state)); + + _runtime_state = state; + _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); + if (_tuple_desc == nullptr) { + std::stringstream ss; + ss << "Failed to get tuple descriptor, _tuple_id=" << _tuple_id; + return Status(ss.str()); + } + + // set up column name vector for ESScrollQueryBuilder + for (auto slot_desc : _tuple_desc->slots()) { + if (!slot_desc->is_materialized()) { + continue; + } + _column_names.push_back(slot_desc->col_name()); + } + + _wait_scanner_timer = ADD_TIMER(runtime_profile(), "WaitScannerTime"); + + return Status::OK; +} + +// build predicate +Status EsHttpScanNode::build_conjuncts_list() { + Status status = Status::OK; + for (int i = 0; i < _conjunct_ctxs.size(); ++i) { + EsPredicate* predicate = _pool->add( + new EsPredicate(_conjunct_ctxs[i], _tuple_desc)); + status = predicate->build_disjuncts_list(); + if (status.ok()) { + _predicates.push_back(predicate); + _predicate_to_conjunct.push_back(i); + } else { + VLOG(1) << status.get_error_msg(); + status = predicate->get_es_query_status(); + if (!status.ok()) { + LOG(WARNING) << status.get_error_msg(); + return status; + } + } + } + + return Status::OK; +} + +Status EsHttpScanNode::open(RuntimeState* state) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + RETURN_IF_ERROR(ExecNode::open(state)); + RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::OPEN)); + RETURN_IF_CANCELLED(state); + + // if conjunct is constant, compute direct and set eos = true + for (int conj_idx = 0; conj_idx < _conjunct_ctxs.size(); ++conj_idx) { + if (_conjunct_ctxs[conj_idx]->root()->is_constant()) { + void* value = _conjunct_ctxs[conj_idx]->get_value(NULL); + if (value == NULL || *reinterpret_cast(value) == false) { + _eos = true; + } + } + } + + RETURN_IF_ERROR(build_conjuncts_list()); + + // remove those predicates which ES cannot support + std::vector list; + BooleanQueryBuilder::validate(_predicates, &list); + DCHECK(list.size() == _predicate_to_conjunct.size()); + for(int i = list.size() - 1; i >= 0; i--) { + if(!list[i]) { + _predicate_to_conjunct.erase(_predicate_to_conjunct.begin() + i); + _predicates.erase(_predicates.begin() + i); + } + } + + // filter the conjuncts and ES will process them later + for (int i = _predicate_to_conjunct.size() - 1; i >= 0; i--) { + int conjunct_index = _predicate_to_conjunct[i]; + _conjunct_ctxs[conjunct_index]->close(_runtime_state); + _conjunct_ctxs.erase(_conjunct_ctxs.begin() + conjunct_index); + } + + RETURN_IF_ERROR(start_scanners()); + + return Status::OK; +} + +Status EsHttpScanNode::start_scanners() { + { + std::unique_lock l(_batch_queue_lock); + _num_running_scanners = _scan_ranges.size(); + } + + for (int i = 0; i < _scan_ranges.size(); i++) { + std::promise p; + std::future f = p.get_future(); + _scanner_threads.emplace_back(&EsHttpScanNode::scanner_worker, this, i, + _scan_ranges.size(), std::ref(p)); + Status status = f.get(); + if (!status.ok()) return status; + } + return Status::OK; +} + +Status EsHttpScanNode::get_next(RuntimeState* state, RowBatch* row_batch, + bool* eos) { + SCOPED_TIMER(_runtime_profile->total_time_counter()); + if (state->is_cancelled()) { + std::unique_lock l(_batch_queue_lock); + if (update_status(Status::CANCELLED)) { + _queue_writer_cond.notify_all(); + } + } + + if (_eos) { + *eos = true; + return Status::OK; + } + + if (_scan_finished.load()) { + *eos = true; + return Status::OK; + } + + std::shared_ptr scanner_batch; + { + std::unique_lock l(_batch_queue_lock); + while (_process_status.ok() && + !_runtime_state->is_cancelled() && + _num_running_scanners > 0 && + _batch_queue.empty()) { + SCOPED_TIMER(_wait_scanner_timer); + _queue_reader_cond.wait_for(l, std::chrono::seconds(1)); + } + if (!_process_status.ok()) { + // Some scanner process failed. + return _process_status; + } + if (_runtime_state->is_cancelled()) { + if (update_status(Status::CANCELLED)) { + _queue_writer_cond.notify_all(); + } + return _process_status; + } + if (!_batch_queue.empty()) { + scanner_batch = _batch_queue.front(); + _batch_queue.pop_front(); + } + } + + // All scanner has been finished, and all cached batch has been read + if (scanner_batch == nullptr) { + _scan_finished.store(true); + *eos = true; + return Status::OK; + } + + // notify one scanner + _queue_writer_cond.notify_one(); + + // get scanner's batch memory + row_batch->acquire_state(scanner_batch.get()); + _num_rows_returned += row_batch->num_rows(); + COUNTER_SET(_rows_returned_counter, _num_rows_returned); + + // This is first time reach limit. + // Only valid when query 'select * from table1 limit 20' + if (reached_limit()) { + int num_rows_over = _num_rows_returned - _limit; + row_batch->set_num_rows(row_batch->num_rows() - num_rows_over); + _num_rows_returned -= num_rows_over; + COUNTER_SET(_rows_returned_counter, _num_rows_returned); + + _scan_finished.store(true); + _queue_writer_cond.notify_all(); + *eos = true; + } else { + *eos = false; + } + + if (VLOG_ROW_IS_ON) { + for (int i = 0; i < row_batch->num_rows(); ++i) { + TupleRow* row = row_batch->get_row(i); + VLOG_ROW << "EsHttpScanNode output row: " + << Tuple::to_string(row->get_tuple(0), *_tuple_desc); + } + } + + return Status::OK; +} + +Status EsHttpScanNode::close(RuntimeState* state) { + if (is_closed()) { + return Status::OK; + } + RETURN_IF_ERROR(exec_debug_action(TExecNodePhase::CLOSE)); + SCOPED_TIMER(_runtime_profile->total_time_counter()); + _scan_finished.store(true); + _queue_writer_cond.notify_all(); + _queue_reader_cond.notify_all(); + for (int i = 0; i < _scanner_threads.size(); ++i) { + _scanner_threads[i].join(); + } + + _batch_queue.clear(); + + return ExecNode::close(state); +} + +// This function is called after plan node has been prepared. +Status EsHttpScanNode::set_scan_ranges(const std::vector& scan_ranges) { + _scan_ranges = scan_ranges; + return Status::OK; +} + +void EsHttpScanNode::debug_string(int ident_level, std::stringstream* out) const { + (*out) << "EsHttpScanNode"; +} + +Status EsHttpScanNode::scanner_scan( + std::unique_ptr scanner, + const std::vector& conjunct_ctxs, + EsScanCounter* counter) { + RETURN_IF_ERROR(scanner->open()); + bool scanner_eof = false; + + while (!scanner_eof) { + // Fill one row batch + std::shared_ptr row_batch( + new RowBatch(row_desc(), _runtime_state->batch_size(), mem_tracker())); + + // create new tuple buffer for row_batch + MemPool* tuple_pool = row_batch->tuple_data_pool(); + int tuple_buffer_size = row_batch->capacity() * _tuple_desc->byte_size(); + void* tuple_buffer = tuple_pool->allocate(tuple_buffer_size); + if (tuple_buffer == nullptr) { + return Status("Allocate memory for row batch failed."); + } + + Tuple* tuple = reinterpret_cast(tuple_buffer); + while (!scanner_eof) { + RETURN_IF_CANCELLED(_runtime_state); + // If we have finished all works + if (_scan_finished.load()) { + return Status::OK; + } + + // This row batch has been filled up, and break this + if (row_batch->is_full()) { + break; + } + + int row_idx = row_batch->add_row(); + TupleRow* row = row_batch->get_row(row_idx); + // scan node is the first tuple of tuple row + row->set_tuple(0, tuple); + memset(tuple, 0, _tuple_desc->num_null_bytes()); + + // Get from scanner + RETURN_IF_ERROR(scanner->get_next(tuple, tuple_pool, &scanner_eof)); + if (scanner_eof) { + continue; + } + + // eval conjuncts of this row. + if (eval_conjuncts(&conjunct_ctxs[0], conjunct_ctxs.size(), row)) { + row_batch->commit_last_row(); + char* new_tuple = reinterpret_cast(tuple); + new_tuple += _tuple_desc->byte_size(); + tuple = reinterpret_cast(new_tuple); + counter->num_rows_returned++; + } else { + counter->num_rows_filtered++; + } + } + + // Row batch has been filled, push this to the queue + if (row_batch->num_rows() > 0) { + std::unique_lock l(_batch_queue_lock); + while (_process_status.ok() && + !_scan_finished.load() && + !_runtime_state->is_cancelled() && + _batch_queue.size() >= _max_buffered_batches) { + _queue_writer_cond.wait_for(l, std::chrono::seconds(1)); + } + // Process already set failed, so we just return OK + if (!_process_status.ok()) { + return Status::OK; + } + // Scan already finished, just return + if (_scan_finished.load()) { + return Status::OK; + } + // Runtime state is canceled, just return cancel + if (_runtime_state->is_cancelled()) { + return Status::CANCELLED; + } + // Queue size Must be samller than _max_buffered_batches + _batch_queue.push_back(row_batch); + + // Notify reader to + _queue_reader_cond.notify_one(); + } + } + + return Status::OK; +} + +// Prefer to the local host +static std::string get_host_port(const std::vector& es_hosts) { + + std::string host_port; + std::string localhost = BackendOptions::get_localhost(); + + TNetworkAddress host = es_hosts[0]; + for (auto& es_host : es_hosts) { + if (es_host.hostname == localhost) { + host = es_host; + break; + } + } + + host_port = host.hostname; + host_port += ":"; + host_port += std::to_string(host.port); + return host_port; +} + +void EsHttpScanNode::scanner_worker(int start_idx, int length, std::promise& p_status) { + // Clone expr context + std::vector scanner_expr_ctxs; + DCHECK(start_idx < length); + auto status = Expr::clone_if_not_exists(_conjunct_ctxs, _runtime_state, + &scanner_expr_ctxs); + if (!status.ok()) { + LOG(WARNING) << "Clone conjuncts failed."; + } + + EsScanCounter counter; + const TEsScanRange& es_scan_range = + _scan_ranges[start_idx].scan_range.es_scan_range; + + // Collect the informations from scan range to perperties + std::map properties(_properties); + properties[ESScanReader::KEY_INDEX] = es_scan_range.index; + if (es_scan_range.__isset.type) { + properties[ESScanReader::KEY_TYPE] = es_scan_range.type; + } + properties[ESScanReader::KEY_SHARD] = std::to_string(es_scan_range.shard_id); + properties[ESScanReader::KEY_BATCH_SIZE] = std::to_string(_runtime_state->batch_size()); + properties[ESScanReader::KEY_HOST_PORT] = get_host_port(es_scan_range.es_hosts); + properties[ESScanReader::KEY_QUERY] + = ESScrollQueryBuilder::build(properties, _column_names, _predicates); + + // start scanner to scan + std::unique_ptr scanner(new EsHttpScanner( + _runtime_state, runtime_profile(), _tuple_id, + properties, scanner_expr_ctxs, &counter)); + status = scanner_scan(std::move(scanner), scanner_expr_ctxs, &counter); + if (!status.ok()) { + LOG(WARNING) << "Scanner[" << start_idx << "] process failed. status=" + << status.get_error_msg(); + } + + // Update stats + _runtime_state->update_num_rows_load_success(counter.num_rows_returned); + _runtime_state->update_num_rows_load_filtered(counter.num_rows_filtered); + + // scanner is going to finish + { + std::lock_guard l(_batch_queue_lock); + if (!status.ok()) { + update_status(status); + } + // This scanner will finish + _num_running_scanners--; + } + _queue_reader_cond.notify_all(); + // If one scanner failed, others don't need scan any more + if (!status.ok()) { + _queue_writer_cond.notify_all(); + } + + p_status.set_value(status); +} +} diff --git a/be/src/exec/es_http_scan_node.h b/be/src/exec/es_http_scan_node.h new file mode 100644 index 00000000000000..555a44a31bde5e --- /dev/null +++ b/be/src/exec/es_http_scan_node.h @@ -0,0 +1,112 @@ +// 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. + +#ifndef BE_EXEC_ES_HTTP_SCAN_NODE_H +#define BE_EXEC_ES_HTTP_SCAN_NODE_H + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/status.h" +#include "exec/scan_node.h" +#include "exec/es_http_scanner.h" +#include "gen_cpp/PaloInternalService_types.h" + +namespace doris { + +class RuntimeState; +class PartRangeKey; +class PartitionInfo; +class EsHttpScanCounter; +class EsPredicate; + +class EsHttpScanNode : public ScanNode { +public: + EsHttpScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + virtual ~EsHttpScanNode(); + + virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; + virtual Status prepare(RuntimeState* state) override; + virtual Status open(RuntimeState* state) override; + virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override; + virtual Status close(RuntimeState* state) override; + virtual Status set_scan_ranges(const std::vector& scan_ranges) override; + +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) { + if (_process_status.ok()) { + _process_status = new_status; + return true; + } + return false; + } + + // Create scanners to do scan job + Status start_scanners(); + + // One scanner worker, This scanner will hanle '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(); + + TupleId _tuple_id; + RuntimeState* _runtime_state; + TupleDescriptor* _tuple_desc; + + int _num_running_scanners; + std::atomic _scan_finished; + bool _eos; + int _max_buffered_batches; + RuntimeProfile::Counter* _wait_scanner_timer; + + bool _all_scanners_finished; + Status _process_status; + + std::vector _scanner_threads; + std::map _properties; + 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; + + std::vector _predicate_to_conjunct; +}; + +} + +#endif diff --git a/be/src/exec/es_http_scanner.cpp b/be/src/exec/es_http_scanner.cpp new file mode 100644 index 00000000000000..331ede8902ea2e --- /dev/null +++ b/be/src/exec/es_http_scanner.cpp @@ -0,0 +1,127 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es_http_scanner.h" + +#include +#include + +#include "runtime/descriptors.h" +#include "runtime/exec_env.h" +#include "runtime/mem_tracker.h" +#include "runtime/raw_value.h" +#include "runtime/runtime_state.h" +#include "runtime/tuple.h" +#include "exprs/expr.h" + +namespace doris { + +EsHttpScanner::EsHttpScanner( + RuntimeState* state, + RuntimeProfile* profile, + TupleId tuple_id, + const std::map& properties, + const std::vector& conjunct_ctxs, + EsScanCounter* counter) : + _state(state), + _profile(profile), + _tuple_id(tuple_id), + _properties(properties), + _conjunct_ctxs(conjunct_ctxs), + _next_range(0), + _line_eof(false), + _batch_eof(false), +#if BE_TEST + _mem_tracker(new MemTracker()), + _mem_pool(_mem_tracker.get()), +#else + _mem_tracker(new MemTracker(-1, "EsHttp Scanner", state->instance_mem_tracker())), + _mem_pool(_state->instance_mem_tracker()), +#endif + _tuple_desc(nullptr), + _counter(counter), + _es_reader(nullptr), + _es_scroll_parser(nullptr), + _rows_read_counter(nullptr), + _read_timer(nullptr), + _materialize_timer(nullptr) { +} + +EsHttpScanner::~EsHttpScanner() { + close(); +} + +Status EsHttpScanner::open() { + _tuple_desc = _state->desc_tbl().get_tuple_descriptor(_tuple_id); + if (_tuple_desc == nullptr) { + std::stringstream ss; + ss << "Unknown tuple descriptor, tuple_id=" << _tuple_id; + return Status(ss.str()); + } + + const std::string& host = _properties.at(ESScanReader::KEY_HOST_PORT); + _es_reader.reset(new ESScanReader(host, _properties)); + if (_es_reader == nullptr) { + return Status("Es reader construct failed."); + } + + RETURN_IF_ERROR(_es_reader->open()); + + _rows_read_counter = ADD_COUNTER(_profile, "RowsRead", TUnit::UNIT); + _read_timer = ADD_TIMER(_profile, "TotalRawReadTime(*)"); + _materialize_timer = ADD_TIMER(_profile, "MaterializeTupleTime(*)"); + + return Status::OK; +} + +Status EsHttpScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) { + SCOPED_TIMER(_read_timer); + if (_line_eof && _batch_eof) { + *eof = true; + return Status::OK; + } + + while (!_batch_eof) { + if (_line_eof || _es_scroll_parser == nullptr) { + RETURN_IF_ERROR(_es_reader->get_next(&_batch_eof, _es_scroll_parser)); + if (_batch_eof) { + *eof = true; + return Status::OK; + } + } + + COUNTER_UPDATE(_rows_read_counter, 1); + SCOPED_TIMER(_materialize_timer); + RETURN_IF_ERROR(_es_scroll_parser->fill_tuple( + _tuple_desc, tuple, tuple_pool, &_line_eof)); + if (!_line_eof) { + break; + } + } + + return Status::OK; +} + +void EsHttpScanner::close() { + if (_es_reader != nullptr) { + _es_reader->close(); + } + + Expr::close(_conjunct_ctxs, _state); +} + +} diff --git a/be/src/exec/es_http_scanner.h b/be/src/exec/es_http_scanner.h new file mode 100644 index 00000000000000..ed4cf9bd8b4294 --- /dev/null +++ b/be/src/exec/es_http_scanner.h @@ -0,0 +1,104 @@ +// 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. + +#ifndef BE_EXEC_ES_HTTP_SCANNER_H +#define BE_EXEC_ES_HTTP_SCANNER_H + +#include +#include +#include +#include +#include + +#include "common/status.h" +#include "common/global_types.h" +#include "exec/es/es_scan_reader.h" +#include "gen_cpp/PlanNodes_types.h" +#include "gen_cpp/Types_types.h" +#include "runtime/mem_pool.h" +#include "util/runtime_profile.h" + +namespace doris { + +class Tuple; +class SlotDescriptor; +class RuntimeState; +class ExprContext; +class TextConverter; +class TupleDescriptor; +class TupleRow; +class RowDescriptor; +class MemTracker; +class RuntimeProfile; + +struct EsScanCounter { + EsScanCounter() : num_rows_returned(0), num_rows_filtered(0) { + } + + int64_t num_rows_returned; + int64_t num_rows_filtered; +}; + +class EsHttpScanner { +public: + EsHttpScanner( + RuntimeState* state, + RuntimeProfile* profile, + TupleId tuple_id, + const std::map& properties, + const std::vector& conjunct_ctxs, + EsScanCounter* counter); + ~EsHttpScanner(); + + Status open(); + + Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof); + + void close(); + +private: + + RuntimeState* _state; + RuntimeProfile* _profile; + TupleId _tuple_id; + const std::map& _properties; + const std::vector& _conjunct_ctxs; + + int _next_range; + bool _line_eof; + bool _batch_eof; + + std::vector _slot_descs; + std::unique_ptr _row_desc; + + std::unique_ptr _mem_tracker; + MemPool _mem_pool; + + const TupleDescriptor* _tuple_desc; + EsScanCounter* _counter; + std::unique_ptr _es_reader; + std::unique_ptr _es_scroll_parser; + + // Profile + RuntimeProfile::Counter* _rows_read_counter; + RuntimeProfile::Counter* _read_timer; + RuntimeProfile::Counter* _materialize_timer; +}; + +} + +#endif diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 679d42c21d9249..c934cf5fff20f7 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -31,6 +31,7 @@ #include "exec/new_partitioned_aggregation_node.h" #include "exec/csv_scan_node.h" #include "exec/es_scan_node.h" +#include "exec/es_http_scan_node.h" #include "exec/pre_aggregation_node.h" #include "exec/hash_join_node.h" #include "exec/broker_scan_node.h" @@ -366,6 +367,10 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN *node = pool->add(new EsScanNode(pool, tnode, descs)); return Status::OK; + case TPlanNodeType::ES_HTTP_SCAN_NODE: + *node = pool->add(new EsHttpScanNode(pool, tnode, descs)); + return Status::OK; + case TPlanNodeType::SCHEMA_SCAN_NODE: *node = pool->add(new SchemaScanNode(pool, tnode, descs)); return Status::OK; @@ -515,6 +520,7 @@ void ExecNode::collect_scan_nodes(vector* nodes) { collect_nodes(TPlanNodeType::OLAP_SCAN_NODE, nodes); collect_nodes(TPlanNodeType::BROKER_SCAN_NODE, nodes); collect_nodes(TPlanNodeType::ES_SCAN_NODE, nodes); + collect_nodes(TPlanNodeType::ES_HTTP_SCAN_NODE, nodes); } void ExecNode::init_runtime_profile(const std::string& name) { diff --git a/be/src/exec/olap_table_sink.cpp b/be/src/exec/olap_table_sink.cpp index 6e4a7b3aacba09..e1c9168ed2873c 100644 --- a/be/src/exec/olap_table_sink.cpp +++ b/be/src/exec/olap_table_sink.cpp @@ -603,6 +603,7 @@ Status OlapTableSink::close(RuntimeState* state, Status close_status) { COUNTER_SET(_convert_batch_timer, _convert_batch_ns); COUNTER_SET(_validate_data_timer, _validate_data_ns); state->update_num_rows_load_filtered(_number_filtered_rows); + state->update_num_rows_load_success(_number_output_rows); } else { for (auto channel : _channels) { channel->cancel(); diff --git a/be/src/exec/plain_text_line_reader.cpp b/be/src/exec/plain_text_line_reader.cpp index 2eebf0f87d4b7f..2ab75768ec9789 100644 --- a/be/src/exec/plain_text_line_reader.cpp +++ b/be/src/exec/plain_text_line_reader.cpp @@ -53,7 +53,7 @@ PlainTextLineReader::PlainTextLineReader( _output_buf_limit(0), _file_eof(false), _eof(false), - _stream_end(false), + _stream_end(true), _more_input_bytes(0), _more_output_bytes(0), _bytes_read_counter(nullptr), @@ -310,7 +310,7 @@ Status PlainTextLineReader::read_line(const uint8_t** ptr, size_t* size, bool* e // (cmy), for now, return failed to avoid potential endless loop std::stringstream ss; - ss << "decompress made no progess." + ss << "decompress made no progress." << " input_read_bytes: " << input_read_bytes << " decompressed_len: " << decompressed_len; LOG(WARNING) << ss.str(); diff --git a/be/src/exec/schema_scan_node.cpp b/be/src/exec/schema_scan_node.cpp index 9977ba3a2d48ad..c99b19c9adb6f4 100644 --- a/be/src/exec/schema_scan_node.cpp +++ b/be/src/exec/schema_scan_node.cpp @@ -21,7 +21,7 @@ #include #include "exec/text_converter.hpp" -#include "exec/schema_scanner/frontend_helper.h" +#include "exec/schema_scanner/schema_helper.h" #include "gen_cpp/PlanNodes_types.h" #include "runtime/runtime_state.h" #include "runtime/row_batch.h" diff --git a/be/src/exec/schema_scanner/schema_columns_scanner.cpp b/be/src/exec/schema_scanner/schema_columns_scanner.cpp index 9a54887ed8d20a..793b4a16724623 100644 --- a/be/src/exec/schema_scanner/schema_columns_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_columns_scanner.cpp @@ -21,7 +21,7 @@ #include "runtime/primitive_type.h" #include "runtime/string_value.h" #include "runtime/datetime_value.h" -#include "exec/schema_scanner/frontend_helper.h" +#include "exec/schema_scanner/schema_helper.h" namespace doris { @@ -74,7 +74,7 @@ Status SchemaColumnsScanner::start(RuntimeState *state) { } if (NULL != _param->ip && 0 != _param->port) { - RETURN_IF_ERROR(FrontendHelper::get_db_names(*(_param->ip), + RETURN_IF_ERROR(SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); } else { return Status("IP or port dosn't exists"); @@ -152,7 +152,7 @@ Status SchemaColumnsScanner::fill_one_row(Tuple *tuple, MemPool *pool) { { void *slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset()); StringValue* str_slot = reinterpret_cast(slot); - std::string db_name = FrontendHelper::extract_db_name(_db_result.dbs[_db_index - 1]); + std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index - 1]); str_slot->ptr = (char *)pool->allocate(db_name.size()); str_slot->len = db_name.size(); memcpy(str_slot->ptr, db_name.c_str(), str_slot->len); @@ -328,7 +328,7 @@ Status SchemaColumnsScanner::get_new_desc() { } if (NULL != _param->ip && 0 != _param->port) { - RETURN_IF_ERROR(FrontendHelper::describe_table(*(_param->ip), + RETURN_IF_ERROR(SchemaHelper::describe_table(*(_param->ip), _param->port, desc_params, &_desc_result)); } else { return Status("IP or port dosn't exists"); @@ -352,7 +352,7 @@ Status SchemaColumnsScanner::get_new_table() { } if (NULL != _param->ip && 0 != _param->port) { - RETURN_IF_ERROR(FrontendHelper::get_table_names(*(_param->ip), + RETURN_IF_ERROR(SchemaHelper::get_table_names(*(_param->ip), _param->port, table_params, &_table_result)); } else { return Status("IP or port dosn't exists"); diff --git a/be/src/exec/schema_scanner/frontend_helper.cpp b/be/src/exec/schema_scanner/schema_helper.cpp similarity index 57% rename from be/src/exec/schema_scanner/frontend_helper.cpp rename to be/src/exec/schema_scanner/schema_helper.cpp index e7384451e67703..89a2f4245abf08 100644 --- a/be/src/exec/schema_scanner/frontend_helper.cpp +++ b/be/src/exec/schema_scanner/schema_helper.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "exec/schema_scanner/frontend_helper.h" +#include "exec/schema_scanner/schema_helper.h" #include @@ -35,6 +35,7 @@ #include "runtime/tuple_row.h" #include "runtime/client_cache.h" #include "util/debug_util.h" +#include "util/frontend_helper.h" #include "util/network_util.h" #include "util/thrift_util.h" #include "util/runtime_profile.h" @@ -42,113 +43,62 @@ namespace doris { -ExecEnv* FrontendHelper::_s_exec_env; - -using apache::thrift::protocol::TProtocol; -using apache::thrift::protocol::TBinaryProtocol; -using apache::thrift::transport::TSocket; -using apache::thrift::transport::TTransport; -using apache::thrift::transport::TBufferedTransport; - -void FrontendHelper::setup(ExecEnv* exec_env) { - _s_exec_env = exec_env; -} - -Status FrontendHelper::get_db_names( +Status SchemaHelper::get_db_names( const std::string& ip, const int32_t port, const TGetDbsParams &request, TGetDbsResult *result) { - return rpc(ip, port, + return FrontendHelper::rpc(ip, port, [&request, &result] (FrontendServiceConnection& client) { client->getDbNames(*result, request); }); } -Status FrontendHelper::get_table_names( +Status SchemaHelper::get_table_names( const std::string& ip, const int32_t port, const TGetTablesParams &request, TGetTablesResult *result) { - return rpc(ip, port, + return FrontendHelper::rpc(ip, port, [&request, &result] (FrontendServiceConnection& client) { client->getTableNames(*result, request); }); } -Status FrontendHelper::list_table_status( +Status SchemaHelper::list_table_status( const std::string& ip, const int32_t port, const TGetTablesParams &request, TListTableStatusResult *result) { - return rpc(ip, port, + return FrontendHelper::rpc(ip, port, [&request, &result] (FrontendServiceConnection& client) { client->listTableStatus(*result, request); }); } -Status FrontendHelper::describe_table( +Status SchemaHelper::describe_table( const std::string& ip, const int32_t port, const TDescribeTableParams &request, TDescribeTableResult *result) { - return rpc(ip, port, + return FrontendHelper::rpc(ip, port, [&request, &result] (FrontendServiceConnection& client) { client->describeTable(*result, request); }); } -Status FrontendHelper::show_varialbes( +Status SchemaHelper::show_varialbes( const std::string& ip, const int32_t port, const TShowVariableRequest &request, TShowVariableResult *result) { - return rpc(ip, port, + return FrontendHelper::rpc(ip, port, [&request, &result] (FrontendServiceConnection& client) { client->showVariables(*result, request); }); } -Status FrontendHelper::rpc( - const std::string& ip, - const int32_t port, - std::function callback, - int timeout_ms) { - TNetworkAddress address = make_network_address(ip, port); - Status status; - FrontendServiceConnection client( - _s_exec_env->frontend_client_cache(), address, timeout_ms, &status); - if (!status.ok()) { - LOG(WARNING) << "Connect frontent failed, address=" << address - << ", status=" << status.get_error_msg(); - return status; - } - try { - try { - callback(client); - } catch (apache::thrift::transport::TTransportException& e) { - LOG(WARNING) << "retrying call frontend service, address=" - << address << ", reason=" << e.what(); - status = client.reopen(timeout_ms); - if (!status.ok()) { - LOG(WARNING) << "client repoen failed. address=" << address - << ", status=" << status.get_error_msg(); - return status; - } - callback(client); - } - } catch (apache::thrift::TException& e) { - // just reopen to disable this connection - client.reopen(timeout_ms); - LOG(WARNING) << "call frontend service failed, address=" << address - << ", reason=" << e.what(); - return Status(TStatusCode::THRIFT_RPC_ERROR, - "failed to call frontend service", false); - } - return Status::OK; -} - -std::string FrontendHelper::extract_db_name(const std::string& full_name) { +std::string SchemaHelper::extract_db_name(const std::string& full_name) { auto found = full_name.find(':'); if (found == std::string::npos) { return full_name; diff --git a/be/src/exec/schema_scanner/frontend_helper.h b/be/src/exec/schema_scanner/schema_helper.h similarity index 69% rename from be/src/exec/schema_scanner/frontend_helper.h rename to be/src/exec/schema_scanner/schema_helper.h index b13e92f26b4054..5b261f174718a5 100644 --- a/be/src/exec/schema_scanner/frontend_helper.h +++ b/be/src/exec/schema_scanner/schema_helper.h @@ -15,42 +15,40 @@ // specific language governing permissions and limitations // under the License. -#ifndef DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_FRONTEND_HELPER_H -#define DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_FRONTEND_HELPER_H +#pragma once #include "common/status.h" #include "gen_cpp/FrontendService_types.h" namespace doris { -class ExecEnv; -class FrontendServiceClient; -template class ClientConnection; - -// this class is a helper for jni call. easy for unit test -class FrontendHelper { +// this class is a helper for getting schema info from FE +class SchemaHelper { public: - static void setup(ExecEnv* exec_env); static Status get_db_names( const std::string& ip, const int32_t port, const TGetDbsParams &db_params, TGetDbsResult *db_result); + static Status get_table_names( const std::string& ip, const int32_t port, const TGetTablesParams &table_params, TGetTablesResult *table_result); + static Status list_table_status( const std::string& ip, const int32_t port, const TGetTablesParams &table_params, TListTableStatusResult *table_result); + static Status describe_table( const std::string& ip, const int32_t port, const TDescribeTableParams &desc_params, TDescribeTableResult *desc_result); + static Status show_varialbes( const std::string& ip, const int32_t port, @@ -58,27 +56,7 @@ class FrontendHelper { TShowVariableResult *var_result); static std::string extract_db_name(const std::string& full_name); - - // for default timeout - static Status rpc( - const std::string& ip, - const int32_t port, - std::function&)> callback) { - - return rpc(ip, port, callback, config::thrift_rpc_timeout_ms); - } - - static Status rpc( - const std::string& ip, - const int32_t port, - std::function&)> callback, - int timeout_ms); - -private: - static ExecEnv* _s_exec_env; }; } -#endif - diff --git a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp index ec30fe0120cbce..f1794e50587fd8 100644 --- a/be/src/exec/schema_scanner/schema_schemata_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_schemata_scanner.cpp @@ -18,7 +18,7 @@ #include "exec/schema_scanner/schema_schemata_scanner.h" #include "runtime/primitive_type.h" #include "runtime/string_value.h" -#include "exec/schema_scanner/frontend_helper.h" +#include "exec/schema_scanner/schema_helper.h" namespace doris { @@ -54,7 +54,7 @@ Status SchemaSchemataScanner::start(RuntimeState *state) { db_params.__set_user_ip(*(_param->user_ip)); } if (NULL != _param->ip && 0 != _param->port) { - RETURN_IF_ERROR(FrontendHelper::get_db_names(*(_param->ip), + RETURN_IF_ERROR(SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); } else { return Status("IP or port dosn't exists"); @@ -75,7 +75,7 @@ Status SchemaSchemataScanner::fill_one_row(Tuple *tuple, MemPool *pool) { { void *slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset()); StringValue* str_slot = reinterpret_cast(slot); - std::string db_name = FrontendHelper::extract_db_name(_db_result.dbs[_db_index]); + std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index]); str_slot->ptr = (char *)pool->allocate(db_name.size()); str_slot->len = db_name.size(); memcpy(str_slot->ptr, db_name.c_str(), str_slot->len); diff --git a/be/src/exec/schema_scanner/schema_tables_scanner.cpp b/be/src/exec/schema_scanner/schema_tables_scanner.cpp index 0f773817e6ba58..9afbf11cb7a9b4 100644 --- a/be/src/exec/schema_scanner/schema_tables_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_tables_scanner.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "exec/schema_scanner/frontend_helper.h" +#include "exec/schema_scanner/schema_helper.h" #include "exec/schema_scanner/schema_tables_scanner.h" #include "runtime/primitive_type.h" #include "runtime/string_value.h" @@ -75,7 +75,7 @@ Status SchemaTablesScanner::start(RuntimeState *state) { } if (NULL != _param->ip && 0 != _param->port) { - RETURN_IF_ERROR(FrontendHelper::get_db_names(*(_param->ip), + RETURN_IF_ERROR(SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result)); } else { return Status("IP or port dosn't exists"); @@ -95,7 +95,7 @@ Status SchemaTablesScanner::fill_one_row(Tuple *tuple, MemPool *pool) { { void *slot = tuple->get_slot(_tuple_desc->slots()[1]->tuple_offset()); StringValue* str_slot = reinterpret_cast(slot); - std::string db_name = FrontendHelper::extract_db_name(_db_result.dbs[_db_index - 1]); + std::string db_name = SchemaHelper::extract_db_name(_db_result.dbs[_db_index - 1]); str_slot->ptr = (char *)pool->allocate(db_name.size()); str_slot->len = db_name.size(); memcpy(str_slot->ptr, db_name.c_str(), str_slot->len); @@ -232,7 +232,7 @@ Status SchemaTablesScanner::get_new_table() { } if (NULL != _param->ip && 0 != _param->port) { - RETURN_IF_ERROR(FrontendHelper::list_table_status(*(_param->ip), + RETURN_IF_ERROR(SchemaHelper::list_table_status(*(_param->ip), _param->port, table_params, &_table_result)); } else { return Status("IP or port dosn't exists"); diff --git a/be/src/exec/schema_scanner/schema_variables_scanner.cpp b/be/src/exec/schema_scanner/schema_variables_scanner.cpp index 53b0170c09eb6a..8604b712a7d65b 100644 --- a/be/src/exec/schema_scanner/schema_variables_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_variables_scanner.cpp @@ -19,7 +19,7 @@ #include "runtime/primitive_type.h" #include "runtime/string_value.h" #include "runtime/runtime_state.h" -#include "exec/schema_scanner/frontend_helper.h" +#include "exec/schema_scanner/schema_helper.h" namespace doris { @@ -53,7 +53,7 @@ Status SchemaVariablesScanner::start(RuntimeState *state) { var_params.__set_threadId(_param->thread_id); if (NULL != _param->ip && 0 != _param->port) { - RETURN_IF_ERROR(FrontendHelper::show_varialbes(*(_param->ip), + RETURN_IF_ERROR(SchemaHelper::show_varialbes(*(_param->ip), _param->port, var_params, &_var_result)); } else { return Status("IP or port dosn't exists"); diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp index 439885f66b5ef9..9ca09ca9e573eb 100644 --- a/be/src/exprs/expr.cpp +++ b/be/src/exprs/expr.cpp @@ -700,6 +700,13 @@ TExprNodeType::type Expr::type_without_cast(const Expr* expr) { return expr->_node_type; } +const Expr* Expr::expr_without_cast(const Expr* expr) { + if (expr->_opcode == TExprOpcode::CAST) { + return expr_without_cast(expr->_children[0]); + } + return expr; +} + doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) { if (!is_constant()) { return NULL; diff --git a/be/src/exprs/expr.h b/be/src/exprs/expr.h index 7d1118acdcc2c1..a32bfd3bbd4e65 100644 --- a/be/src/exprs/expr.h +++ b/be/src/exprs/expr.h @@ -181,6 +181,8 @@ class Expr { static TExprNodeType::type type_without_cast(const Expr* expr); + static const Expr* expr_without_cast(const Expr* expr); + // Returns true if expr doesn't contain slotrefs, ie, can be evaluated // with get_value(NULL). The default implementation returns true if all of // the children are constant. diff --git a/be/src/exprs/expr_context.h b/be/src/exprs/expr_context.h index cbf2b6ea991134..de57638857be3f 100644 --- a/be/src/exprs/expr_context.h +++ b/be/src/exprs/expr_context.h @@ -176,6 +176,7 @@ class ExprContext { friend class InPredicate; friend class OlapScanNode; friend class EsScanNode; + friend class EsPredicate; /// FunctionContexts for each registered expression. The FunctionContexts are created /// and owned by this ExprContext. diff --git a/be/src/exprs/in_predicate.h b/be/src/exprs/in_predicate.h index aec176730f764b..4b3c6fa5729bb2 100644 --- a/be/src/exprs/in_predicate.h +++ b/be/src/exprs/in_predicate.h @@ -55,7 +55,7 @@ class InPredicate : public Predicate { // if add to children, when List is long, copy is a expensive op. void insert(void* value); - HybirdSetBase* hybird_set() { + HybirdSetBase* hybird_set() const { return _hybird_set.get(); } diff --git a/be/src/http/CMakeLists.txt b/be/src/http/CMakeLists.txt index 38e63458994dc9..792e63406c9fce 100644 --- a/be/src/http/CMakeLists.txt +++ b/be/src/http/CMakeLists.txt @@ -29,7 +29,6 @@ add_library(Webserver STATIC http_channel.cpp http_status.cpp http_parser.cpp - message_body_sink.cpp web_page_handler.cpp monitor_action.cpp default_path_handlers.cpp diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp index d43d085ad9c5ec..bbc0e3dea57224 100644 --- a/be/src/http/action/stream_load.cpp +++ b/be/src/http/action/stream_load.cpp @@ -29,7 +29,8 @@ #include #include "common/logging.h" -#include "exec/schema_scanner/frontend_helper.h" +#include "common/utils.h" +#include "util/frontend_helper.h" #include "gen_cpp/FrontendService.h" #include "gen_cpp/FrontendService_types.h" #include "gen_cpp/HeartbeatService_types.h" @@ -44,8 +45,10 @@ #include "runtime/fragment_mgr.h" #include "runtime/load_path_mgr.h" #include "runtime/plan_fragment_executor.h" -#include "runtime/stream_load_pipe.h" -#include "runtime/load_stream_mgr.h" +#include "runtime/stream_load/stream_load_executor.h" +#include "runtime/stream_load/stream_load_pipe.h" +#include "runtime/stream_load/stream_load_context.h" +#include "runtime/stream_load/load_stream_mgr.h" #include "util/byte_buffer.h" #include "util/debug_util.h" #include "util/json_util.h" @@ -62,11 +65,7 @@ IntCounter k_streaming_load_duration_ms; static IntGauge k_streaming_load_current_processing; #ifdef BE_TEST -TLoadTxnBeginResult k_stream_load_begin_result; -TLoadTxnCommitResult k_stream_load_commit_result; -TLoadTxnRollbackResult k_stream_load_rollback_result; TStreamLoadPutResult k_stream_load_put_result; -Status k_stream_load_plan_status; #endif static TFileFormatType::type parse_format(const std::string& format_str) { @@ -85,134 +84,6 @@ static bool is_format_support_streaming(TFileFormatType::type format) { } } -// stream load context -struct StreamLoadContext { - StreamLoadContext(StreamLoadAction* action_) : action(action_), _refs(0) { - start_nanos = MonotonicNanos(); - } - - ~StreamLoadContext(); - - StreamLoadAction* action; - // id for each load - UniqueId id; - - std::string db; - std::string table; - // load label, used to identify - std::string label; - - std::string user_ip; - - HttpAuthInfo auth; - - // only used to check if we receive whole body - size_t body_bytes = 0; - size_t receive_bytes = 0; - - int64_t txn_id = -1; - - bool need_rollback = false; - // when use_streaming is true, we use stream_pipe to send source data, - // otherwise we save source data to file first, then process it. - bool use_streaming = false; - TFileFormatType::type format = TFileFormatType::FORMAT_CSV_PLAIN; - - std::shared_ptr body_sink; - - TStreamLoadPutResult put_result; - double max_filter_ratio = 0.0; - std::vector commit_infos; - - std::promise promise; - std::future future = promise.get_future(); - - Status status; - - int64_t number_loaded_rows = 0; - int64_t number_filtered_rows = 0; - int64_t start_nanos = 0; - int64_t load_cost_nanos = 0; - std::string error_url; - - std::string to_json() const; - - std::string brief() const; - - void ref() { _refs.fetch_add(1); } - // If unref() returns true, this object should be delete - bool unref() { return _refs.fetch_sub(1) == 1; } - -private: - std::atomic _refs; -}; - -StreamLoadContext::~StreamLoadContext() { - if (need_rollback) { - action->rollback(this); - need_rollback = false; - } -} - -std::string StreamLoadContext::to_json() const { - rapidjson::StringBuffer s; - rapidjson::PrettyWriter writer(s); - - writer.StartObject(); - // txn id - writer.Key("TxnId"); - writer.Int64(txn_id); - - // label - writer.Key("Label"); - writer.String(label.c_str()); - - // status - writer.Key("Status"); - switch (status.code()) { - case TStatusCode::OK: - writer.String("Success"); - break; - case TStatusCode::PUBLISH_TIMEOUT: - writer.String("Publish Timeout"); - break; - case TStatusCode::LABEL_ALREADY_EXISTS: - writer.String("Label Already Exists"); - break; - default: - writer.String("Fail"); - break; - } - // msg - writer.Key("Message"); - if (status.ok()) { - writer.String("OK"); - } else { - writer.String(status.get_error_msg().c_str()); - } - // number_load_rows - writer.Key("NumberLoadedRows"); - writer.Int64(number_loaded_rows); - writer.Key("NumberFilteredRows"); - writer.Int64(number_filtered_rows); - writer.Key("LoadBytes"); - writer.Int64(receive_bytes); - writer.Key("LoadTimeMs"); - writer.Int64(load_cost_nanos / 1000000); - if (!error_url.empty()) { - writer.Key("ErrorURL"); - writer.String(error_url.c_str()); - } - writer.EndObject(); - return s.GetString(); -} - -std::string StreamLoadContext::brief() const { - std::stringstream ss; - ss << " id=" << id << ", txn id=" << txn_id << ", label=" << label; - return ss.str(); -} - StreamLoadAction::StreamLoadAction(ExecEnv* exec_env) : _exec_env(exec_env) { DorisMetrics::metrics()->register_metric("streaming_load_requests_total", &k_streaming_load_requests_total); @@ -245,7 +116,7 @@ void StreamLoadAction::handle(HttpRequest* req) { if (!ctx->status.ok()) { if (ctx->need_rollback) { - rollback(ctx); + _exec_env->stream_load_executor()->rollback_txn(ctx); ctx->need_rollback = false; } if (ctx->body_sink.get() != nullptr) { @@ -272,10 +143,10 @@ Status StreamLoadAction::_handle(StreamLoadContext* ctx) { } if (!ctx->use_streaming) { // if we use non-streaming, we need to close file first, - // then _execute_plan_fragment here + // then execute_plan_fragment here // this will close file ctx->body_sink.reset(); - RETURN_IF_ERROR(_execute_plan_fragment(ctx)); + RETURN_IF_ERROR(_exec_env->stream_load_executor()->execute_plan_fragment(ctx)); } else { RETURN_IF_ERROR(ctx->body_sink->finish()); } @@ -284,36 +155,7 @@ Status StreamLoadAction::_handle(StreamLoadContext* ctx) { RETURN_IF_ERROR(ctx->future.get()); // If put file succeess we need commit this load - TLoadTxnCommitRequest request; - set_http_auth(&request, ctx->auth); - request.db = ctx->db; - request.tbl = ctx->table; - request.txnId = ctx->txn_id; - request.sync = true; - request.commitInfos = std::move(ctx->commit_infos); - request.__isset.commitInfos = true; - - TNetworkAddress master_addr = _exec_env->master_info()->network_address; - TLoadTxnCommitResult result; -#ifndef BE_TEST - RETURN_IF_ERROR(FrontendHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result] (FrontendServiceConnection& client) { - client->loadTxnCommit(result, request); - }, config::txn_commit_rpc_timeout_ms)); -#else - result = k_stream_load_commit_result; -#endif - // Return if this transaction is committed successful; otherwise, we need try to - // rollback this transaction - Status status(result.status); - if (!status.ok()) { - LOG(WARNING) << "commit transaction failed, id=" << ctx->id - << ", errmsg=" << status.get_error_msg(); - return status; - } - // commit success, set need_rollback to false - ctx->need_rollback = false; + RETURN_IF_ERROR(_exec_env->stream_load_executor()->commit_txn(ctx)); return Status::OK; } @@ -321,9 +163,12 @@ Status StreamLoadAction::_handle(StreamLoadContext* ctx) { int StreamLoadAction::on_header(HttpRequest* req) { k_streaming_load_current_processing.increment(1); - StreamLoadContext* ctx = new StreamLoadContext(this); + StreamLoadContext* ctx = new StreamLoadContext(_exec_env); ctx->ref(); req->set_handler_ctx(ctx); + + ctx->load_type = TLoadType::MANUL_LOAD; + ctx->load_src_type = TLoadSourceType::RAW; ctx->db = req->param(HTTP_DB_KEY); ctx->table = req->param(HTTP_TABLE_KEY); @@ -339,7 +184,7 @@ int StreamLoadAction::on_header(HttpRequest* req) { if (!st.ok()) { ctx->status = st; if (ctx->need_rollback) { - rollback(ctx); + _exec_env->stream_load_executor()->rollback_txn(ctx); ctx->need_rollback = false; } if (ctx->body_sink.get() != nullptr) { @@ -394,34 +239,7 @@ Status StreamLoadAction::_on_header(HttpRequest* http_req, StreamLoadContext* ct TNetworkAddress master_addr = _exec_env->master_info()->network_address; // begin transaction - { - TLoadTxnBeginRequest request; - set_http_auth(&request, ctx->auth); - request.db = ctx->db; - request.tbl = ctx->table; - request.label = ctx->label; - // set timestamp - request.__set_timestamp(GetCurrentTimeMicros()); - - TLoadTxnBeginResult result; -#ifndef BE_TEST - RETURN_IF_ERROR(FrontendHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result] (FrontendServiceConnection& client) { - client->loadTxnBegin(result, request); - })); -#else - result = k_stream_load_begin_result; -#endif - Status status(result.status); - if (!status.ok()) { - LOG(WARNING) << "begin transaction failed, errmsg=" << status.get_error_msg() - << ctx->brief(); - return status; - } - ctx->txn_id = result.txnId; - ctx->need_rollback = true; - } + RETURN_IF_ERROR(_exec_env->stream_load_executor()->begin_txn(ctx)); // process put file return _process_put(http_req, ctx); @@ -453,7 +271,7 @@ void StreamLoadAction::on_chunk_data(HttpRequest* req) { } void StreamLoadAction::free_handler_ctx(void* param) { - StreamLoadContext* ctx = (StreamLoadContext*)param; + StreamLoadContext* ctx = (StreamLoadContext*) param; if (ctx == nullptr) { return; } @@ -472,7 +290,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext* // put request TStreamLoadPutRequest request; - set_http_auth(&request, ctx->auth); + set_request_auth(&request, ctx->auth); request.db = ctx->db; request.tbl = ctx->table; request.txnId = ctx->txn_id; @@ -531,79 +349,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext* if (!ctx->use_streaming) { return Status::OK; } - return _execute_plan_fragment(ctx); -} - -Status StreamLoadAction::_execute_plan_fragment(StreamLoadContext* ctx) { - // submit this params -#ifndef BE_TEST - ctx->ref(); - auto st = _exec_env->fragment_mgr()->exec_plan_fragment( - ctx->put_result.params, - [ctx] (PlanFragmentExecutor* executor) { - ctx->commit_infos = std::move(executor->runtime_state()->tablet_commit_infos()); - Status status = executor->status(); - if (status.ok()) { - ctx->number_loaded_rows = executor->runtime_state()->num_rows_load_success(); - ctx->number_filtered_rows = executor->runtime_state()->num_rows_load_filtered(); - int64_t num_total_rows = - ctx->number_loaded_rows + ctx->number_filtered_rows; - if ((0.0 + ctx->number_filtered_rows) / num_total_rows > ctx->max_filter_ratio) { - status = Status("too many filtered rows"); - } - if (ctx->number_filtered_rows > 0 && - !executor->runtime_state()->get_error_log_file_path().empty()) { - ctx->error_url = to_load_error_http_path( - executor->runtime_state()->get_error_log_file_path()); - } - } else { - LOG(WARNING) << "fragment execute failed" - << ", query_id=" << UniqueId(ctx->put_result.params.params.query_id) - << ", errmsg=" << status.get_error_msg() - << ctx->brief(); - // cancel body_sink, make sender known it - if (ctx->body_sink != nullptr) { - ctx->body_sink->cancel(); - } - } - ctx->promise.set_value(status); - if (ctx->unref()) { - delete ctx; - } - }); - if (!st.ok()) { - // no need to check unref's return value - ctx->unref(); - return st; - } -#else - ctx->promise.set_value(k_stream_load_plan_status); -#endif - return Status::OK; -} - -void StreamLoadAction::rollback(StreamLoadContext* ctx) { - TNetworkAddress master_addr = _exec_env->master_info()->network_address; - TLoadTxnRollbackRequest request; - set_http_auth(&request, ctx->auth); - request.db = ctx->db; - request.tbl = ctx->table; - request.txnId = ctx->txn_id; - request.__set_reason(ctx->status.get_error_msg()); - TLoadTxnRollbackResult result; -#ifndef BE_TEST - auto rpc_st = FrontendHelper::rpc( - master_addr.hostname, master_addr.port, - [&request, &result] (FrontendServiceConnection& client) { - client->loadTxnRollback(result, request); - }); - if (!rpc_st.ok()) { - LOG(WARNING) << "transaction rollback failed. errmsg=" << rpc_st.get_error_msg() - << ctx->brief(); - } -#else - result = k_stream_load_rollback_result; -#endif + return _exec_env->stream_load_executor()->execute_plan_fragment(ctx); } Status StreamLoadAction::_data_saved_path(HttpRequest* req, std::string* file_path) { diff --git a/be/src/http/action/stream_load.h b/be/src/http/action/stream_load.h index b82a4c2d16594b..34ee2b6498a027 100644 --- a/be/src/http/action/stream_load.h +++ b/be/src/http/action/stream_load.h @@ -19,19 +19,15 @@ #include -#include "common/status.h" #include "gen_cpp/PlanNodes_types.h" #include "http/http_handler.h" -#include "http/message_body_sink.h" #include "runtime/client_cache.h" +#include "runtime/message_body_sink.h" namespace doris { class ExecEnv; -class TStreamLoadPutRequest; -class TStreamLoadPutResult; -class HttpAuthInfo; -class TTabletCommitInfo; +class Status; class StreamLoadContext; class StreamLoadAction : public HttpHandler { @@ -48,9 +44,6 @@ class StreamLoadAction : public HttpHandler { void on_chunk_data(HttpRequest* req) override; void free_handler_ctx(void* ctx) override; - // called by deconstructor - void rollback(StreamLoadContext* ctx); - private: Status _on_header(HttpRequest* http_req, StreamLoadContext* ctx); Status _handle(StreamLoadContext* ctx); diff --git a/be/src/http/http_client.cpp b/be/src/http/http_client.cpp index 88c4374fe6f618..f56592125d427e 100644 --- a/be/src/http/http_client.cpp +++ b/be/src/http/http_client.cpp @@ -145,18 +145,24 @@ size_t HttpClient::on_response_data(const void* data, size_t length) { // return execute(callback); // } -Status HttpClient::execute_post_request(const std::string& post_data, std::string* response) { +Status HttpClient::execute_post_request(const std::string& payload, std::string* response) { set_method(POST); - set_post_body(post_data); + set_payload(payload); return execute(response); } +Status HttpClient::execute_delete_request(const std::string& payload, std::string* response) { + set_method(DELETE); + set_payload(payload); + return execute(response); +} + Status HttpClient::execute(const std::function& callback) { _callback = &callback; auto code = curl_easy_perform(_curl); if (code != CURLE_OK) { LOG(WARNING) << "fail to execute HTTP client, errmsg=" << _to_errmsg(code); - return Status("fail to execute HTTP client"); + return Status(_to_errmsg(code)); } return Status::OK; } diff --git a/be/src/http/http_client.h b/be/src/http/http_client.h index 83a27b8d63646f..d54bc680d33aaf 100644 --- a/be/src/http/http_client.h +++ b/be/src/http/http_client.h @@ -61,8 +61,7 @@ class HttpClient { curl_easy_setopt(_curl, CURLOPT_HTTPHEADER, _header_list); } - // you must set CURLOPT_POSTFIELDSIZE before CURLOPT_COPYPOSTFIELDS options, otherwise will cause request hanging up - void set_post_body(const std::string& post_body) { + void set_payload(const std::string& post_body) { curl_easy_setopt(_curl, CURLOPT_POSTFIELDSIZE, (long)post_body.length()); curl_easy_setopt(_curl, CURLOPT_COPYPOSTFIELDS, post_body.c_str()); } @@ -114,7 +113,9 @@ class HttpClient { // a file to local_path Status download(const std::string& local_path); - Status execute_post_request(const std::string& post_data, std::string* response); + Status execute_post_request(const std::string& payload, std::string* response); + + Status execute_delete_request(const std::string& payload, std::string* response); // execute a simple method, and its response is saved in response argument Status execute(std::string* response); diff --git a/be/src/http/http_common.h b/be/src/http/http_common.h index 05bc5d878d93d1..c3d34681f60156 100644 --- a/be/src/http/http_common.h +++ b/be/src/http/http_common.h @@ -21,13 +21,6 @@ namespace doris { -struct HttpAuthInfo { - std::string user; - std::string passwd; - std::string cluster; - std::string user_ip; -}; - static const std::string HTTP_DB_KEY = "db"; static const std::string HTTP_TABLE_KEY = "table"; static const std::string HTTP_LABEL_KEY = "label"; diff --git a/be/src/http/utils.cpp b/be/src/http/utils.cpp index 2e8be225cf9ad4..7953a7cda10c92 100644 --- a/be/src/http/utils.cpp +++ b/be/src/http/utils.cpp @@ -18,6 +18,7 @@ #include #include "common/logging.h" +#include "common/utils.h" #include "http/http_common.h" #include "http/http_headers.h" #include "http/http_request.h" @@ -54,7 +55,7 @@ bool parse_basic_auth(const HttpRequest& req, std::string* user, std::string* pa return true; } -bool parse_basic_auth(const HttpRequest& req, HttpAuthInfo* auth) { +bool parse_basic_auth(const HttpRequest& req, AuthInfo* auth) { std::string full_user; if (!parse_basic_auth(req, &full_user, &auth->passwd)) { return false; diff --git a/be/src/http/utils.h b/be/src/http/utils.h index b388e6af1e5f5a..8e82d7bed58aa9 100644 --- a/be/src/http/utils.h +++ b/be/src/http/utils.h @@ -19,12 +19,12 @@ #include +#include "common/utils.h" #include "http/http_common.h" namespace doris { class HttpRequest; -class HttpAuthInfo; std::string encode_basic_auth(const std::string& user, const std::string& passwd); // parse Basic authorization @@ -32,16 +32,6 @@ std::string encode_basic_auth(const std::string& user, const std::string& passwd // Otherwise return fasle bool parse_basic_auth(const HttpRequest& req, std::string* user, std::string* passwd); -bool parse_basic_auth(const HttpRequest& req, HttpAuthInfo* auth); - -template -void set_http_auth(T* req, const HttpAuthInfo& auth) { - req->user = auth.user; - req->passwd = auth.passwd; - if (!auth.cluster.empty()) { - req->__set_cluster(auth.cluster); - } - req->__set_user_ip(auth.user_ip); -} +bool parse_basic_auth(const HttpRequest& req, AuthInfo* auth); } diff --git a/be/src/olap/column_reader.h b/be/src/olap/column_reader.h index 566dde1dc6ee0e..106976342e1d74 100644 --- a/be/src/olap/column_reader.h +++ b/be/src/olap/column_reader.h @@ -347,7 +347,8 @@ class DefaultValueReader : public ColumnReader { } break; } - case OLAP_FIELD_TYPE_VARCHAR: { + case OLAP_FIELD_TYPE_VARCHAR: + case OLAP_FIELD_TYPE_HLL: { _values = reinterpret_cast(mem_pool->allocate(size * sizeof(Slice))); int32_t length = _default_value.length(); diff --git a/be/src/olap/olap_header.cpp b/be/src/olap/olap_header.cpp index 33ed89e8e117a0..71aa38b14de264 100644 --- a/be/src/olap/olap_header.cpp +++ b/be/src/olap/olap_header.cpp @@ -338,7 +338,7 @@ OLAPStatus OLAPHeader::add_pending_segment_group( LOG(WARNING) << "pending segment_group already exists in header." << "transaction_id:" << transaction_id << ", pending_segment_group_id: " << pending_segment_group_id; - return OLAP_ERR_HEADER_ADD_PENDING_DELTA; + return OLAP_SUCCESS; } } } diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 365f1fb27ae2a0..aacf8dd6fec265 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -35,7 +35,6 @@ #include "common/resource_tls.h" #include "agent/cgroups_mgr.h" - using std::deque; using std::list; using std::nothrow; @@ -256,8 +255,21 @@ bool RowBlockChanger::change_row_block( } write_helper.set_not_null(i); - char* buf = field_to_read->get_ptr(read_helper.get_buf()); - write_helper.set_field_content(i, buf, mem_pool); + if (mutable_block->tablet_schema()[i].type == OLAP_FIELD_TYPE_CHAR) { + // if modify length of CHAR type, the size of slice should be equal + // to new length. + Slice* src = (Slice*)(field_to_read->get_ptr(read_helper.get_buf())); + size_t size = mutable_block->tablet_schema()[i].length; + char* buf = reinterpret_cast(mem_pool->allocate(size)); + memset(buf, 0, size); + size_t copy_size = (size < src->size) ? size : src->size; + memcpy(buf, src->data, copy_size); + Slice dst(buf, size); + write_helper.set_field_content(i, reinterpret_cast(&dst), mem_pool); + } else { + char* src = field_to_read->get_ptr(read_helper.get_buf()); + write_helper.set_field_content(i, src, mem_pool); + } } } diff --git a/be/src/olap/wrapper_field.cpp b/be/src/olap/wrapper_field.cpp index 02ab4363dd82d5..6df628dc17a637 100644 --- a/be/src/olap/wrapper_field.cpp +++ b/be/src/olap/wrapper_field.cpp @@ -21,7 +21,9 @@ namespace doris { WrapperField* WrapperField::create(const FieldInfo& info, uint32_t len) { bool is_string_type = - (info.type == OLAP_FIELD_TYPE_CHAR || info.type == OLAP_FIELD_TYPE_VARCHAR); + (info.type == OLAP_FIELD_TYPE_CHAR + || info.type == OLAP_FIELD_TYPE_VARCHAR + || info.type == OLAP_FIELD_TYPE_HLL); if (is_string_type && len > OLAP_STRING_MAX_LENGTH) { OLAP_LOG_WARNING("length of string parameter is too long[len=%lu, max_len=%lu].", len, OLAP_STRING_MAX_LENGTH); @@ -36,7 +38,7 @@ WrapperField* WrapperField::create(const FieldInfo& info, uint32_t len) { size_t variable_len = 0; if (info.type == OLAP_FIELD_TYPE_CHAR) { variable_len = std::max(len, info.length); - } else if (info.type == OLAP_FIELD_TYPE_VARCHAR) { + } else if (info.type == OLAP_FIELD_TYPE_VARCHAR || info.type == OLAP_FIELD_TYPE_HLL) { variable_len = std::max(len, static_cast(info.length - sizeof(StringLengthType))); } else { @@ -52,7 +54,9 @@ WrapperField* WrapperField::create_by_type(const FieldType& type) { if (rep == nullptr) { return nullptr; } - bool is_string_type = (type == OLAP_FIELD_TYPE_CHAR || type == OLAP_FIELD_TYPE_VARCHAR); + bool is_string_type = (type == OLAP_FIELD_TYPE_CHAR + || type == OLAP_FIELD_TYPE_VARCHAR + || type == OLAP_FIELD_TYPE_HLL); WrapperField* wrapper = new WrapperField(rep, 0, is_string_type); return wrapper; } diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index 6f820b9867c32b..7f19dcd0c7725b 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -93,8 +93,14 @@ add_library(Runtime STATIC bufferpool/system_allocator.cc initial_reservations.cc snapshot_loader.cpp - kafka_consumer_pipe.cpp query_statistics.cpp + message_body_sink.cpp + stream_load/stream_load_context.cpp + stream_load/stream_load_executor.cpp + routine_load/data_consumer.cpp + routine_load/data_consumer_group.cpp + routine_load/data_consumer_pool.cpp + routine_load/routine_load_task_executor.cpp ) # This test runs forever so should not be part of 'make test' diff --git a/be/src/runtime/data_spliter.cpp b/be/src/runtime/data_spliter.cpp index 7edd7411dfd7f7..de6a26a8f33606 100644 --- a/be/src/runtime/data_spliter.cpp +++ b/be/src/runtime/data_spliter.cpp @@ -170,6 +170,8 @@ Status DataSpliter::process_partition( std::stringstream error_log; error_log << "there is no corresponding partition for this key: "; ctx->print_value(row, &error_log); + state->update_num_rows_load_filtered(1); + state->update_num_rows_load_success(-1); return Status(error_log.str(), true); } *info = _partition_infos[*part_index]; diff --git a/be/src/runtime/data_stream_mgr.cpp b/be/src/runtime/data_stream_mgr.cpp index 0644964dec9b8d..5f171716e6e99e 100644 --- a/be/src/runtime/data_stream_mgr.cpp +++ b/be/src/runtime/data_stream_mgr.cpp @@ -111,16 +111,19 @@ Status DataStreamMgr::transmit_data(const PTransmitDataParams* request, ::google return Status::OK; } + // request can only be used before calling recvr's add_batch or when request + // is the last for the sender, because request maybe released after it's batch + // is consumed by ExchangeNode. + if (request->has_query_statistics()) { + recvr->add_sub_plan_statistics(request->query_statistics(), request->sender_id()); + } + bool eos = request->eos(); if (request->has_row_batch()) { recvr->add_batch(request->row_batch(), request->sender_id(), request->be_number(), request->packet_seq(), eos ? nullptr : done); } - if (request->has_query_statistics()) { - recvr->add_sub_plan_statistics(request->query_statistics(), request->sender_id()); - } - if (eos) { recvr->remove_sender(request->sender_id(), request->be_number()); } diff --git a/be/src/runtime/decimal_value.cpp b/be/src/runtime/decimal_value.cpp index 564a67f1a01b69..8e6522d5bed3a9 100755 --- a/be/src/runtime/decimal_value.cpp +++ b/be/src/runtime/decimal_value.cpp @@ -84,16 +84,20 @@ inline void sub(const int32_t value1, const int32_t value2, int32_t* to, int32_t // Note: the input carry may > 1, after the summation process of three number (value1, value2, *carry), // the maximum value of carry may be 2, when sum() >= 2 * DIG_BASE. inline void add2(const int32_t value1, const int32_t value2, int32_t* to, int32_t* carry) { - int32_t sum = value1 + value2 + *carry; + // NOTE: When three int32_t integers (the maximum value of each number is 10 ^ 9 - 1) are added, + // because the maximum value of int32_t is 2147483647, the result may overflow, so it is + // necessary to convert int32_t to int64_t. + int64_t sum = (int64_t) value1 + value2 + *carry; *carry = (sum >= DIG_BASE) ? 1 : 0; if (*carry) { sum -= DIG_BASE; } - if (sum > DIG_BASE) { + if (sum >= DIG_BASE) { sum -= DIG_BASE; ++(*carry); } - *to = sum; + // the value of sum must small than DIG_BASE here + *to = (int32_t) sum; } // to = value1 - value2 ƒ diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index 15219e09ec2944..e88cfb44783209 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -124,7 +124,7 @@ class SlotDescriptor { return _slot_size; } - std::string col_name() const { + const std::string& col_name() const { return _col_name; } diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index fb07a7c792eef5..5d7b09f75d497a 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -50,6 +50,8 @@ class ThreadPool; class ThreadResourceMgr; class TmpFileMgr; class WebPageHandler; +class StreamLoadExecutor; +class RoutineLoadTaskExecutor; class BackendServiceClient; class FrontendServiceClient; @@ -110,11 +112,15 @@ class ExecEnv { BufferPool* buffer_pool() { return _buffer_pool; } TabletWriterMgr* tablet_writer_mgr() { return _tablet_writer_mgr; } LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr; } + const std::vector& store_paths() const { return _store_paths; } void set_store_paths(const std::vector& paths) { _store_paths = paths; } OLAPEngine* olap_engine() { return _olap_engine; } void set_olap_engine(OLAPEngine* olap_engine) { _olap_engine = olap_engine; } + StreamLoadExecutor* stream_load_executor() { return _stream_load_executor; } + RoutineLoadTaskExecutor* routine_load_task_executor() { return _routine_load_task_executor; } + private: Status _init(const std::vector& store_paths); void _destory(); @@ -158,6 +164,9 @@ class ExecEnv { BufferPool* _buffer_pool = nullptr; OLAPEngine* _olap_engine = nullptr; + + StreamLoadExecutor* _stream_load_executor = nullptr; + RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr; }; } diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 6a04f0b0b5db17..b20657b942cdea 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -43,8 +43,10 @@ #include "util/bfd_parser.h" #include "runtime/etl_job_mgr.h" #include "runtime/load_path_mgr.h" -#include "runtime/load_stream_mgr.h" #include "runtime/pull_load_task_mgr.h" +#include "runtime/routine_load/routine_load_task_executor.h" +#include "runtime/stream_load/load_stream_mgr.h" +#include "runtime/stream_load/stream_load_executor.h" #include "util/pretty_printer.h" #include "util/doris_metrics.h" #include "util/brpc_stub_cache.h" @@ -95,6 +97,8 @@ Status ExecEnv::_init(const std::vector& store_paths) { _tablet_writer_mgr = new TabletWriterMgr(this); _load_stream_mgr = new LoadStreamMgr(); _brpc_stub_cache = new BrpcStubCache(); + _stream_load_executor = new StreamLoadExecutor(this); + _routine_load_task_executor = new RoutineLoadTaskExecutor(this); _client_cache->init_metrics(DorisMetrics::metrics(), "backend"); _frontend_client_cache->init_metrics(DorisMetrics::metrics(), "frontend"); @@ -208,6 +212,9 @@ void ExecEnv::_destory() { delete _client_cache; delete _result_mgr; delete _stream_mgr; + delete _stream_load_executor; + delete _routine_load_task_executor; + _metrics = nullptr; } diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 78358ea3c8a52c..d443cabc6e57e9 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -44,6 +44,9 @@ namespace doris { std::string to_load_error_http_path(const std::string& file_name) { + if (file_name.empty()) { + return ""; + } std::stringstream url; url << "http://" << BackendOptions::get_localhost() << ":" << config::webserver_port << "/api/_load_error_log?" diff --git a/be/src/runtime/large_int_value.h b/be/src/runtime/large_int_value.h index 4ced5bb0d43242..4b7d4f1a6df21e 100644 --- a/be/src/runtime/large_int_value.h +++ b/be/src/runtime/large_int_value.h @@ -52,6 +52,13 @@ class LargeIntValue { *len = (buffer + *len) - d; return d; } + + static std::string to_string(__int128 value) { + char buf[64] = {0}; + int len = 64; + char *str = to_string(value, buf, &len); + return std::string(str, len); + } }; std::ostream& operator<<(std::ostream& os, __int128 const& value); diff --git a/be/src/http/message_body_sink.cpp b/be/src/runtime/message_body_sink.cpp similarity index 95% rename from be/src/http/message_body_sink.cpp rename to be/src/runtime/message_body_sink.cpp index bb831b819b5e99..93fa788bca04f1 100644 --- a/be/src/http/message_body_sink.cpp +++ b/be/src/runtime/message_body_sink.cpp @@ -15,16 +15,13 @@ // specific language governing permissions and limitations // under the License. -#include "http/message_body_sink.h" +#include "runtime/message_body_sink.h" #include #include #include #include -#include "http/http_channel.h" -#include "http/http_parser.h" - #include "util/runtime_profile.h" namespace doris { diff --git a/be/src/http/message_body_sink.h b/be/src/runtime/message_body_sink.h similarity index 94% rename from be/src/http/message_body_sink.h rename to be/src/runtime/message_body_sink.h index 25d082c1822d12..b8ebe7f01f3bce 100644 --- a/be/src/http/message_body_sink.h +++ b/be/src/runtime/message_body_sink.h @@ -23,9 +23,6 @@ namespace doris { -class HttpChannel; -class BodySink; - class MessageBodySink { public: virtual ~MessageBodySink() { } @@ -41,7 +38,7 @@ class MessageBodySink { virtual void cancel() { } }; -// write HTTP request's message-body to a local file +// write message to a local file class MessageBodyFileSink : public MessageBodySink { public: MessageBodyFileSink(const std::string& path) : _path(path) { } diff --git a/be/src/runtime/routine_load/data_consumer.cpp b/be/src/runtime/routine_load/data_consumer.cpp new file mode 100644 index 00000000000000..ae6007e884b9aa --- /dev/null +++ b/be/src/runtime/routine_load/data_consumer.cpp @@ -0,0 +1,225 @@ +// 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 "runtime/routine_load/data_consumer.h" + +#include +#include +#include +#include + +#include "common/status.h" +#include "service/backend_options.h" +#include "util/defer_op.h" +#include "util/stopwatch.hpp" +#include "util/uid_util.h" + +namespace doris { + +// init kafka consumer will only set common configs such as +// brokers, groupid +Status KafkaDataConsumer::init(StreamLoadContext* ctx) { + std::unique_lock l(_lock); + if (_init) { + // this consumer has already been initialized. + return Status::OK; + } + + RdKafka::Conf *conf = RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL); + + // conf has to be deleted finally + auto conf_deleter = [conf] () { delete conf; }; + DeferOp delete_conf(std::bind(conf_deleter)); + + std::stringstream ss; + ss << BackendOptions::get_localhost() << "_"; + std::string group_id = ss.str() + UniqueId().to_string(); + LOG(INFO) << "init kafka consumer with group id: " << group_id; + + std::string errstr; + auto set_conf = [&conf, &errstr](const std::string& conf_key, const std::string& conf_val) { + if (conf->set(conf_key, conf_val, errstr) != RdKafka::Conf::CONF_OK) { + std::stringstream ss; + ss << "failed to set '" << conf_key << "'"; + LOG(WARNING) << ss.str(); + return Status(ss.str()); + } + VLOG(3) << "set " << conf_key << ": " << conf_val; + return Status::OK; + }; + + RETURN_IF_ERROR(set_conf("metadata.broker.list", ctx->kafka_info->brokers)); + RETURN_IF_ERROR(set_conf("group.id", group_id)); + RETURN_IF_ERROR(set_conf("enable.partition.eof", "false")); + RETURN_IF_ERROR(set_conf("enable.auto.offset.store", "false")); + // TODO: set it larger than 0 after we set rd_kafka_conf_set_stats_cb() + RETURN_IF_ERROR(set_conf("statistics.interval.ms", "0")); + RETURN_IF_ERROR(set_conf("auto.offset.reset", "error")); + + if (conf->set("event_cb", &_k_event_cb, errstr) != RdKafka::Conf::CONF_OK) { + std::stringstream ss; + ss << "failed to set 'event_cb'"; + LOG(WARNING) << ss.str(); + return Status(ss.str()); + } + + // create consumer + _k_consumer = RdKafka::KafkaConsumer::create(conf, errstr); + if (!_k_consumer) { + LOG(WARNING) << "failed to create kafka consumer"; + return Status("failed to create kafka consumer"); + } + + VLOG(3) << "finished to init kafka consumer. " << ctx->brief(); + + _init = true; + return Status::OK; +} + +Status KafkaDataConsumer::assign_topic_partitions( + const std::map& begin_partition_offset, + const std::string& topic, + StreamLoadContext* ctx) { + + DCHECK(_k_consumer); + // create TopicPartitions + std::stringstream ss; + std::vector topic_partitions; + for (auto& entry : begin_partition_offset) { + RdKafka::TopicPartition* tp1 = RdKafka::TopicPartition::create( + topic, entry.first, entry.second); + topic_partitions.push_back(tp1); + ss << "[" << entry.first << ": " << entry.second << "] "; + } + + LOG(INFO) << "consumer: " << _id << ", grp: " << _grp_id + << " assign topic partitions: " << topic << ", " << ss.str(); + + // delete TopicPartition finally + auto tp_deleter = [&topic_partitions] () { + std::for_each(topic_partitions.begin(), topic_partitions.end(), + [](RdKafka::TopicPartition* tp1) { delete tp1; }); + }; + DeferOp delete_tp(std::bind(tp_deleter)); + + // assign partition + RdKafka::ErrorCode err = _k_consumer->assign(topic_partitions); + if (err) { + LOG(WARNING) << "failed to assign topic partitions: " << ctx->brief(true) + << ", err: " << RdKafka::err2str(err); + return Status("failed to assign topic partitions"); + } + + return Status::OK; +} + +Status KafkaDataConsumer::group_consume( + BlockingQueue* queue, + int64_t max_running_time_ms) { + _last_visit_time = time(nullptr); + int64_t left_time = max_running_time_ms; + LOG(INFO) << "start kafka consumer: " << _id << ", grp: " << _grp_id + << ", max running time(ms): " << left_time; + + int64_t received_rows = 0; + int64_t put_rows = 0; + Status st = Status::OK; + MonotonicStopWatch consumer_watch; + MonotonicStopWatch watch; + watch.start(); + while (true) { + { + std::unique_lock l(_lock); + if (_cancelled) { break; } + } + + if (left_time <= 0) { break; } + + bool done = false; + // consume 1 message at a time + consumer_watch.start(); + std::unique_ptr msg(_k_consumer->consume(1000 /* timeout, ms */)); + consumer_watch.stop(); + switch (msg->err()) { + case RdKafka::ERR_NO_ERROR: + if (!queue->blocking_put(msg.get())) { + // queue is shutdown + done = true; + } else { + ++put_rows; + msg.release(); // release the ownership, msg will be deleted after being processed + } + ++received_rows; + break; + case RdKafka::ERR__TIMED_OUT: + // leave the status as OK, because this may happend + // if there is no data in kafka. + LOG(WARNING) << "kafka consume timeout: " << _id; + break; + default: + LOG(WARNING) << "kafka consume failed: " << _id + << ", msg: " << msg->errstr(); + done = true; + st = Status(msg->errstr()); + break; + } + + left_time = max_running_time_ms - watch.elapsed_time() / 1000 / 1000; + if (done) { break; } + } + + LOG(INFO) << "kafka conumer done: " << _id << ", grp: " << _grp_id + << ". cancelled: " << _cancelled + << ", left time(ms): " << left_time + << ", total cost(ms): " << watch.elapsed_time() / 1000 / 1000 + << ", consume cost(ms): " << consumer_watch.elapsed_time() / 1000 / 1000 + << ", received rows: " << received_rows + << ", put rows: " << put_rows; + + return st; +} + +Status KafkaDataConsumer::cancel(StreamLoadContext* ctx) { + std::unique_lock l(_lock); + if (!_init) { + return Status("consumer is not initialized"); + } + + _cancelled = true; + LOG(INFO) << "kafka consumer cancelled. " << _id; + return Status::OK; +} + +Status KafkaDataConsumer::reset() { + std::unique_lock l(_lock); + _cancelled = false; + return Status::OK; +} + +// if the kafka brokers and topic are same, +// we considered this consumer as matched, thus can be reused. +bool KafkaDataConsumer::match(StreamLoadContext* ctx) { + if (ctx->load_src_type != TLoadSourceType::KAFKA) { + return false; + } + if (_brokers != ctx->kafka_info->brokers || _topic != ctx->kafka_info->topic) { + return false; + } + return true; +} + +} // end namespace doris diff --git a/be/src/runtime/routine_load/data_consumer.h b/be/src/runtime/routine_load/data_consumer.h new file mode 100644 index 00000000000000..b42d36c89110ba --- /dev/null +++ b/be/src/runtime/routine_load/data_consumer.h @@ -0,0 +1,153 @@ +// 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 "librdkafka/rdkafkacpp.h" + +#include "runtime/stream_load/stream_load_context.h" +#include "util/blocking_queue.hpp" +#include "util/uid_util.h" + +namespace doris { + +class KafkaConsumerPipe; +class Status; +class StreamLoadPipe; + +class DataConsumer { +public: + DataConsumer(StreamLoadContext* ctx): + _has_grp(false), + _init(false), + _cancelled(false), + _last_visit_time(0) { + } + + virtual ~DataConsumer() { + } + + // init the consumer with the given parameters + virtual Status init(StreamLoadContext* ctx) = 0; + // start consuming + virtual Status consume(StreamLoadContext* ctx) = 0; + // cancel the consuming process. + // if the consumer is not initialized, or the consuming + // process is already finished, call cancel() will + // return ERROR + virtual Status cancel(StreamLoadContext* ctx) = 0; + // reset the data consumer before being reused + virtual Status reset() = 0; + // return true the if the consumer match the need + virtual bool match(StreamLoadContext* ctx) = 0; + + const UniqueId& id() { return _id; } + time_t last_visit_time() { return _last_visit_time; } + void set_grp(const UniqueId& grp_id) { + _grp_id = grp_id; + _has_grp = true; + } + +protected: + UniqueId _id; + UniqueId _grp_id; + bool _has_grp; + + // lock to protect the following bools + std::mutex _lock; + bool _init; + bool _cancelled; + time_t _last_visit_time; +}; + +class KafkaEventCb : public RdKafka::EventCb { +public: + void event_cb(RdKafka::Event &event) { + switch (event.type()) { + case RdKafka::Event::EVENT_ERROR: + LOG(INFO) << "kafka error: " << RdKafka::err2str(event.err()) + << ", event: " << event.str(); + break; + case RdKafka::Event::EVENT_STATS: + LOG(INFO) << "kafka stats: " << event.str(); + break; + + case RdKafka::Event::EVENT_LOG: + LOG(INFO) << "kafka log-" << event.severity() << "-" << event.fac().c_str() + << ", event: " << event.str(); + break; + + case RdKafka::Event::EVENT_THROTTLE: + LOG(INFO) << "kafka throttled: " << event.throttle_time() << "ms by " + << event.broker_name() << " id " << (int) event.broker_id(); + break; + + default: + LOG(INFO) << "kafka event: " << event.type() + << ", err: " << RdKafka::err2str(event.err()) + << ", event: " << event.str(); + break; + } + } +}; + +class KafkaDataConsumer : public DataConsumer { +public: + KafkaDataConsumer(StreamLoadContext* ctx): + DataConsumer(ctx), + _brokers(ctx->kafka_info->brokers), + _topic(ctx->kafka_info->topic) { + } + + virtual ~KafkaDataConsumer() { + VLOG(3) << "deconstruct consumer"; + if (_k_consumer) { + _k_consumer->close(); + delete _k_consumer; + _k_consumer = nullptr; + } + } + + virtual Status init(StreamLoadContext* ctx) override; + // TODO(cmy): currently do not implement single consumer start method, using group_consume + virtual Status consume(StreamLoadContext* ctx) override { return Status::OK; } + virtual Status cancel(StreamLoadContext* ctx) override; + // reassign partition topics + virtual Status reset() override; + virtual bool match(StreamLoadContext* ctx) override; + + Status assign_topic_partitions( + const std::map& begin_partition_offset, + const std::string& topic, + StreamLoadContext* ctx); + + // start the consumer and put msgs to queue + Status group_consume(BlockingQueue* queue, int64_t max_running_time_ms); + +private: + std::string _brokers; + std::string _topic; + + KafkaEventCb _k_event_cb; + RdKafka::KafkaConsumer* _k_consumer = nullptr; + std::shared_ptr _k_consumer_pipe; +}; + +} // end namespace doris diff --git a/be/src/runtime/routine_load/data_consumer_group.cpp b/be/src/runtime/routine_load/data_consumer_group.cpp new file mode 100644 index 00000000000000..4332ce5c1db35d --- /dev/null +++ b/be/src/runtime/routine_load/data_consumer_group.cpp @@ -0,0 +1,185 @@ +// 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 "runtime/routine_load/data_consumer_group.h" +#include "runtime/routine_load/kafka_consumer_pipe.h" + +namespace doris { + +Status KafkaDataConsumerGroup::assign_topic_partitions(StreamLoadContext* ctx) { + DCHECK(ctx->kafka_info); + DCHECK(_consumers.size() >= 1); + + // divide partitions + int consumer_size = _consumers.size(); + std::vector> divide_parts(consumer_size); + int i = 0; + for (auto& kv : ctx->kafka_info->begin_offset) { + int idx = i % consumer_size; + divide_parts[idx].emplace(kv.first, kv.second); + i++; + } + + // assign partitions to consumers equally + for (int i = 0; i < consumer_size; ++i) { + RETURN_IF_ERROR(std::static_pointer_cast(_consumers[i])->assign_topic_partitions( + divide_parts[i], ctx->kafka_info->topic, ctx)); + } + + return Status::OK; +} + +KafkaDataConsumerGroup::~KafkaDataConsumerGroup() { + // clean the msgs left in queue + _queue.shutdown(); + while(true) { + RdKafka::Message* msg; + if (_queue.blocking_get(&msg)) { + delete msg; + msg = nullptr; + } else { + break; + } + } + DCHECK(_queue.get_size() == 0); +} + +Status KafkaDataConsumerGroup::start_all(StreamLoadContext* ctx) { + Status result_st = Status::OK; + // start all consumers + for(auto& consumer : _consumers) { + if (!_thread_pool.offer( + boost::bind(&KafkaDataConsumerGroup::actual_consume, this, consumer, &_queue, ctx->max_interval_s * 1000, + [this, &result_st] (const Status& st) { + std::unique_lock lock(_mutex); + _counter--; + if (_counter == 0) { + _queue.shutdown(); + LOG(INFO) << "all consumers are finished. shutdown queue. group id: " << _grp_id; + } + if (result_st.ok() && !st.ok()) { + result_st = st; + } + }))) { + + LOG(WARNING) << "failed to submit data consumer: " << consumer->id() << ", group id: " << _grp_id; + return Status("failed to submit data consumer"); + } else { + VLOG(1) << "submit a data consumer: " << consumer->id() << ", group id: " << _grp_id; + } + } + + // consuming from queue and put data to stream load pipe + int64_t left_time = ctx->max_interval_s * 1000; + int64_t left_rows = ctx->max_batch_rows; + int64_t left_bytes = ctx->max_batch_size; + + std::shared_ptr kafka_pipe = std::static_pointer_cast(ctx->body_sink); + + LOG(INFO) << "start consumer group: " << _grp_id + << ". max time(ms): " << left_time + << ", batch rows: " << left_rows + << ", batch size: " << left_bytes + << ". " << ctx->brief(); + + // copy one + std::map cmt_offset = ctx->kafka_info->cmt_offset; + + MonotonicStopWatch watch; + watch.start(); + Status st; + bool eos = false; + while (true) { + if (eos || left_time <= 0 || left_rows <= 0 || left_bytes <=0) { + LOG(INFO) << "consumer group done: " << _grp_id + << ". consume time(ms)=" << ctx->max_interval_s * 1000 - left_time + << ", received rows=" << ctx->max_batch_rows - left_rows + << ", received bytes=" << ctx->max_batch_size - left_bytes + << ", eos: " << eos + << ", blocking get time(us): " << _queue.total_get_wait_time() / 1000 + << ", blocking put time(us): " << _queue.total_put_wait_time() / 1000; + + // shutdown queue + _queue.shutdown(); + // cancel all consumers + for(auto& consumer : _consumers) { consumer->cancel(ctx); } + + if (!result_st.ok()) { + // some of consumers encounter errors, cancel this task + return result_st; + } + + if (left_bytes == ctx->max_batch_size) { + // nothing to be consumed, we have to cancel it, because + // we do not allow finishing stream load pipe without data + kafka_pipe->cancel(); + return Status::CANCELLED; + } else { + DCHECK(left_bytes < ctx->max_batch_size); + DCHECK(left_rows < ctx->max_batch_rows); + kafka_pipe->finish(); + ctx->kafka_info->cmt_offset = std::move(cmt_offset); + ctx->receive_bytes = ctx->max_batch_size - left_bytes; + return Status::OK; + } + } + + RdKafka::Message* msg; + bool res = _queue.blocking_get(&msg); + if (res) { + VLOG(3) << "get kafka message" + << ", partition: " << msg->partition() + << ", offset: " << msg->offset() + << ", len: " << msg->len(); + + st = kafka_pipe->append_with_line_delimiter( + static_cast(msg->payload()), + static_cast(msg->len())); + if (st.ok()) { + left_rows--; + left_bytes -= msg->len(); + cmt_offset[msg->partition()] = msg->offset(); + VLOG(3) << "consume partition[" << msg->partition() + << " - " << msg->offset() << "]"; + } else { + // failed to append this msg, we must stop + LOG(WARNING) << "failed to append msg to pipe"; + eos = true; + } + delete msg; + } else { + // queue is empty and shutdown + eos = true; + } + + left_time = ctx->max_interval_s * 1000 - watch.elapsed_time() / 1000 / 1000; + } + + return Status::OK; +} + +void KafkaDataConsumerGroup::actual_consume( + std::shared_ptr consumer, + BlockingQueue* queue, + int64_t max_running_time_ms, + ConsumeFinishCallback cb) { + Status st = std::static_pointer_cast(consumer)->group_consume(queue, max_running_time_ms); + cb(st); +} + +} // end namespace diff --git a/be/src/runtime/routine_load/data_consumer_group.h b/be/src/runtime/routine_load/data_consumer_group.h new file mode 100644 index 00000000000000..233d07ce8e6d32 --- /dev/null +++ b/be/src/runtime/routine_load/data_consumer_group.h @@ -0,0 +1,95 @@ +// 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 "runtime/routine_load/data_consumer.h" +#include "util/blocking_queue.hpp" +#include "util/thread_pool.hpp" + +namespace doris { + +// data consumer group saves a group of data consumers. +// These data consumers share the same stream load pipe. +// This class is not thread safe. +class DataConsumerGroup { +public: + typedef std::function ConsumeFinishCallback; + + DataConsumerGroup(): + _thread_pool(3, 10), + _counter(0) {} + + virtual ~DataConsumerGroup() { + _consumers.clear(); + } + + const UniqueId& grp_id() { return _grp_id; } + + const std::vector>& consumers() { + return _consumers; + } + + void add_consumer(std::shared_ptr consumer) { + consumer->set_grp(_grp_id); + _consumers.push_back(consumer); + ++_counter; + } + + // start all consumers + virtual Status start_all(StreamLoadContext* ctx) { return Status::OK; } + +protected: + UniqueId _grp_id; + std::vector> _consumers; + // thread pool to run each consumer in multi thread + ThreadPool _thread_pool; + // mutex to protect counter. + // the counter is init as the number of consumers. + // once a consumer is done, decrease the counter. + // when the counter becomes zero, shutdown the queue to finish + std::mutex _mutex; + int _counter; +}; + +// for kafka +class KafkaDataConsumerGroup : public DataConsumerGroup { +public: + KafkaDataConsumerGroup(): + DataConsumerGroup(), + _queue(500) {} + + virtual ~KafkaDataConsumerGroup(); + + virtual Status start_all(StreamLoadContext* ctx) override; + // assign topic partitions to all consumers equally + Status assign_topic_partitions(StreamLoadContext* ctx); + +private: + // start a single consumer + void actual_consume( + std::shared_ptr consumer, + BlockingQueue* queue, + int64_t max_running_time_ms, + ConsumeFinishCallback cb); + +private: + // blocking queue to receive msgs from all consumers + BlockingQueue _queue; +}; + +} // end namespace doris diff --git a/be/src/runtime/routine_load/data_consumer_pool.cpp b/be/src/runtime/routine_load/data_consumer_pool.cpp new file mode 100644 index 00000000000000..0267fee119ce5a --- /dev/null +++ b/be/src/runtime/routine_load/data_consumer_pool.cpp @@ -0,0 +1,149 @@ +// 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 "runtime/routine_load/data_consumer_pool.h" +#include "runtime/routine_load/data_consumer_group.h" +#include "common/config.h" + +namespace doris { + +Status DataConsumerPool::get_consumer( + StreamLoadContext* ctx, + std::shared_ptr* ret) { + + std::unique_lock l(_lock); + + // check if there is an available consumer. + // if has, return it, also remove it from the pool + auto iter = std::begin(_pool); + while (iter != std::end(_pool)) { + if ((*iter)->match(ctx)) { + VLOG(3) << "get an available data consumer from pool: " << (*iter)->id(); + (*iter)->reset(); + *ret = *iter; + iter = _pool.erase(iter); + return Status::OK; + } else { + ++iter; + } + } + + // no available consumer, create a new one + std::shared_ptr consumer; + switch (ctx->load_src_type) { + case TLoadSourceType::KAFKA: + consumer = std::make_shared(ctx); + break; + default: + std::stringstream ss; + ss << "unknown routine load task type: " << ctx->load_type; + return Status(ss.str()); + } + + // init the consumer + RETURN_IF_ERROR(consumer->init(ctx)); + + VLOG(3) << "create new data consumer: " << consumer->id(); + *ret = consumer; + return Status::OK; +} + +Status DataConsumerPool::get_consumer_grp( + StreamLoadContext* ctx, + std::shared_ptr* ret) { + if (ctx->load_src_type != TLoadSourceType::KAFKA) { + return Status("Currently nly support consumer group for Kafka data source"); + } + DCHECK(ctx->kafka_info); + + std::shared_ptr grp = std::make_shared(); + + // one data consumer group contains at least one data consumers. + int max_consumer_num = config::max_consumer_num_per_group; + size_t consumer_num = std::min((size_t) max_consumer_num, ctx->kafka_info->begin_offset.size()); + for (int i = 0; i < consumer_num; ++i) { + std::shared_ptr consumer; + RETURN_IF_ERROR(get_consumer(ctx, &consumer)); + grp->add_consumer(consumer); + } + + LOG(INFO) << "get consumer group " << grp->grp_id() << " with " + << consumer_num << " consumers"; + *ret = grp; + return Status::OK; +} + +void DataConsumerPool::return_consumer(std::shared_ptr consumer) { + std::unique_lock l(_lock); + + if (_pool.size() == _max_pool_size) { + VLOG(3) << "data consumer pool is full: " << _pool.size() + << "-" << _max_pool_size << ", discard the returned consumer: " + << consumer->id(); + return; + } + + consumer->reset(); + _pool.push_back(consumer); + VLOG(3) << "return the data consumer: " << consumer->id() + << ", current pool size: " << _pool.size(); + return; +} + +void DataConsumerPool::return_consumers(DataConsumerGroup* grp) { + for (std::shared_ptr consumer : grp->consumers()) { + return_consumer(consumer); + } +} + +Status DataConsumerPool::start_bg_worker() { + _clean_idle_consumer_thread = std::thread( + [this] { + #ifdef GOOGLE_PROFILER + ProfilerRegisterThread(); + #endif + + uint32_t interval = 60; + while (true) { + _clean_idle_consumer_bg(); + sleep(interval); + } + }); + _clean_idle_consumer_thread.detach(); + return Status::OK; +} + +void DataConsumerPool::_clean_idle_consumer_bg() { + const static int32_t max_idle_time_second = 600; + + std::unique_lock l(_lock); + time_t now = time(nullptr); + + auto iter = std::begin(_pool); + while (iter != std::end(_pool)) { + if (difftime(now, (*iter)->last_visit_time()) >= max_idle_time_second) { + LOG(INFO) << "remove data consumer " << (*iter)->id() + << ", since it last visit: " << (*iter)->last_visit_time() + << ", now: " << now; + iter = _pool.erase(iter); + } else { + ++iter; + } + } +} + +} // end namespace doris diff --git a/be/src/runtime/routine_load/data_consumer_pool.h b/be/src/runtime/routine_load/data_consumer_pool.h new file mode 100644 index 00000000000000..16808dbb2332fc --- /dev/null +++ b/be/src/runtime/routine_load/data_consumer_pool.h @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include + +#include "runtime/routine_load/data_consumer.h" +#include "util/lru_cache.hpp" + +namespace doris { + +class DataConsumer; +class DataConsumerGroup; +class Status; + +// DataConsumerPool saves all available data consumer +// to be reused +class DataConsumerPool { +public: + DataConsumerPool(int64_t max_pool_size): + _max_pool_size(max_pool_size) { + } + + ~DataConsumerPool() { + } + + // get a already initialized consumer from cache, + // if not found in cache, create a new one. + Status get_consumer( + StreamLoadContext* ctx, + std::shared_ptr* ret); + + // get several consumers and put them into group + Status get_consumer_grp( + StreamLoadContext* ctx, + std::shared_ptr* ret); + + // return the consumer to the pool + void return_consumer(std::shared_ptr consumer); + // return the consumers in consumer group to the pool + void return_consumers(DataConsumerGroup* grp); + + Status start_bg_worker(); + +private: + void _clean_idle_consumer_bg(); + +private: + std::mutex _lock; + std::list> _pool; + int64_t _max_pool_size; + + std::thread _clean_idle_consumer_thread; +}; + +} // end namespace doris diff --git a/be/src/runtime/kafka_consumer_pipe.h b/be/src/runtime/routine_load/kafka_consumer_pipe.h similarity index 61% rename from be/src/runtime/kafka_consumer_pipe.h rename to be/src/runtime/routine_load/kafka_consumer_pipe.h index 4d647273e3422c..798a7a3eef5470 100644 --- a/be/src/runtime/kafka_consumer_pipe.h +++ b/be/src/runtime/routine_load/kafka_consumer_pipe.h @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -#ifndef DORIS_BE_SRC_RUNTIME_KAFKA_COMSUMER_PIPE_H -#define DORIS_BE_SRC_RUNTIME_KAFKA_COMSUMER_PIPE_H +#pragma once #include @@ -27,24 +26,32 @@ #include "librdkafka/rdkafka.h" #include "exec/file_reader.h" -#include "http/message_body_sink.h" +#include "runtime/message_body_sink.h" +#include "runtime/stream_load/stream_load_pipe.h" namespace doris { -class KafkaConsumerPipe : public MessageBodySink, public FileReader { +class KafkaConsumerPipe : public StreamLoadPipe { + public: - KafkaConsumerPipe(); - ~KafkaConsumerPipe(); + KafkaConsumerPipe(size_t max_buffered_bytes = 1024 * 1024, + size_t min_chunk_size = 64 * 1024) + : StreamLoadPipe(max_buffered_bytes, min_chunk_size) { + } -private: - // this is only for testing librdkafka.a - void test_kafka_lib() { - //rd_kafka_conf_t *conf = rd_kafka_conf_new(); - //rd_kafka_topic_conf_t *topic_conf = rd_kafka_topic_conf_new(); + virtual ~KafkaConsumerPipe() {} + + Status append_with_line_delimiter(const char* data, size_t size) { + Status st = append(data, size); + if (!st.ok()) { + return st; + } + + // append the line delimiter + st = append("\n", 1); + return st; } }; } // end namespace doris - -#endif // DORIS_BE_SRC_RUNTIME_KAFKA_COMSUMER_PIPE_H diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp new file mode 100644 index 00000000000000..ec0afa4c55d3be --- /dev/null +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -0,0 +1,248 @@ +// 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 "runtime/routine_load/routine_load_task_executor.h" + +#include "common/status.h" +#include "runtime/exec_env.h" +#include "runtime/routine_load/data_consumer_group.h" +#include "runtime/routine_load/kafka_consumer_pipe.h" +#include "runtime/stream_load/stream_load_context.h" +#include "runtime/stream_load/stream_load_executor.h" +#include "util/uid_util.h" + +#include + +#include "gen_cpp/FrontendService_types.h" +#include "gen_cpp/BackendService_types.h" +#include "gen_cpp/Types_types.h" + +namespace doris { + +Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { + std::unique_lock l(_lock); + if (_task_map.find(task.id) != _task_map.end()) { + // already submitted + LOG(INFO) << "routine load task " << UniqueId(task.id) << " has already been submitted"; + return Status::OK; + } + + // create the context + StreamLoadContext* ctx = new StreamLoadContext(_exec_env); + ctx->load_type = TLoadType::ROUTINE_LOAD; + ctx->load_src_type = task.type; + ctx->job_id = task.job_id; + ctx->id = UniqueId(task.id); + ctx->txn_id = task.txn_id; + ctx->db = task.db; + ctx->table = task.tbl; + ctx->label = task.label; + ctx->auth.auth_code = task.auth_code; + + if (task.__isset.max_interval_s) { ctx->max_interval_s = task.max_interval_s; } + if (task.__isset.max_batch_rows) { ctx->max_batch_rows = task.max_batch_rows; } + if (task.__isset.max_batch_size) { ctx->max_batch_size = task.max_batch_size; } + + // set execute plan params + TStreamLoadPutResult put_result; + TStatus tstatus; + tstatus.status_code = TStatusCode::OK; + put_result.status = tstatus; + put_result.params = std::move(task.params); + put_result.__isset.params = true; + ctx->put_result = std::move(put_result); + + // the routine load task'txn has alreay began in FE. + // so it need to rollback if encounter error. + ctx->need_rollback = true; + ctx->max_filter_ratio = 1.0; + + // set source related params + switch (task.type) { + case TLoadSourceType::KAFKA: + ctx->kafka_info = new KafkaLoadInfo(task.kafka_load_info); + break; + default: + LOG(WARNING) << "unknown load source type: " << task.type; + delete ctx; + return Status("unknown load source type"); + } + + VLOG(1) << "receive a new routine load task: " << ctx->brief(); + // register the task + ctx->ref(); + _task_map[ctx->id] = ctx; + + // offer the task to thread pool + if (!_thread_pool.offer( + boost::bind(&RoutineLoadTaskExecutor::exec_task, this, ctx, + &_data_consumer_pool, + [this] (StreamLoadContext* ctx) { + std::unique_lock l(_lock); + _task_map.erase(ctx->id); + LOG(INFO) << "finished routine load task " << ctx->brief() + << ", status: " << ctx->status.get_error_msg() + << ", current tasks num: " << _task_map.size(); + if (ctx->unref()) { + delete ctx; + } + }))) { + + // failed to submit task, clear and return + LOG(WARNING) << "failed to submit routine load task: " << ctx->brief(); + _task_map.erase(ctx->id); + if (ctx->unref()) { + delete ctx; + } + return Status("failed to submit routine load task"); + } else { + LOG(INFO) << "submit a new routine load task: " << ctx->brief() + << ", current tasks num: " << _task_map.size(); + return Status::OK; + } +} + +void RoutineLoadTaskExecutor::exec_task( + StreamLoadContext* ctx, + DataConsumerPool* consumer_pool, + ExecFinishCallback cb) { + +#define HANDLE_ERROR(stmt, err_msg) \ + do { \ + Status _status_ = (stmt); \ + if (UNLIKELY(!_status_.ok())) { \ + err_handler(ctx, _status_, err_msg); \ + cb(ctx); \ + return; \ + } \ + } while (false); + + VLOG(1) << "begin to execute routine load task: " << ctx->brief(); + + // create data consumer group + std::shared_ptr consumer_grp; + HANDLE_ERROR(consumer_pool->get_consumer_grp(ctx, &consumer_grp), "failed to get consumers"); + + // create and set pipe + std::shared_ptr pipe; + switch (ctx->load_src_type) { + case TLoadSourceType::KAFKA: { + pipe = std::make_shared(); + Status st = std::static_pointer_cast(consumer_grp)->assign_topic_partitions(ctx); + if (!st.ok()) { + err_handler(ctx, st, st.get_error_msg()); + cb(ctx); + return; + } + break; + } + default: { + std::stringstream ss; + ss << "unknown routine load task type: " << ctx->load_type; + err_handler(ctx, Status::CANCELLED, ss.str()); + cb(ctx); + return; + } + } + ctx->body_sink = pipe; + + // must put pipe before executing plan fragment + HANDLE_ERROR(_exec_env->load_stream_mgr()->put(ctx->id, pipe), "failed to add pipe"); + +#ifndef BE_TEST + // execute plan fragment, async + HANDLE_ERROR(_exec_env->stream_load_executor()->execute_plan_fragment(ctx), + "failed to execute plan fragment"); +#else + // only for test + HANDLE_ERROR(_execute_plan_for_test(ctx), "test failed"); +#endif + + // start to consume, this may block a while + HANDLE_ERROR(consumer_grp->start_all(ctx), "consuming failed"); + + // wait for all consumers finished + HANDLE_ERROR(ctx->future.get(), "consume failed"); + + ctx->load_cost_nanos = MonotonicNanos() - ctx->start_nanos; + + // return the consumer back to pool + // call this before commit txn, in case the next task can come very fast + consumer_pool->return_consumers(consumer_grp.get()); + + // commit txn + HANDLE_ERROR(_exec_env->stream_load_executor()->commit_txn(ctx), "commit failed"); + + cb(ctx); +} + +void RoutineLoadTaskExecutor::err_handler( + StreamLoadContext* ctx, + const Status& st, + const std::string& err_msg) { + + LOG(WARNING) << err_msg; + ctx->status = st; + if (ctx->need_rollback) { + _exec_env->stream_load_executor()->rollback_txn(ctx); + ctx->need_rollback = false; + } + if (ctx->body_sink.get() != nullptr) { + ctx->body_sink->cancel(); + } + + return; +} + +// for test only +Status RoutineLoadTaskExecutor::_execute_plan_for_test(StreamLoadContext* ctx) { + auto mock_consumer = [this, ctx]() { + std::shared_ptr pipe = _exec_env->load_stream_mgr()->get(ctx->id); + bool eof = false; + std::stringstream ss; + while (true) { + char one; + size_t len = 1; + Status st = pipe->read((uint8_t*) &one, &len, &eof); + if (!st.ok()) { + LOG(WARNING) << "read failed"; + ctx->promise.set_value(st); + break; + } + + if (eof) { + ctx->promise.set_value(Status::OK); + break; + } + + if (one == '\n') { + LOG(INFO) << "get line: " << ss.str(); + ss.str(""); + ctx->number_loaded_rows++; + } else { + ss << one; + } + } + }; + + std::thread t1(mock_consumer); + t1.detach(); + return Status::OK; +} + +} // end namespace + diff --git a/be/src/runtime/routine_load/routine_load_task_executor.h b/be/src/runtime/routine_load/routine_load_task_executor.h new file mode 100644 index 00000000000000..604a83e17e3cc0 --- /dev/null +++ b/be/src/runtime/routine_load/routine_load_task_executor.h @@ -0,0 +1,80 @@ +// 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 "runtime/routine_load/data_consumer_pool.h" +#include "util/thread_pool.hpp" +#include "util/uid_util.h" + +namespace doris { + +class ExecEnv; +class Status; +class StreamLoadContext; +class TRoutineLoadTask; + +// A routine load task executor will receive routine load +// tasks from FE, put it to a fixed thread pool. +// The thread pool will process each task and report the result +// to FE finally. +class RoutineLoadTaskExecutor { +public: + typedef std::function ExecFinishCallback; + + RoutineLoadTaskExecutor(ExecEnv* exec_env): + _exec_env(exec_env), + _thread_pool(10, 1000), + _data_consumer_pool(10) { + + _data_consumer_pool.start_bg_worker(); + } + + ~RoutineLoadTaskExecutor() { + + } + + // submit a routine load task + Status submit_task(const TRoutineLoadTask& task); + +private: + // execute the task + void exec_task(StreamLoadContext* ctx, DataConsumerPool* pool, ExecFinishCallback cb); + + void err_handler( + StreamLoadContext* ctx, + const Status& st, + const std::string& err_msg); + + // for test only + Status _execute_plan_for_test(StreamLoadContext* ctx); + +private: + ExecEnv* _exec_env; + ThreadPool _thread_pool; + DataConsumerPool _data_consumer_pool; + + std::mutex _lock; + // task id -> load context + std::unordered_map _task_map; +}; + +} // end namespace diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 5accbebb7232f3..4b901a74a128a3 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -59,11 +59,14 @@ RuntimeState::RuntimeState( _is_cancelled(false), _per_fragment_instance_idx(0), _root_node_id(-1), + _num_rows_load_total(0), _num_rows_load_success(0), _num_rows_load_filtered(0), + _num_rows_load_unselected(0), _num_print_error_rows(0), _normal_row_number(0), _error_row_number(0), + _error_log_file_path(""), _error_log_file(nullptr), _instance_buffer_reservation(new ReservationTracker) { Status status = init(fragment_instance_id, query_options, now, exec_env); @@ -84,11 +87,14 @@ RuntimeState::RuntimeState( _is_cancelled(false), _per_fragment_instance_idx(0), _root_node_id(-1), + _num_rows_load_total(0), _num_rows_load_success(0), _num_rows_load_filtered(0), + _num_rows_load_unselected(0), _num_print_error_rows(0), _normal_row_number(0), _error_row_number(0), + _error_log_file_path(""), _error_log_file(nullptr), _instance_buffer_reservation(new ReservationTracker) { Status status = init(fragment_params.params.fragment_instance_id, query_options, now, exec_env); @@ -448,7 +454,7 @@ void RuntimeState::append_error_msg_to_file( if (_error_row_number < MAX_ERROR_NUM) { // Note: export reason first in case src line too long and be truncated. out << "Reason: " << error_msg; - out << "src line: [" << line << "]; "; + out << ". src line: [" << line << "]; "; } else if (_error_row_number == MAX_ERROR_NUM) { out << "TOO MUCH ERROR! already reach " << MAX_ERROR_NUM << "." << " no more show next error."; diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index e89cbaaa78d4e5..a6a1a626323adf 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -411,6 +411,10 @@ class RuntimeState { void append_error_msg_to_file(const std::string& line, const std::string& error_msg, bool is_summary = false); + int64_t num_rows_load_total() { + return _num_rows_load_total.load(); + } + int64_t num_rows_load_success() { return _num_rows_load_success.load(); } @@ -419,6 +423,14 @@ class RuntimeState { return _num_rows_load_filtered.load(); } + int64_t num_rows_load_unselected() { + return _num_rows_load_unselected.load(); + } + + void update_num_rows_load_total(int64_t num_rows) { + _num_rows_load_total.fetch_add(num_rows); + } + void update_num_rows_load_success(int64_t num_rows) { _num_rows_load_success.fetch_add(num_rows); } @@ -426,6 +438,11 @@ class RuntimeState { void update_num_rows_load_filtered(int64_t num_rows) { _num_rows_load_filtered.fetch_add(num_rows); } + + void update_num_rows_load_unselected(int64_t num_rows) { + _num_rows_load_unselected.fetch_add(num_rows); + } + void export_load_error(const std::string& error_msg); void set_per_fragment_instance_idx(int idx) { @@ -583,8 +600,10 @@ class RuntimeState { // put here to collect files?? std::vector _output_files; - std::atomic _num_rows_load_success; - std::atomic _num_rows_load_filtered; + std::atomic _num_rows_load_total; // total rows read from source + std::atomic _num_rows_load_success; // rows loaded success + std::atomic _num_rows_load_filtered; // unqualified rows + std::atomic _num_rows_load_unselected; // rows filtered by predicates std::atomic _num_print_error_rows; std::vector _export_output_files; diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp index 584269284c09b0..b2e3a0d5956782 100644 --- a/be/src/runtime/snapshot_loader.cpp +++ b/be/src/runtime/snapshot_loader.cpp @@ -28,13 +28,13 @@ #include "common/logging.h" #include "exec/broker_reader.h" #include "exec/broker_writer.h" -#include "exec/schema_scanner/frontend_helper.h" #include "olap/file_helper.h" #include "olap/olap_engine.h" #include "olap/olap_table.h" #include "runtime/exec_env.h" #include "runtime/broker_mgr.h" #include "util/file_utils.h" +#include "util/frontend_helper.h" namespace doris { diff --git a/be/src/runtime/load_stream_mgr.h b/be/src/runtime/stream_load/load_stream_mgr.h similarity index 82% rename from be/src/runtime/load_stream_mgr.h rename to be/src/runtime/stream_load/load_stream_mgr.h index 02dfae035b32cb..cfaf08a94a0a63 100644 --- a/be/src/runtime/load_stream_mgr.h +++ b/be/src/runtime/stream_load/load_stream_mgr.h @@ -21,7 +21,7 @@ #include #include -#include "runtime/stream_load_pipe.h" // for StreamLoadPipe +#include "runtime/stream_load/stream_load_pipe.h" // for StreamLoadPipe #include "util/uid_util.h" // for std::hash for UniqueId namespace doris { @@ -40,6 +40,7 @@ class LoadStreamMgr { return Status("id already exist"); } _stream_map.emplace(id, stream); + VLOG(3) << "put stream load pipe: " << id; return Status::OK; } @@ -54,6 +55,16 @@ class LoadStreamMgr { return stream; } + void remove(const UniqueId& id) { + std::lock_guard l(_lock); + auto it = _stream_map.find(id); + if (it != std::end(_stream_map)) { + _stream_map.erase(it); + VLOG(3) << "remove stream load pipe: " << id; + } + return; + } + private: std::mutex _lock; std::unordered_map> _stream_map; diff --git a/be/src/runtime/stream_load/stream_load_context.cpp b/be/src/runtime/stream_load/stream_load_context.cpp new file mode 100644 index 00000000000000..c4ec694a00c9d7 --- /dev/null +++ b/be/src/runtime/stream_load/stream_load_context.cpp @@ -0,0 +1,102 @@ +// 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 "runtime/stream_load/stream_load_context.h" + +namespace doris { + +std::string StreamLoadContext::to_json() const { + rapidjson::StringBuffer s; + rapidjson::PrettyWriter writer(s); + + writer.StartObject(); + // txn id + writer.Key("TxnId"); + writer.Int64(txn_id); + + // label + writer.Key("Label"); + writer.String(label.c_str()); + + // status + writer.Key("Status"); + switch (status.code()) { + case TStatusCode::OK: + writer.String("Success"); + break; + case TStatusCode::PUBLISH_TIMEOUT: + writer.String("Publish Timeout"); + break; + case TStatusCode::LABEL_ALREADY_EXISTS: + writer.String("Label Already Exists"); + break; + default: + writer.String("Fail"); + break; + } + // msg + writer.Key("Message"); + if (status.ok()) { + writer.String("OK"); + } else { + writer.String(status.get_error_msg().c_str()); + } + // number_load_rows + writer.Key("NumberTotalRows"); + writer.Int64(number_total_rows); + writer.Key("NumberLoadedRows"); + writer.Int64(number_loaded_rows); + writer.Key("NumberFilteredRows"); + writer.Int64(number_filtered_rows); + writer.Key("NumberUnselectedRows"); + writer.Int64(number_unselected_rows); + writer.Key("LoadBytes"); + writer.Int64(receive_bytes); + writer.Key("LoadTimeMs"); + writer.Int64(load_cost_nanos / 1000000); + if (!error_url.empty()) { + writer.Key("ErrorURL"); + writer.String(error_url.c_str()); + } + writer.EndObject(); + return s.GetString(); +} + +std::string StreamLoadContext::brief(bool detail) const { + std::stringstream ss; + ss << "id=" << id << ", job id=" << job_id << ", txn id=" << txn_id << ", label=" << label; + if (detail) { + switch(load_src_type) { + case TLoadSourceType::KAFKA: + if (kafka_info != nullptr) { + ss << ", kafka" + << ", brokers: " << kafka_info->brokers + << ", topic: " << kafka_info->topic + << ", partition: "; + for (auto& entry : kafka_info->begin_offset) { + ss << "[" << entry.first << ": " << entry.second << "]"; + } + } + break; + default: + break; + } + } + return ss.str(); +} + +} // end namespace diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h new file mode 100644 index 00000000000000..b530a409000893 --- /dev/null +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -0,0 +1,177 @@ +// 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 "gen_cpp/BackendService_types.h" +#include "gen_cpp/FrontendService_types.h" + +#include "common/status.h" +#include "common/logging.h" +#include "common/utils.h" +#include "runtime/exec_env.h" +#include "runtime/stream_load/load_stream_mgr.h" +#include "runtime/stream_load/stream_load_executor.h" +#include "service/backend_options.h" +#include "util/string_util.h" +#include "util/time.h" +#include "util/uid_util.h" + +namespace doris { + +// kafka related info +class KafkaLoadInfo { +public: + KafkaLoadInfo(const TKafkaLoadInfo& t_info): + brokers(t_info.brokers), + topic(t_info.topic), + begin_offset(t_info.partition_begin_offset) { + + for (auto& p : t_info.partition_begin_offset) { + cmt_offset[p.first] = p.second -1; + } + } + + void reset_offset() { + // reset the commit offset + for (auto& p : begin_offset) { + cmt_offset[p.first] = p.second -1; + } + } + +public: + std::string brokers; + std::string topic; + + // the following members control the max progress of a consuming + // process. if any of them reach, the consuming will finish. + int64_t max_interval_s = 5; + int64_t max_batch_rows = 1024; + int64_t max_batch_size = 100 * 1024 * 1024; // 100MB + + // partition -> begin offset, inclusive. + std::map begin_offset; + // partiton -> commit offset, inclusive. + std::map cmt_offset; +}; + +class MessageBodySink; + +class StreamLoadContext { +public: + StreamLoadContext(ExecEnv* exec_env) : + _exec_env(exec_env), + _refs(0) { + start_nanos = MonotonicNanos(); + } + + ~StreamLoadContext() { + if (need_rollback) { + _exec_env->stream_load_executor()->rollback_txn(this); + need_rollback = false; + } + + _exec_env->load_stream_mgr()->remove(id); + + if (kafka_info != nullptr) { + delete kafka_info; + kafka_info = nullptr; + } + } + + std::string to_json() const; + + // return the brief info of this context. + // also print the load source info if detail is set to true + std::string brief(bool detail = false) const; + + void ref() { _refs.fetch_add(1); } + // If unref() returns true, this object should be delete + bool unref() { return _refs.fetch_sub(1) == 1; } + +public: + // load type, eg: ROUTINE LOAD/MANUL LOAD + TLoadType::type load_type; + // load data source: eg: KAFKA/RAW + TLoadSourceType::type load_src_type; + + // the job this stream load task belongs to, + // set to -1 if there is no job + int64_t job_id = -1; + + // id for each load + UniqueId id; + + std::string db; + std::string table; + std::string label; + + AuthInfo auth; + + // the following members control the max progress of a consuming + // process. if any of them reach, the consuming will finish. + int64_t max_interval_s = 5; + int64_t max_batch_rows = 100000; + int64_t max_batch_size = 100 * 1024 * 1024; // 100MB + + // only used to check if we receive whole body + size_t body_bytes = 0; + size_t receive_bytes = 0; + + int64_t txn_id = -1; + + bool need_rollback = false; + // when use_streaming is true, we use stream_pipe to send source data, + // otherwise we save source data to file first, then process it. + bool use_streaming = false; + TFileFormatType::type format = TFileFormatType::FORMAT_CSV_PLAIN; + + std::shared_ptr body_sink; + + TStreamLoadPutResult put_result; + double max_filter_ratio = 0.0; + std::vector commit_infos; + + std::promise promise; + std::future future = promise.get_future(); + + Status status; + + int64_t number_total_rows = 0; + int64_t number_loaded_rows = 0; + int64_t number_filtered_rows = 0; + int64_t number_unselected_rows = 0; + int64_t loaded_bytes = 0; + int64_t start_nanos = 0; + int64_t load_cost_nanos = 0; + std::string error_url = ""; + + KafkaLoadInfo* kafka_info = nullptr; + + // consumer_id is used for data consumer cache key. + // to identified a specified data consumer. + int64_t consumer_id; +private: + ExecEnv* _exec_env; + std::atomic _refs; +}; + +} // end namespace diff --git a/be/src/runtime/stream_load/stream_load_executor.cpp b/be/src/runtime/stream_load/stream_load_executor.cpp new file mode 100644 index 00000000000000..432d0c5b5ae62d --- /dev/null +++ b/be/src/runtime/stream_load/stream_load_executor.cpp @@ -0,0 +1,261 @@ +// 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 "runtime/stream_load/stream_load_executor.h" + +#include "common/status.h" +#include "common/utils.h" +#include "runtime/client_cache.h" +#include "runtime/exec_env.h" +#include "runtime/fragment_mgr.h" +#include "runtime/plan_fragment_executor.h" +#include "runtime/runtime_state.h" +#include "runtime/stream_load/stream_load_context.h" +#include "util/frontend_helper.h" + +#include "gen_cpp/FrontendService.h" +#include "gen_cpp/FrontendService_types.h" +#include "gen_cpp/HeartbeatService_types.h" +#include "gen_cpp/Types_types.h" + +namespace doris { + +#ifdef BE_TEST +TLoadTxnBeginResult k_stream_load_begin_result; +TLoadTxnCommitResult k_stream_load_commit_result; +TLoadTxnRollbackResult k_stream_load_rollback_result; +Status k_stream_load_plan_status; +#endif + +Status StreamLoadExecutor::execute_plan_fragment(StreamLoadContext* ctx) { + // submit this params +#ifndef BE_TEST + ctx->ref(); + auto st = _exec_env->fragment_mgr()->exec_plan_fragment( + ctx->put_result.params, + [ctx] (PlanFragmentExecutor* executor) { + ctx->commit_infos = std::move(executor->runtime_state()->tablet_commit_infos()); + Status status = executor->status(); + if (status.ok()) { + ctx->number_total_rows = executor->runtime_state()->num_rows_load_total(); + ctx->number_loaded_rows = executor->runtime_state()->num_rows_load_success(); + ctx->number_filtered_rows = executor->runtime_state()->num_rows_load_filtered(); + ctx->number_unselected_rows = executor->runtime_state()->num_rows_load_unselected(); + + int64_t num_selected_rows = + ctx->number_total_rows - ctx->number_unselected_rows; + if ((0.0 + ctx->number_filtered_rows) / num_selected_rows > ctx->max_filter_ratio) { + status = Status("too many filtered rows"); + } + else if(ctx->number_loaded_rows == 0){ + status = Status("all partitions have no load data"); + } + if (ctx->number_filtered_rows > 0 && + !executor->runtime_state()->get_error_log_file_path().empty()) { + + // if (ctx->load_type == TLoadType::MANUL_LOAD) { + ctx->error_url = to_load_error_http_path( + executor->runtime_state()->get_error_log_file_path()); + // } + } + } else { + LOG(WARNING) << "fragment execute failed" + << ", query_id=" << UniqueId(ctx->put_result.params.params.query_id) + << ", errmsg=" << status.get_error_msg() + << ctx->brief(); + // cancel body_sink, make sender known it + if (ctx->body_sink != nullptr) { + ctx->body_sink->cancel(); + } + + switch(ctx->load_src_type) { + // reset the stream load ctx's kafka commit offset + case TLoadSourceType::KAFKA: + ctx->kafka_info->reset_offset(); + break; + default: + break; + } + } + ctx->promise.set_value(status); + if (ctx->unref()) { + delete ctx; + } + }); + if (!st.ok()) { + // no need to check unref's return value + ctx->unref(); + return st; + } +#else + ctx->promise.set_value(k_stream_load_plan_status); +#endif + return Status::OK; +} + +Status StreamLoadExecutor::begin_txn(StreamLoadContext* ctx) { + TNetworkAddress master_addr = _exec_env->master_info()->network_address; + + TLoadTxnBeginRequest request; + set_request_auth(&request, ctx->auth); + request.db = ctx->db; + request.tbl = ctx->table; + request.label = ctx->label; + // set timestamp + request.__set_timestamp(GetCurrentTimeMicros()); + + TLoadTxnBeginResult result; +#ifndef BE_TEST + RETURN_IF_ERROR(FrontendHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result] (FrontendServiceConnection& client) { + client->loadTxnBegin(result, request); + })); +#else + result = k_stream_load_begin_result; +#endif + Status status(result.status); + if (!status.ok()) { + LOG(WARNING) << "begin transaction failed, errmsg=" << status.get_error_msg() + << ctx->brief(); + return status; + } + ctx->txn_id = result.txnId; + ctx->need_rollback = true; + + return Status::OK; +} + +Status StreamLoadExecutor::commit_txn(StreamLoadContext* ctx) { + TNetworkAddress master_addr = _exec_env->master_info()->network_address; + + TLoadTxnCommitRequest request; + set_request_auth(&request, ctx->auth); + request.db = ctx->db; + request.tbl = ctx->table; + request.txnId = ctx->txn_id; + request.sync = true; + request.commitInfos = std::move(ctx->commit_infos); + request.__isset.commitInfos = true; + + // set attachment if has + TTxnCommitAttachment attachment; + if (collect_load_stat(ctx, &attachment)) { + request.txnCommitAttachment = std::move(attachment); + request.__isset.txnCommitAttachment = true; + } + + TLoadTxnCommitResult result; +#ifndef BE_TEST + RETURN_IF_ERROR(FrontendHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result] (FrontendServiceConnection& client) { + client->loadTxnCommit(result, request); + }, config::txn_commit_rpc_timeout_ms)); +#else + result = k_stream_load_commit_result; +#endif + // Return if this transaction is committed successful; otherwise, we need try to + // rollback this transaction + Status status(result.status); + if (!status.ok()) { + LOG(WARNING) << "commit transaction failed, id=" << ctx->id + << ", errmsg=" << status.get_error_msg(); + return status; + } + // commit success, set need_rollback to false + ctx->need_rollback = false; + return Status::OK; +} + +void StreamLoadExecutor::rollback_txn(StreamLoadContext* ctx) { + TNetworkAddress master_addr = _exec_env->master_info()->network_address; + TLoadTxnRollbackRequest request; + set_request_auth(&request, ctx->auth); + request.db = ctx->db; + request.tbl = ctx->table; + request.txnId = ctx->txn_id; + request.__set_reason(ctx->status.get_error_msg()); + + // set attachment if has + TTxnCommitAttachment attachment; + if (collect_load_stat(ctx, &attachment)) { + request.txnCommitAttachment = std::move(attachment); + request.__isset.txnCommitAttachment = true; + } + + TLoadTxnRollbackResult result; +#ifndef BE_TEST + auto rpc_st = FrontendHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result] (FrontendServiceConnection& client) { + client->loadTxnRollback(result, request); + }); + if (!rpc_st.ok()) { + LOG(WARNING) << "transaction rollback failed. errmsg=" << rpc_st.get_error_msg() + << ctx->brief(); + } +#else + result = k_stream_load_rollback_result; +#endif +} + +bool StreamLoadExecutor::collect_load_stat(StreamLoadContext* ctx, TTxnCommitAttachment* attach) { + if (ctx->load_type != TLoadType::ROUTINE_LOAD) { + // currently, only routine load need to set attachment + return false; + } + + switch(ctx->load_src_type) { + case TLoadSourceType::KAFKA: { + attach->loadType = TLoadType::ROUTINE_LOAD; + + TRLTaskTxnCommitAttachment rl_attach; + rl_attach.loadSourceType = TLoadSourceType::KAFKA; + rl_attach.jobId = ctx->job_id; + rl_attach.id = ctx->id.to_thrift(); + rl_attach.__set_loadedRows(ctx->number_loaded_rows); + rl_attach.__set_filteredRows(ctx->number_filtered_rows); + rl_attach.__set_unselectedRows(ctx->number_unselected_rows); + rl_attach.__set_receivedBytes(ctx->receive_bytes); + rl_attach.__set_loadedBytes(ctx->loaded_bytes); + rl_attach.__set_loadCostMs(ctx->load_cost_nanos / 1000 / 1000); + + TKafkaRLTaskProgress kafka_progress; + kafka_progress.partitionCmtOffset = std::move(ctx->kafka_info->cmt_offset); + rl_attach.kafkaRLTaskProgress = std::move(kafka_progress); + rl_attach.__isset.kafkaRLTaskProgress = true; + + if (!ctx->error_url.empty()) { + rl_attach.__set_errorLogUrl(ctx->error_url); + } + + attach->rlTaskTxnCommitAttachment = std::move(rl_attach); + attach->__isset.rlTaskTxnCommitAttachment = true; + + return true; + } + case TLoadSourceType::RAW: + return false; + default: + // unknown type, should not happend + return false; + } + return false; +} + +} // end namespace diff --git a/fe/src/main/java/org/apache/doris/task/KafkaRoutineLoadTask.java b/be/src/runtime/stream_load/stream_load_executor.h similarity index 51% rename from fe/src/main/java/org/apache/doris/task/KafkaRoutineLoadTask.java rename to be/src/runtime/stream_load/stream_load_executor.h index 922127387374ed..68d49be828f6e4 100644 --- a/fe/src/main/java/org/apache/doris/task/KafkaRoutineLoadTask.java +++ b/be/src/runtime/stream_load/stream_load_executor.h @@ -15,26 +15,37 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.task; +#pragma once -import org.apache.doris.load.routineload.LoadDataSourceType; -import org.apache.doris.thrift.TResourceInfo; +namespace doris { -import java.util.Map; +class ExecEnv; +class StreamLoadContext; +class Status; +class TTxnCommitAttachment; +class StreamLoadExecutor { -public class KafkaRoutineLoadTask extends RoutineLoadTask { +public: + StreamLoadExecutor(ExecEnv* exec_env): + _exec_env(exec_env) { + } - private Map partitionIdToOffset; + Status begin_txn(StreamLoadContext* ctx); - public KafkaRoutineLoadTask(TResourceInfo resourceInfo, long backendId, - long dbId, long tableId, String taskId, - long txnId, Map partitionIdToOffset) { - super(resourceInfo, backendId, dbId, tableId, taskId, LoadDataSourceType.KAFKA, txnId); - this.partitionIdToOffset = partitionIdToOffset; - } + Status commit_txn(StreamLoadContext* ctx); + + void rollback_txn(StreamLoadContext* ctx); + + Status execute_plan_fragment(StreamLoadContext* ctx); + +private: + // collect the load statistics from context and set them to stat + // return true if stat is set, otherwise, return false + bool collect_load_stat(StreamLoadContext* ctx, TTxnCommitAttachment* attachment); + +private: + ExecEnv* _exec_env; +}; - public Map getPartitionIdToOffset() { - return partitionIdToOffset; - } } diff --git a/be/src/runtime/stream_load_pipe.h b/be/src/runtime/stream_load/stream_load_pipe.h similarity index 99% rename from be/src/runtime/stream_load_pipe.h rename to be/src/runtime/stream_load/stream_load_pipe.h index 781f4d66232a20..c422967635b2a3 100644 --- a/be/src/runtime/stream_load_pipe.h +++ b/be/src/runtime/stream_load/stream_load_pipe.h @@ -22,7 +22,7 @@ #include #include "exec/file_reader.h" -#include "http/message_body_sink.h" +#include "runtime/message_body_sink.h" #include "util/bit_util.h" #include "util/byte_buffer.h" diff --git a/be/src/runtime/string_value.cpp b/be/src/runtime/string_value.cpp index 71a91faba50ea1..8ac089236fbc2f 100644 --- a/be/src/runtime/string_value.cpp +++ b/be/src/runtime/string_value.cpp @@ -27,6 +27,10 @@ std::string StringValue::debug_string() const { return std::string(ptr, len); } +std::string StringValue::to_string() const { + return std::string(ptr, len); +} + std::ostream& operator<<(std::ostream& os, const StringValue& string_value) { return os << string_value.debug_string(); } diff --git a/be/src/runtime/string_value.h b/be/src/runtime/string_value.h index f840f0d905dcb8..d03604625c4f1d 100644 --- a/be/src/runtime/string_value.h +++ b/be/src/runtime/string_value.h @@ -105,6 +105,8 @@ struct StringValue { std::string debug_string() const; + std::string to_string() const; + // Returns the substring starting at start_pos until the end of string. StringValue substring(int start_pos) const; diff --git a/be/src/runtime/tablet_writer_mgr.cpp b/be/src/runtime/tablet_writer_mgr.cpp index ee0693eacd74a7..59bd5cd70a1775 100644 --- a/be/src/runtime/tablet_writer_mgr.cpp +++ b/be/src/runtime/tablet_writer_mgr.cpp @@ -28,6 +28,7 @@ #include "runtime/row_batch.h" #include "runtime/tuple_row.h" #include "util/bitmap.h" +#include "util/stopwatch.hpp" #include "olap/delta_writer.h" #include "olap/lru_cache.h" @@ -165,7 +166,7 @@ Status TabletsChannel::close(int sender_id, bool* finished, google::protobuf::RepeatedPtrField* tablet_vec) { std::lock_guard l(_lock); if (_closed_senders.Get(sender_id)) { - // Dobule close from one sender, just return OK + // Double close from one sender, just return OK *finished = (_num_remaining_senders == 0); return _close_status; } diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index 496fa89131806c..193d4a79dc746d 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -34,6 +34,7 @@ #include "runtime/pull_load_task_mgr.h" #include "runtime/export_task_mgr.h" #include "runtime/result_buffer_mgr.h" +#include "runtime/routine_load/routine_load_task_executor.h" namespace doris { @@ -228,4 +229,20 @@ void BackendService::get_tablet_stat(TTabletStatResult& result) { OLAPEngine::get_instance()->get_tablet_stat(result); } +void BackendService::submit_routine_load_task( + TStatus& t_status, const std::vector& tasks) { + + for (auto& task : tasks) { + Status st = _exec_env->routine_load_task_executor()->submit_task(task); + if (!st.ok()) { + LOG(WARNING) << "failed to submit routine load task. job id: " << task.job_id + << " task id: " << task.id; + } + } + + // we do not care about each task's submit result. just return OK. + // FE will handle the failure. + return Status::OK.to_thrift(&t_status); +} + } // namespace doris diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h index 094be53c10cadf..51f9505a4a6c53 100644 --- a/be/src/service/backend_service.h +++ b/be/src/service/backend_service.h @@ -147,6 +147,8 @@ class BackendService : public BackendServiceIf { virtual void get_tablet_stat(TTabletStatResult& result) override; + virtual void submit_routine_load_task(TStatus& t_status, const std::vector& tasks) override; + private: Status start_plan_fragment_execution(const TExecPlanFragmentParams& exec_params); diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index ceec930eb7d81c..9ad546fd91db89 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -55,7 +55,7 @@ #include "service/http_service.h" #include #include "common/resource_tls.h" -#include "exec/schema_scanner/frontend_helper.h" +#include "util/frontend_helper.h" static void help(const char*); diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt index 3fa44dc6fa1059..541a178c86d927 100644 --- a/be/src/util/CMakeLists.txt +++ b/be/src/util/CMakeLists.txt @@ -42,7 +42,7 @@ add_library(Util STATIC parse_util.cpp path_builder.cpp # TODO: not supported on RHEL 5 -# perf-counters.cpp +# perf-counters.cpp progress_updater.cpp runtime_profile.cpp static_asserts.cpp @@ -67,13 +67,14 @@ add_library(Util STATIC null_load_error_hub.cpp time.cpp os_info.cpp -# coding_util.cpp +# coding_util.cpp cidr.cpp core_local.cpp uid_util.cpp aes_util.cpp string_util.cpp md5.cpp + frontend_helper.cpp ) #ADD_BE_TEST(integer-array-test) @@ -87,3 +88,4 @@ add_library(Util STATIC #ADD_BE_TEST(bit-util-test) #ADD_BE_TEST(rle-test) ##ADD_BE_TEST(perf-counters-test) +##ADD_BE_TEST(es-scan-reader-test) diff --git a/be/src/util/frontend_helper.cpp b/be/src/util/frontend_helper.cpp new file mode 100644 index 00000000000000..ddc36befc067c3 --- /dev/null +++ b/be/src/util/frontend_helper.cpp @@ -0,0 +1,91 @@ +// 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 "util/frontend_helper.h" + +#include + +#include +#include +#include +#include + +#include "gen_cpp/FrontendService_types.h" +#include "gen_cpp/FrontendService.h" +#include "runtime/runtime_state.h" +#include "runtime/exec_env.h" +#include "runtime/client_cache.h" +#include "util/network_util.h" +#include "util/thrift_util.h" +#include "util/runtime_profile.h" +#include "runtime/client_cache.h" + +namespace doris { + +ExecEnv* FrontendHelper::_s_exec_env; + +using apache::thrift::protocol::TProtocol; +using apache::thrift::protocol::TBinaryProtocol; +using apache::thrift::transport::TSocket; +using apache::thrift::transport::TTransport; +using apache::thrift::transport::TBufferedTransport; + +void FrontendHelper::setup(ExecEnv* exec_env) { + _s_exec_env = exec_env; +} + +Status FrontendHelper::rpc( + const std::string& ip, + const int32_t port, + std::function callback, + int timeout_ms) { + TNetworkAddress address = make_network_address(ip, port); + Status status; + FrontendServiceConnection client( + _s_exec_env->frontend_client_cache(), address, timeout_ms, &status); + if (!status.ok()) { + LOG(WARNING) << "Connect frontent failed, address=" << address + << ", status=" << status.get_error_msg(); + return status; + } + try { + try { + callback(client); + } catch (apache::thrift::transport::TTransportException& e) { + LOG(WARNING) << "retrying call frontend service, address=" + << address << ", reason=" << e.what(); + status = client.reopen(timeout_ms); + if (!status.ok()) { + LOG(WARNING) << "client repoen failed. address=" << address + << ", status=" << status.get_error_msg(); + return status; + } + callback(client); + } + } catch (apache::thrift::TException& e) { + // just reopen to disable this connection + client.reopen(timeout_ms); + LOG(WARNING) << "call frontend service failed, address=" << address + << ", reason=" << e.what(); + return Status(TStatusCode::THRIFT_RPC_ERROR, + "failed to call frontend service", false); + } + return Status::OK; +} + +} + diff --git a/be/src/util/frontend_helper.h b/be/src/util/frontend_helper.h new file mode 100644 index 00000000000000..d0ef6d0a8e4e00 --- /dev/null +++ b/be/src/util/frontend_helper.h @@ -0,0 +1,54 @@ +// 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 "common/status.h" +#include "gen_cpp/FrontendService_types.h" + +namespace doris { + +class ExecEnv; +class FrontendServiceClient; +template class ClientConnection; + +// this class is a helper for jni call. easy for unit test +class FrontendHelper { +public: + static void setup(ExecEnv* exec_env); + + // for default timeout + static Status rpc( + const std::string& ip, + const int32_t port, + std::function&)> callback) { + + return rpc(ip, port, callback, config::thrift_rpc_timeout_ms); + } + + static Status rpc( + const std::string& ip, + const int32_t port, + std::function&)> callback, + int timeout_ms); + +private: + static ExecEnv* _s_exec_env; +}; + +} + diff --git a/be/src/util/mysql_global.h b/be/src/util/mysql_global.h index 6586df597d6cdb..da51d90335e67d 100644 --- a/be/src/util/mysql_global.h +++ b/be/src/util/mysql_global.h @@ -47,9 +47,9 @@ typedef unsigned char uchar; #define MAX_DATETIME_WIDTH 19 /* YYYY-MM-DD HH:MM:SS */ /* -[digits].E+## */ -#define MAX_FLOAT_STR_LENGTH (FLT_DIG + 6) +#define MAX_FLOAT_STR_LENGTH 24 // see gutil/strings/numbers.h kFloatToBufferSize /* -[digits].E+### */ -#define MAX_DOUBLE_STR_LENGTH (DBL_DIG + 7) +#define MAX_DOUBLE_STR_LENGTH 32 // see gutil/strings/numbers.h kDoubleToBufferSize /* -[digits].[frac] */ #define MAX_DECIMAL_STR_LENGTH 29 diff --git a/be/src/util/semaphore.hpp b/be/src/util/semaphore.hpp new file mode 100644 index 00000000000000..398c5ae3ed6cab --- /dev/null +++ b/be/src/util/semaphore.hpp @@ -0,0 +1,50 @@ +// 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 + +namespace { + +class Semaphore { + public: + explicit Semaphore(int count = 0) : _count(count) { + } + + void set_count(int count) { _count = count; } + + void signal() { + std::unique_lock lock(_mutex); + ++count_; + cv_.notify_one(); + } + + void wait() { + std::unique_lock lock(_mutex); + cv_.wait(lock, [=] { return _count > 0; }); + --_count; + } + + private: + std::mutex _mutex; + std::condition_variable _cv; + int _count; +}; + +} // end namespace diff --git a/be/src/util/uid_util.h b/be/src/util/uid_util.h index 92e2fec1cf6304..d7a73979876c4d 100644 --- a/be/src/util/uid_util.h +++ b/be/src/util/uid_util.h @@ -59,7 +59,7 @@ struct UniqueId { std::string to_string() const { char buf[33]; to_hex(hi, buf); - buf[16] = ':'; + buf[16] = '-'; to_hex(lo, buf + 17); return {buf, 33}; } diff --git a/be/test/exec/CMakeLists.txt b/be/test/exec/CMakeLists.txt index 7b683602ce0b1c..79e33f5b8f516a 100644 --- a/be/test/exec/CMakeLists.txt +++ b/be/test/exec/CMakeLists.txt @@ -44,6 +44,10 @@ ADD_BE_TEST(broker_reader_test) ADD_BE_TEST(broker_scanner_test) ADD_BE_TEST(broker_scan_node_test) ADD_BE_TEST(es_scan_node_test) +ADD_BE_TEST(es_http_scan_node_test) +ADD_BE_TEST(es_predicate_test) +ADD_BE_TEST(es_query_builder_test) +ADD_BE_TEST(es_scan_reader_test) ADD_BE_TEST(olap_table_info_test) ADD_BE_TEST(olap_table_sink_test) #ADD_BE_TEST(schema_scan_node_test) diff --git a/be/test/exec/es_http_scan_node_test.cpp b/be/test/exec/es_http_scan_node_test.cpp new file mode 100644 index 00000000000000..e3fd63a9d2e64f --- /dev/null +++ b/be/test/exec/es_http_scan_node_test.cpp @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es_http_scan_node.h" + +#include +#include + +#include "common/object_pool.h" +#include "gen_cpp/PlanNodes_types.h" +#include "runtime/mem_pool.h" +#include "runtime/descriptors.h" +#include "runtime/runtime_state.h" +#include "runtime/row_batch.h" +#include "runtime/string_value.h" +#include "runtime/tuple_row.h" +#include "util/runtime_profile.h" +#include "util/debug_util.h" + +using std::vector; + +namespace doris { + +// mock +class EsHttpScanNodeTest : public testing::Test { +public: + EsHttpScanNodeTest() : _runtime_state("EsHttpScanNodeTest") { + _runtime_state._instance_mem_tracker.reset(new MemTracker()); + TDescriptorTable t_desc_table; + + // table descriptors + TTableDescriptor t_table_desc; + t_table_desc.id = 0; + t_table_desc.tableType = TTableType::ES_TABLE; + t_table_desc.numCols = 1; + t_table_desc.numClusteringCols = 0; + t_table_desc.__isset.esTable = true; + t_desc_table.tableDescriptors.push_back(t_table_desc); + t_desc_table.__isset.tableDescriptors = true; + + // TSlotDescriptor + int offset = 1; + int i = 0; + // id + { + TSlotDescriptor t_slot_desc; + t_slot_desc.__set_slotType(TypeDescriptor(TYPE_INT).to_thrift()); + t_slot_desc.__set_columnPos(i); + t_slot_desc.__set_byteOffset(offset); + t_slot_desc.__set_nullIndicatorByte(0); + t_slot_desc.__set_nullIndicatorBit(-1); + t_slot_desc.__set_slotIdx(i); + t_slot_desc.__set_isMaterialized(true); + t_desc_table.slotDescriptors.push_back(t_slot_desc); + offset += sizeof(int); + } + + TTupleDescriptor t_tuple_desc; + t_tuple_desc.id = 0; + t_tuple_desc.byteSize = offset; + t_tuple_desc.numNullBytes = 1; + t_tuple_desc.tableId = 0; + t_tuple_desc.__isset.tableId = true; + t_desc_table.__isset.slotDescriptors = true; + t_desc_table.tupleDescriptors.push_back(t_tuple_desc); + + DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl); + _runtime_state.set_desc_tbl(_desc_tbl); + + // Node Id + _tnode.node_id = 0; + _tnode.node_type = TPlanNodeType::SCHEMA_SCAN_NODE; + _tnode.num_children = 0; + _tnode.limit = -1; + _tnode.row_tuples.push_back(0); + _tnode.nullable_tuples.push_back(false); + _tnode.es_scan_node.tuple_id = 0; + std::map properties; + _tnode.es_scan_node.__set_properties(properties); + _tnode.__isset.es_scan_node = true; + } + +protected: + virtual void SetUp() { + } + virtual void TearDown() { + } + TPlanNode _tnode; + ObjectPool _obj_pool; + DescriptorTbl* _desc_tbl; + RuntimeState _runtime_state; +}; + +TEST_F(EsHttpScanNodeTest, normal_use) { + + EsHttpScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl); + Status status = scan_node.init(_tnode, &_runtime_state); + ASSERT_TRUE(status.ok()); + + status = scan_node.prepare(&_runtime_state); + ASSERT_TRUE(status.ok()); + + // scan range + TEsScanRange es_scan_range; + es_scan_range.__set_index("index1"); + es_scan_range.__set_type("docs"); + es_scan_range.__set_shard_id(0); + TNetworkAddress es_host; + es_host.__set_hostname("unknown"); + es_host.__set_port(8200); + std::vector es_hosts; + es_hosts.push_back(es_host); + es_scan_range.__set_es_hosts(es_hosts); + TScanRange scan_range; + scan_range.__set_es_scan_range(es_scan_range); + TScanRangeParams scan_range_params; + scan_range_params.__set_scan_range(scan_range); + std::vector scan_ranges; + scan_ranges.push_back(scan_range_params); + + status = scan_node.set_scan_ranges(scan_ranges); + ASSERT_TRUE(status.ok()); + + status = scan_node.open(&_runtime_state); + ASSERT_FALSE(status.ok()); + + status = scan_node.close(&_runtime_state); + ASSERT_TRUE(status.ok()); +} + +} + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + diff --git a/be/test/exec/es_predicate_test.cpp b/be/test/exec/es_predicate_test.cpp new file mode 100644 index 00000000000000..3c18bf1af4c454 --- /dev/null +++ b/be/test/exec/es_predicate_test.cpp @@ -0,0 +1,174 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/es/es_predicate.h" + +#include +#include +#include +#include "common/logging.h" +#include "common/status.h" +#include "exprs/binary_predicate.h" +#include "gen_cpp/Exprs_types.h" +#include "exec/es/es_query_builder.h" +#include "rapidjson/document.h" +#include "rapidjson/rapidjson.h" +#include "rapidjson/stringbuffer.h" +#include "rapidjson/writer.h" +#include "runtime/mem_tracker.h" +#include "runtime/primitive_type.h" +#include "runtime/runtime_state.h" +#include "runtime/string_value.h" + +namespace doris { + +class RuntimeState; + +class EsPredicateTest : public testing::Test { +public: + EsPredicateTest() : _runtime_state("EsPredicateTest") { + _runtime_state._instance_mem_tracker.reset(new MemTracker()); + TDescriptorTable t_desc_table; + + // table descriptors + TTableDescriptor t_table_desc; + t_table_desc.id = 0; + t_table_desc.tableType = TTableType::ES_TABLE; + t_table_desc.numCols = 1; + t_table_desc.numClusteringCols = 0; + t_table_desc.__isset.esTable = true; + t_desc_table.tableDescriptors.push_back(t_table_desc); + t_desc_table.__isset.tableDescriptors = true; + + // TSlotDescriptor + int offset = 1; + int i = 0; + // id + { + TSlotDescriptor t_slot_desc; + t_slot_desc.__set_slotType(TypeDescriptor(TYPE_INT).to_thrift()); + t_slot_desc.__set_columnPos(i); + t_slot_desc.__set_byteOffset(offset); + t_slot_desc.__set_nullIndicatorByte(0); + t_slot_desc.__set_nullIndicatorBit(-1); + t_slot_desc.__set_slotIdx(i); + t_slot_desc.__set_isMaterialized(true); + t_slot_desc.colName = "id"; + t_desc_table.slotDescriptors.push_back(t_slot_desc); + offset += sizeof(int); + } + + TTupleDescriptor t_tuple_desc; + t_tuple_desc.id = 0; + t_tuple_desc.byteSize = offset; + t_tuple_desc.numNullBytes = 1; + t_tuple_desc.tableId = 0; + t_tuple_desc.__isset.tableId = true; + t_desc_table.__isset.slotDescriptors = true; + t_desc_table.tupleDescriptors.push_back(t_tuple_desc); + + DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl); + _runtime_state.set_desc_tbl(_desc_tbl); + } + + Status build_expr_context_list(std::vector& conjunct_ctxs); + void init(); + void SetUp() override {} + void TearDown() override {} + +private: + + ObjectPool _obj_pool; + DescriptorTbl* _desc_tbl; + RuntimeState _runtime_state; +}; + +Status EsPredicateTest::build_expr_context_list(std::vector& conjunct_ctxs) { + + TExpr texpr; + { + TExprNode node0; + node0.opcode = TExprOpcode::GT; + node0.child_type = TPrimitiveType::BIGINT; + node0.node_type = TExprNodeType::BINARY_PRED; + node0.num_children = 2; + node0.__isset.opcode = true; + node0.__isset.child_type = true; + node0.type = gen_type_desc(TPrimitiveType::BOOLEAN); + texpr.nodes.emplace_back(node0); + + TExprNode node1; + node1.node_type = TExprNodeType::SLOT_REF; + node1.type = gen_type_desc(TPrimitiveType::INT); + node1.__isset.slot_ref = true; + node1.num_children = 0; + node1.slot_ref.slot_id = 0; + node1.slot_ref.tuple_id = 0; + node1.output_column = true; + node1.__isset.output_column = true; + texpr.nodes.emplace_back(node1); + + TExprNode node2; + TIntLiteral intLiteral; + intLiteral.value = 10; + node2.node_type = TExprNodeType::INT_LITERAL; + node2.type = gen_type_desc(TPrimitiveType::BIGINT); + node2.__isset.int_literal = true; + node2.int_literal = intLiteral; + texpr.nodes.emplace_back(node2); + } + + std::vector conjuncts; + conjuncts.emplace_back(texpr); + Status status = Expr::create_expr_trees(&_obj_pool, conjuncts, &conjunct_ctxs); + + return status; +} + +TEST_F(EsPredicateTest, normal) { + std::vector conjunct_ctxs; + Status status = build_expr_context_list(conjunct_ctxs); + + TupleDescriptor *tuple_desc = _desc_tbl->get_tuple_descriptor(0); + std::vector predicates; + for (int i = 0; i < conjunct_ctxs.size(); ++i) { + EsPredicate* predicate = new EsPredicate(conjunct_ctxs[i], tuple_desc); + if (predicate->build_disjuncts_list().ok()) { + predicates.push_back(predicate); + } + } + + rapidjson::Document document; + rapidjson::Value compound_bool_value(rapidjson::kObjectType); + compound_bool_value.SetObject(); + BooleanQueryBuilder::to_query(predicates, &document, &compound_bool_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + compound_bool_value.Accept(writer); + std::string actual_bool_json = buffer.GetString(); + std::string expected_json = "{\"bool\":{\"filter\":[{\"bool\":{\"should\":[{\"range\":{\"id\":{\"gt\":\"10\"}}}]}}]}}"; + LOG(INFO) << "compound bool query" << actual_bool_json; + ASSERT_STREQ(expected_json.c_str(), actual_bool_json.c_str()); +} + + +} // end namespace doris + +int main(int argc, char** argv) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/exec/es_query_builder_test.cpp b/be/test/exec/es_query_builder_test.cpp new file mode 100644 index 00000000000000..f6d7938a9ff702 --- /dev/null +++ b/be/test/exec/es_query_builder_test.cpp @@ -0,0 +1,459 @@ +// 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 +#include +#include + +#include "common/logging.h" +#include "exec/es/es_query_builder.h" +#include "exec/es/es_predicate.h" +#include "rapidjson/document.h" +#include "rapidjson/rapidjson.h" +#include "rapidjson/stringbuffer.h" +#include "rapidjson/writer.h" +#include "runtime/string_value.h" + +namespace doris { + +class BooleanQueryBuilderTest : public testing::Test { +public: + BooleanQueryBuilderTest() { } + virtual ~BooleanQueryBuilderTest() { } +}; + +TEST_F(BooleanQueryBuilderTest, term_query) { + // content = "wyf" + char str[] = "wyf"; + StringValue value(str, 3); + ExtLiteral term_literal(TYPE_VARCHAR, &value); + TypeDescriptor type_desc = TypeDescriptor::create_varchar_type(3); + std::string name = "content"; + ExtBinaryPredicate term_predicate(TExprNodeType::BINARY_PRED, name, type_desc, TExprOpcode::EQ, term_literal); + TermQueryBuilder term_query(term_predicate); + rapidjson::Document document; + rapidjson::Value term_value(rapidjson::kObjectType); + term_value.SetObject(); + term_query.to_json(&document, &term_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + term_value.Accept(writer); + std::string actual_json = buffer.GetString(); + //LOG(INFO) << "term query" << actual_json; + ASSERT_STREQ("{\"term\":{\"content\":\"wyf\"}}", actual_json.c_str()); +} + +TEST_F(BooleanQueryBuilderTest, range_query) { + // k >= a + char str[] = "a"; + StringValue value(str, 1); + ExtLiteral term_literal(TYPE_VARCHAR, &value); + TypeDescriptor type_desc = TypeDescriptor::create_varchar_type(1); + std::string name = "k"; + ExtBinaryPredicate range_predicate(TExprNodeType::BINARY_PRED, name, type_desc, TExprOpcode::GE, term_literal); + RangeQueryBuilder range_query(range_predicate); + rapidjson::Document document; + rapidjson::Value range_value(rapidjson::kObjectType); + range_value.SetObject(); + range_query.to_json(&document, &range_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + range_value.Accept(writer); + std::string actual_json = buffer.GetString(); + //LOG(INFO) << "range query" << actual_json; + ASSERT_STREQ("{\"range\":{\"k\":{\"ge\":\"a\"}}}", actual_json.c_str()); +} + +TEST_F(BooleanQueryBuilderTest, es_query) { + // esquery('random', "{\"bool\": {\"must_not\": {\"exists\": {\"field\": \"f1\"}}}}") + char str[] = "{\"bool\": {\"must_not\": {\"exists\": {\"field\": \"f1\"}}}}"; + int length = (int)strlen(str); + TypeDescriptor type_desc = TypeDescriptor::create_varchar_type(length); + std::string name = "random"; + ExtColumnDesc col_des(name, type_desc); + std::vector cols = {col_des}; + StringValue value(str, length); + ExtLiteral term_literal(TYPE_VARCHAR, &value); + std::vector values = {term_literal}; + std::string function_name = "esquery"; + ExtFunction function_predicate(TExprNodeType::FUNCTION_CALL, function_name, cols, values); + ESQueryBuilder es_query(function_predicate); + rapidjson::Document document; + rapidjson::Value es_query_value(rapidjson::kObjectType); + es_query_value.SetObject(); + es_query.to_json(&document, &es_query_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + es_query_value.Accept(writer); + std::string actual_json = buffer.GetString(); + //LOG(INFO) << "es query" << actual_json; + ASSERT_STREQ("{\"bool\":{\"must_not\":{\"exists\":{\"field\":\"f1\"}}}}", actual_json.c_str()); +} + +TEST_F(BooleanQueryBuilderTest, like_query) { + // content like 'a%e%g_' + char str[] = "a%e%g_"; + int length = (int)strlen(str); + LOG(INFO) << "length " << length; + TypeDescriptor type_desc = TypeDescriptor::create_varchar_type(length); + StringValue value(str, length); + ExtLiteral like_literal(TYPE_VARCHAR, &value); + std::string name = "content"; + ExtLikePredicate like_predicate(TExprNodeType::LIKE_PRED, name, type_desc, like_literal); + WildCardQueryBuilder like_query(like_predicate); + rapidjson::Document document; + rapidjson::Value like_query_value(rapidjson::kObjectType); + like_query_value.SetObject(); + like_query.to_json(&document, &like_query_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + like_query_value.Accept(writer); + std::string actual_json = buffer.GetString(); + // LOG(INFO) << "wildcard query" << actual_json; + ASSERT_STREQ("{\"wildcard\":{\"content\":\"a*e*g?\"}}", actual_json.c_str()); +} + +TEST_F(BooleanQueryBuilderTest, terms_in_query) { + // dv in ["2.0", "4.0", "8.0"] + std::string terms_in_field = "dv"; + int terms_in_field_length = terms_in_field.length(); + TypeDescriptor terms_in_col_type_desc = TypeDescriptor::create_varchar_type(terms_in_field_length); + + char value_1[] = "2.0"; + int value_1_length = (int)strlen(value_1); + StringValue string_value_1(value_1, value_1_length); + ExtLiteral term_literal_1(TYPE_VARCHAR, &string_value_1); + + char value_2[] = "4.0"; + int value_2_length = (int)strlen(value_2); + StringValue string_value_2(value_2, value_2_length); + ExtLiteral term_literal_2(TYPE_VARCHAR, &string_value_2); + + char value_3[] = "8.0"; + int value_3_length = (int)strlen(value_3); + StringValue string_value_3(value_3, value_3_length); + ExtLiteral term_literal_3(TYPE_VARCHAR, &string_value_3); + + std::vector terms_values = {term_literal_1, term_literal_2, term_literal_3}; + ExtInPredicate in_predicate(TExprNodeType::IN_PRED, false, terms_in_field, terms_in_col_type_desc, terms_values); + TermsInSetQueryBuilder terms_query(in_predicate); + rapidjson::Document document; + rapidjson::Value in_query_value(rapidjson::kObjectType); + in_query_value.SetObject(); + terms_query.to_json(&document, &in_query_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + in_query_value.Accept(writer); + std::string actual_json = buffer.GetString(); + //LOG(INFO) << "terms in sets query" << actual_json; + ASSERT_STREQ("{\"terms\":{\"dv\":[\"2.0\",\"4.0\",\"8.0\"]}}", actual_json.c_str()); +} + +TEST_F(BooleanQueryBuilderTest, match_all_query) { + // match all docs + MatchAllQueryBuilder match_all_query; + rapidjson::Document document; + rapidjson::Value match_all_query_value(rapidjson::kObjectType); + match_all_query_value.SetObject(); + match_all_query.to_json(&document, &match_all_query_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + match_all_query_value.Accept(writer); + std::string actual_json = buffer.GetString(); + //LOG(INFO) << "match all query" << actual_json; + ASSERT_STREQ("{\"match_all\":{}}", actual_json.c_str()); +} + + +TEST_F(BooleanQueryBuilderTest, bool_query) { + // content like 'a%e%g_' + char like_value[] = "a%e%g_"; + int like_value_length = (int)strlen(like_value); + TypeDescriptor like_type_desc = TypeDescriptor::create_varchar_type(like_value_length); + StringValue like_term_value(like_value, like_value_length); + ExtLiteral like_literal(TYPE_VARCHAR, &like_term_value); + std::string like_field_name = "content"; + ExtLikePredicate* like_predicate = new ExtLikePredicate(TExprNodeType::LIKE_PRED, like_field_name, like_type_desc, like_literal); + // esquery("random", "{\"bool\": {\"must_not\": {\"exists\": {\"field\": \"f1\"}}}}") + char es_query_str[] = "{\"bool\": {\"must_not\": {\"exists\": {\"field\": \"f1\"}}}}"; + int es_query_length = (int)strlen(es_query_str); + StringValue value(es_query_str, es_query_length); + TypeDescriptor es_query_type_desc = TypeDescriptor::create_varchar_type(es_query_length); + std::string es_query_field_name = "random"; + ExtColumnDesc es_query_col_des(es_query_field_name, es_query_type_desc); + std::vector es_query_cols = {es_query_col_des}; + StringValue es_query_value(es_query_str, es_query_length); + ExtLiteral es_query_term_literal(TYPE_VARCHAR, &es_query_value); + std::vector es_query_values = {es_query_term_literal}; + std::string function_name = "esquery"; + ExtFunction* function_predicate = new ExtFunction(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, es_query_values); + // k >= a + char range_value_str[] = "a"; + int range_value_length = (int)strlen(range_value_str); + StringValue range_value(range_value_str, range_value_length); + ExtLiteral range_literal(TYPE_VARCHAR, &range_value); + TypeDescriptor range_type_desc = TypeDescriptor::create_varchar_type(range_value_length); + std::string range_field_name = "k"; + ExtBinaryPredicate* range_predicate = new ExtBinaryPredicate(TExprNodeType::BINARY_PRED, range_field_name, range_type_desc, TExprOpcode::GE, range_literal); + // content = "wyf" + char term_str[] = "wyf"; + int term_value_length = (int)strlen(term_str); + StringValue term_value(term_str, term_value_length); + ExtLiteral term_literal(TYPE_VARCHAR, &term_value); + TypeDescriptor term_type_desc = TypeDescriptor::create_varchar_type(term_value_length); + std::string term_field_name = "content"; + ExtBinaryPredicate* term_predicate = new ExtBinaryPredicate(TExprNodeType::BINARY_PRED, term_field_name, term_type_desc, TExprOpcode::EQ, term_literal); + + // content like 'a%e%g_' or k >= a or content = "wyf" + std::vector or_predicates = {like_predicate, function_predicate, range_predicate, term_predicate}; + BooleanQueryBuilder bool_query(or_predicates); + rapidjson::Document document; + rapidjson::Value bool_query_value(rapidjson::kObjectType); + bool_query_value.SetObject(); + bool_query.to_json(&document, &bool_query_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + bool_query_value.Accept(writer); + std::string actual_json = buffer.GetString(); + std::string expected_json = "{\"bool\":{\"should\":[{\"wildcard\":{\"content\":\"a*e*g?\"}},{\"bool\":{\"must_not\":{\"exists\":{\"field\":\"f1\"}}}},{\"range\":{\"k\":{\"ge\":\"a\"}}},{\"term\":{\"content\":\"wyf\"}}]}}"; + //LOG(INFO) << "bool query" << actual_json; + ASSERT_STREQ(expected_json.c_str(), actual_json.c_str()); +} + +TEST_F(BooleanQueryBuilderTest, compound_bool_query) { + // content like "a%e%g_" or esquery(random, '{"bool": {"must_not": {"exists": {"field": "f1"}}}}') + char like_value[] = "a%e%g_"; + int like_value_length = (int)strlen(like_value); + TypeDescriptor like_type_desc = TypeDescriptor::create_varchar_type(like_value_length); + StringValue like_term_value(like_value, like_value_length); + ExtLiteral like_literal(TYPE_VARCHAR, &like_term_value); + std::string like_field_name = "content"; + ExtLikePredicate* like_predicate = new ExtLikePredicate(TExprNodeType::LIKE_PRED, like_field_name, like_type_desc, like_literal); + + char es_query_str[] = "{\"bool\": {\"must_not\": {\"exists\": {\"field\": \"f1\"}}}}"; + int es_query_length = (int)strlen(es_query_str); + StringValue value(es_query_str, es_query_length); + TypeDescriptor es_query_type_desc = TypeDescriptor::create_varchar_type(es_query_length); + std::string es_query_field_name = "random"; + ExtColumnDesc es_query_col_des(es_query_field_name, es_query_type_desc); + std::vector es_query_cols = {es_query_col_des}; + StringValue es_query_value(es_query_str, es_query_length); + ExtLiteral es_query_term_literal(TYPE_VARCHAR, &es_query_value); + std::vector es_query_values = {es_query_term_literal}; + std::string function_name = "esquery"; + ExtFunction* function_predicate = new ExtFunction(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, es_query_values); + std::vector bool_predicates_1 = {like_predicate, function_predicate}; + EsPredicate* bool_predicate_1 = new EsPredicate(bool_predicates_1); + + // k >= "a" + char range_value_str[] = "a"; + int range_value_length = (int)strlen(range_value_str); + StringValue range_value(range_value_str, range_value_length); + ExtLiteral range_literal(TYPE_VARCHAR, &range_value); + TypeDescriptor range_type_desc = TypeDescriptor::create_varchar_type(range_value_length); + std::string range_field_name = "k"; + ExtBinaryPredicate* range_predicate = new ExtBinaryPredicate(TExprNodeType::BINARY_PRED, range_field_name, range_type_desc, TExprOpcode::GE, range_literal); + + std::vector bool_predicates_2 = {range_predicate}; + EsPredicate* bool_predicate_2 = new EsPredicate(bool_predicates_2); + + // content != "wyf" + char term_str[] = "wyf"; + int term_value_length = (int)strlen(term_str); + StringValue term_value(term_str, term_value_length); + ExtLiteral term_literal(TYPE_VARCHAR, &term_value); + TypeDescriptor term_type_desc = TypeDescriptor::create_varchar_type(term_value_length); + std::string term_field_name = "content"; + ExtBinaryPredicate* term_ne_predicate = new ExtBinaryPredicate(TExprNodeType::BINARY_PRED, term_field_name, term_type_desc, TExprOpcode::NE, term_literal); + std::vector bool_predicates_3 = {term_ne_predicate}; + EsPredicate* bool_predicate_3 = new EsPredicate(bool_predicates_3); + + // fv not in [8.0, 16.0] + std::string terms_in_field = "fv"; + int terms_in_field_length = terms_in_field.length(); + TypeDescriptor terms_in_col_type_desc = TypeDescriptor::create_varchar_type(terms_in_field_length); + + char value_1[] = "8.0"; + int value_1_length = (int)strlen(value_1); + StringValue string_value_1(value_1, value_1_length); + ExtLiteral term_literal_1(TYPE_VARCHAR, &string_value_1); + + char value_2[] = "16.0"; + int value_2_length = (int)strlen(value_2); + StringValue string_value_2(value_2, value_2_length); + ExtLiteral term_literal_2(TYPE_VARCHAR, &string_value_2); + + std::vector terms_values = {term_literal_1, term_literal_2}; + ExtInPredicate* in_predicate = new ExtInPredicate(TExprNodeType::IN_PRED, true, terms_in_field, terms_in_col_type_desc, terms_values); + std::vector bool_predicates_4 = {in_predicate}; + EsPredicate* bool_predicate_4 = new EsPredicate(bool_predicates_4); + + // (content like "a%e%g_" or esquery(random, '{"bool": {"must_not": {"exists": {"field": "f1"}}}}')) and content != "wyf" and fv not in [8.0, 16.0] + std::vector and_bool_predicates = {bool_predicate_1, bool_predicate_2, bool_predicate_3, bool_predicate_4}; + + rapidjson::Document document; + rapidjson::Value compound_bool_value(rapidjson::kObjectType); + compound_bool_value.SetObject(); + BooleanQueryBuilder::to_query(and_bool_predicates, &document, &compound_bool_value); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + compound_bool_value.Accept(writer); + std::string actual_bool_json = buffer.GetString(); + std::string expected_json = "{\"bool\":{\"filter\":[{\"bool\":{\"should\":[{\"wildcard\":{\"content\":\"a*e*g?\"}},{\"bool\":{\"must_not\":{\"exists\":{\"field\":\"f1\"}}}}]}},{\"bool\":{\"should\":[{\"range\":{\"k\":{\"ge\":\"a\"}}}]}},{\"bool\":{\"should\":[{\"bool\":{\"must_not\":[{\"term\":{\"content\":\"wyf\"}}]}}]}},{\"bool\":{\"should\":[{\"bool\":{\"must_not\":[{\"terms\":{\"fv\":[\"8.0\",\"16.0\"]}}]}}]}}]}}"; + //LOG(INFO) << "compound bool query" << actual_bool_json; + ASSERT_STREQ(expected_json.c_str(), actual_bool_json.c_str()); +} +TEST_F(BooleanQueryBuilderTest, validate_esquery) { + std::string function_name = "esquery"; + char field[] = "random"; + int field_length = (int)strlen(field); + TypeDescriptor es_query_type_desc = TypeDescriptor::create_varchar_type(field_length); + ExtColumnDesc es_query_col_des(field, es_query_type_desc); + std::vector es_query_cols = {es_query_col_des}; + char es_query_str[] = "{\"bool\": {\"must_not\": {\"exists\": {\"field\": \"f1\"}}}}"; + int es_query_length = (int)strlen(es_query_str); + StringValue es_query_value(es_query_str, es_query_length); + ExtLiteral es_query_term_literal(TYPE_VARCHAR, &es_query_value); + std::vector es_query_values = {es_query_term_literal}; + ExtFunction legal_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, es_query_values); + auto st = BooleanQueryBuilder::check_es_query(legal_es_query); + ASSERT_TRUE(st.ok()); + char empty_query[] = "{}"; + int empty_query_length = (int)strlen(empty_query); + StringValue empty_query_value(empty_query, empty_query_length); + ExtLiteral empty_query_term_literal(TYPE_VARCHAR, &empty_query_value); + std::vector empty_query_values = {empty_query_term_literal}; + ExtFunction empty_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, empty_query_values); + st = BooleanQueryBuilder::check_es_query(empty_es_query); + ASSERT_STREQ(st.get_error_msg().c_str(), "esquery must only one root"); + //LOG(INFO) <<"error msg:" << st1.get_error_msg(); + char malformed_query[] = "{\"bool\": {\"must_not\": {\"exists\": {"; + int malformed_query_length = (int)strlen(malformed_query); + StringValue malformed_query_value(malformed_query, malformed_query_length); + ExtLiteral malformed_query_term_literal(TYPE_VARCHAR, &malformed_query_value); + std::vector malformed_query_values = {malformed_query_term_literal}; + ExtFunction malformed_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, malformed_query_values); + st = BooleanQueryBuilder::check_es_query(malformed_es_query); + ASSERT_STREQ(st.get_error_msg().c_str(), "malformed esquery json"); + char illegal_query[] = "{\"term\": {\"k1\" : \"2\"},\"match\": {\"k1\": \"3\"}}"; + int illegal_query_length = (int)strlen(illegal_query); + StringValue illegal_query_value(illegal_query, illegal_query_length); + ExtLiteral illegal_query_term_literal(TYPE_VARCHAR, &illegal_query_value); + std::vector illegal_query_values = {illegal_query_term_literal}; + ExtFunction illegal_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, illegal_query_values); + st = BooleanQueryBuilder::check_es_query(illegal_es_query); + ASSERT_STREQ(st.get_error_msg().c_str(), "esquery must only one root"); + char illegal_key_query[] = "[\"22\"]"; + int illegal_key_query_length = (int)strlen(illegal_key_query); + StringValue illegal_key_query_value(illegal_key_query, illegal_key_query_length); + ExtLiteral illegal_key_query_term_literal(TYPE_VARCHAR, &illegal_key_query_value); + std::vector illegal_key_query_values = {illegal_key_query_term_literal}; + ExtFunction illegal_key_es_query(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, illegal_key_query_values); + st = BooleanQueryBuilder::check_es_query(illegal_key_es_query); + ASSERT_STREQ(st.get_error_msg().c_str(), "esquery must be a object"); +} + +TEST_F(BooleanQueryBuilderTest, validate_partial) { + char like_value[] = "a%e%g_"; + int like_value_length = (int)strlen(like_value); + TypeDescriptor like_type_desc = TypeDescriptor::create_varchar_type(like_value_length); + StringValue like_term_value(like_value, like_value_length); + ExtLiteral like_literal(TYPE_VARCHAR, &like_term_value); + std::string like_field_name = "content"; + ExtLikePredicate* like_predicate = new ExtLikePredicate(TExprNodeType::LIKE_PRED, like_field_name, like_type_desc, like_literal); + + // k >= "a" + char range_value_str[] = "a"; + int range_value_length = (int)strlen(range_value_str); + StringValue range_value(range_value_str, range_value_length); + ExtLiteral range_literal(TYPE_VARCHAR, &range_value); + TypeDescriptor range_type_desc = TypeDescriptor::create_varchar_type(range_value_length); + std::string range_field_name = "k"; + ExtBinaryPredicate* range_predicate = new ExtBinaryPredicate(TExprNodeType::BINARY_PRED, range_field_name, range_type_desc, TExprOpcode::GE, range_literal); + + std::vector bool_predicates_1 = {like_predicate, range_predicate}; + EsPredicate* bool_predicate_1 = new EsPredicate(bool_predicates_1); + + // fv not in [8.0, 16.0] + std::string terms_in_field = "fv"; + int terms_in_field_length = terms_in_field.length(); + TypeDescriptor terms_in_col_type_desc = TypeDescriptor::create_varchar_type(terms_in_field_length); + + char value_1[] = "8.0"; + int value_1_length = (int)strlen(value_1); + StringValue string_value_1(value_1, value_1_length); + ExtLiteral term_literal_1(TYPE_VARCHAR, &string_value_1); + + char value_2[] = "16.0"; + int value_2_length = (int)strlen(value_2); + StringValue string_value_2(value_2, value_2_length); + ExtLiteral term_literal_2(TYPE_VARCHAR, &string_value_2); + + std::vector terms_values = {term_literal_1, term_literal_2}; + ExtInPredicate* in_predicate = new ExtInPredicate(TExprNodeType::IN_PRED, true, terms_in_field, terms_in_col_type_desc, terms_values); + std::vector bool_predicates_2 = {in_predicate}; + EsPredicate* bool_predicate_2 = new EsPredicate(bool_predicates_2); + + // content != "wyf" + char term_str[] = "wyf"; + int term_value_length = (int)strlen(term_str); + StringValue term_value(term_str, term_value_length); + ExtLiteral term_literal(TYPE_VARCHAR, &term_value); + TypeDescriptor term_type_desc = TypeDescriptor::create_varchar_type(term_value_length); + std::string term_field_name = "content"; + ExtBinaryPredicate* term_ne_predicate = new ExtBinaryPredicate(TExprNodeType::BINARY_PRED, term_field_name, term_type_desc, TExprOpcode::NE, term_literal); + + char es_query_str[] = "{\"bool\": {\"must_not\": {\"exists\": {\"field\": \"f1\"}}}}"; + int es_query_length = (int)strlen(es_query_str); + StringValue value(es_query_str, es_query_length); + TypeDescriptor es_query_type_desc = TypeDescriptor::create_varchar_type(es_query_length); + std::string es_query_field_name = "random"; + ExtColumnDesc es_query_col_des(es_query_field_name, es_query_type_desc); + std::vector es_query_cols = {es_query_col_des}; + StringValue es_query_value(es_query_str, es_query_length); + ExtLiteral es_query_term_literal(TYPE_VARCHAR, &es_query_value); + std::vector es_query_values = {es_query_term_literal}; + std::string function_name = "esquery"; + ExtFunction* function_predicate = new ExtFunction(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, es_query_values); + std::vector bool_predicates_3 = {term_ne_predicate, function_predicate}; + EsPredicate* bool_predicate_3 = new EsPredicate(bool_predicates_3); + + std::vector and_bool_predicates = {bool_predicate_1, bool_predicate_2, bool_predicate_3}; + std::vector result; + BooleanQueryBuilder::validate(and_bool_predicates, &result); + std::vector expected = {true, true, true}; + ASSERT_TRUE(result == expected); + char illegal_query[] = "{\"term\": {\"k1\" : \"2\"},\"match\": {\"k1\": \"3\"}}"; + int illegal_query_length = (int)strlen(illegal_query); + StringValue illegal_query_value(illegal_query, illegal_query_length); + ExtLiteral illegal_query_term_literal(TYPE_VARCHAR, &illegal_query_value); + std::vector illegal_query_values = {illegal_query_term_literal}; + ExtFunction* illegal_function_preficate = new ExtFunction(TExprNodeType::FUNCTION_CALL, function_name, es_query_cols, illegal_query_values); + std::vector illegal_bool_predicates_3 = {term_ne_predicate, illegal_function_preficate}; + EsPredicate* illegal_bool_predicate_3 = new EsPredicate(illegal_bool_predicates_3); + std::vector and_bool_predicates_1 = {bool_predicate_1, bool_predicate_2, illegal_bool_predicate_3}; + std::vector result1; + BooleanQueryBuilder::validate(and_bool_predicates_1, &result1); + std::vector expected1 = {true, true, false}; + ASSERT_TRUE(result1 == expected1); +} +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/exec/es_scan_reader_test.cpp b/be/test/exec/es_scan_reader_test.cpp new file mode 100644 index 00000000000000..2da96c025f8355 --- /dev/null +++ b/be/test/exec/es_scan_reader_test.cpp @@ -0,0 +1,245 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include + +#include "common/logging.h" +#include "exec/es/es_scan_reader.h" +#include "exec/es/es_scroll_query.h" +#include "http/ev_http_server.h" +#include "http/http_channel.h" +#include "http/http_handler.h" +#include "http/http_request.h" +#include "rapidjson/document.h" +#include "rapidjson/writer.h" +#include "rapidjson/stringbuffer.h" + +namespace doris { + +class RestSearchAction : public HttpHandler { +public: + void handle(HttpRequest* req) override { + std::string user; + std::string passwd; + if (!parse_basic_auth(*req, &user, &passwd) || user != "root") { + HttpChannel::send_basic_challenge(req, "abc"); + return; + } + req->add_output_header(HttpHeaders::CONTENT_TYPE, "application/json"); + if (req->method() == HttpMethod::POST) { + std::string post_body = req->get_request_body(); + rapidjson::Document post_doc; + post_doc.Parse<0>(post_body.c_str()); + int size = 1; + if (post_doc.HasMember("size")) { + rapidjson::Value& size_value = post_doc["size"]; + size = size_value.GetInt(); + } + std::string _scroll_id(std::to_string(size)); + rapidjson::Document search_result; + rapidjson::Document::AllocatorType &allocator = search_result.GetAllocator(); + search_result.SetObject(); + rapidjson::Value scroll_id_value(_scroll_id.c_str(), allocator); + search_result.AddMember("_scroll_id", scroll_id_value, allocator); + + rapidjson::Value outer_hits(rapidjson::kObjectType); + outer_hits.AddMember("total", 10, allocator); + rapidjson::Value inner_hits(rapidjson::kArrayType); + rapidjson::Value source_docuement(rapidjson::kObjectType); + source_docuement.AddMember("id", 1, allocator); + rapidjson::Value value_node("1", allocator); + source_docuement.AddMember("value", value_node, allocator); + inner_hits.PushBack(source_docuement, allocator); + outer_hits.AddMember("hits", inner_hits, allocator); + search_result.AddMember("hits", outer_hits, allocator); + + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + search_result.Accept(writer); + //send DELETE scorll post request + std::string search_result_json = buffer.GetString(); + HttpChannel::send_reply(req, search_result_json); + } else { + std::string response = "test1"; + HttpChannel::send_reply(req, response); + } + } +}; + +class RestSearchScrollAction : public HttpHandler { +public: + void handle(HttpRequest* req) override { + std::string user; + std::string passwd; + if (!parse_basic_auth(*req, &user, &passwd) || user != "root") { + HttpChannel::send_basic_challenge(req, "abc"); + return; + } + if (req->method() == HttpMethod::POST) { + std::string post_body = req->get_request_body(); + rapidjson::Document post_doc; + post_doc.Parse<0>(post_body.c_str()); + std::string scroll_id; + if (!post_doc.HasMember("scroll_id")) { + HttpChannel::send_reply(req,HttpStatus::NOT_FOUND, "invalid scroll request"); + return; + } else { + rapidjson::Value& scroll_id_value = post_doc["scroll_id"]; + scroll_id = scroll_id_value.GetString(); + int offset = atoi(scroll_id.c_str()); + if (offset > 10) { + rapidjson::Document end_search_result; + rapidjson::Document::AllocatorType &allocator = end_search_result.GetAllocator(); + end_search_result.SetObject(); + rapidjson::Value scroll_id_value("11", allocator); + end_search_result.AddMember("_scroll_id", scroll_id_value, allocator); + + rapidjson::Value outer_hits(rapidjson::kObjectType); + outer_hits.AddMember("total", 10, allocator); + end_search_result.AddMember("hits", outer_hits, allocator); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + end_search_result.Accept(writer); + //send DELETE scorll post request + std::string end_search_result_json = buffer.GetString(); + HttpChannel::send_reply(req, end_search_result_json); + return; + } else { + int start = offset + 1; + rapidjson::Document search_result; + rapidjson::Document::AllocatorType &allocator = search_result.GetAllocator(); + search_result.SetObject(); + rapidjson::Value scroll_id_value(std::to_string(start).c_str(), allocator); + search_result.AddMember("_scroll_id", scroll_id_value, allocator); + + rapidjson::Value outer_hits(rapidjson::kObjectType); + outer_hits.AddMember("total", 10, allocator); + rapidjson::Value inner_hits(rapidjson::kArrayType); + rapidjson::Value source_docuement(rapidjson::kObjectType); + source_docuement.AddMember("id", start, allocator); + rapidjson::Value value_node(std::to_string(start).c_str(), allocator); + source_docuement.AddMember("value", value_node, allocator); + inner_hits.PushBack(source_docuement, allocator); + outer_hits.AddMember("hits", inner_hits, allocator); + search_result.AddMember("hits", outer_hits, allocator); + + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + search_result.Accept(writer); + //send DELETE scorll post request + std::string search_result_json = buffer.GetString(); + HttpChannel::send_reply(req, search_result_json); + return; + } + + } + } + } +}; + +class RestClearScrollAction : public HttpHandler { +public: + void handle(HttpRequest* req) override { + std::string user; + std::string passwd; + if (!parse_basic_auth(*req, &user, &passwd) || user != "root") { + HttpChannel::send_basic_challenge(req, "abc"); + return; + } + if (req->method() == HttpMethod::DELETE) { + std::string post_body = req->get_request_body(); + rapidjson::Document post_doc; + post_doc.Parse<0>(post_body.c_str()); + std::string scroll_id; + if (!post_doc.HasMember("scroll_id")) { + HttpChannel::send_reply(req,HttpStatus::NOT_FOUND, "invalid scroll request"); + return; + } else { + rapidjson::Document clear_scroll_result; + rapidjson::Document::AllocatorType &allocator = clear_scroll_result.GetAllocator(); + clear_scroll_result.SetObject(); + clear_scroll_result.AddMember("succeeded", true, allocator); + clear_scroll_result.AddMember("num_freed", 1, allocator); + rapidjson::StringBuffer buffer; + rapidjson::Writer writer(buffer); + clear_scroll_result.Accept(writer); + std::string clear_scroll_result_json = buffer.GetString(); + HttpChannel::send_reply(req, clear_scroll_result_json); + return; + } + } + } +}; + +static RestSearchAction rest_search_action = RestSearchAction(); +static RestSearchScrollAction rest_search_scroll_action = RestSearchScrollAction(); +static RestClearScrollAction rest_clear_scroll_action = RestClearScrollAction(); +static EvHttpServer* mock_es_server = nullptr; + +class MockESServerTest : public testing::Test { +public: + MockESServerTest() { } + ~MockESServerTest() override { } + + static void SetUpTestCase() { + mock_es_server = new EvHttpServer(29386); + mock_es_server->register_handler(POST, "/{index}/{type}/_search", &rest_search_action); + mock_es_server->register_handler(POST, "/_search/scroll", &rest_search_scroll_action); + mock_es_server->register_handler(DELETE, "/_search/scroll", &rest_clear_scroll_action); + mock_es_server->start(); + } + + static void TearDownTestCase() { + delete mock_es_server; + } +}; + +TEST_F(MockESServerTest, workflow) { + std::string target = "http://127.0.0.1:29386"; + std::vector fields = {"id", "value"}; + std::map props; + props[ESScanReader::KEY_INDEX] = "tindex"; + props[ESScanReader::KEY_TYPE] = "doc"; + props[ESScanReader::KEY_USER_NAME] = "root"; + props[ESScanReader::KEY_PASS_WORD] = "root"; + props[ESScanReader::KEY_SHARD] = "0"; + props[ESScanReader::KEY_BATCH_SIZE] = "1"; + std::vector predicates; + props[ESScanReader::KEY_QUERY] = ESScrollQueryBuilder::build(props, fields, predicates); + ESScanReader reader(target, props); + auto st = reader.open(); + // ASSERT_TRUE(st.ok()); + bool eos = false; + std::unique_ptr parser = nullptr; + while(!eos){ + st = reader.get_next(&eos, parser); + if(eos) { + break; + } + } + auto cst = reader.close(); + ASSERT_TRUE(cst.ok()); +} +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/exec/olap_table_sink_test.cpp b/be/test/exec/olap_table_sink_test.cpp index 8f4bbab9c5a3df..8709d1ad4773ee 100644 --- a/be/test/exec/olap_table_sink_test.cpp +++ b/be/test/exec/olap_table_sink_test.cpp @@ -23,11 +23,11 @@ #include "gen_cpp/internal_service.pb.h" #include "runtime/decimal_value.h" #include "runtime/exec_env.h" -#include "runtime/load_stream_mgr.h" #include "runtime/row_batch.h" #include "runtime/runtime_state.h" #include "runtime/thread_resource_mgr.h" #include "runtime/tuple_row.h" +#include "runtime/stream_load/load_stream_mgr.h" #include "service/brpc.h" #include "util/brpc_stub_cache.h" #include "util/cpu_info.h" diff --git a/be/test/exec/plain_text_line_reader_uncompressed_test.cpp b/be/test/exec/plain_text_line_reader_uncompressed_test.cpp index eff25b110e0296..7641a3675c2ecd 100644 --- a/be/test/exec/plain_text_line_reader_uncompressed_test.cpp +++ b/be/test/exec/plain_text_line_reader_uncompressed_test.cpp @@ -272,6 +272,27 @@ TEST_F(PlainTextLineReaderTest, uncompressed_test_limit5) { ASSERT_TRUE(eof); } +TEST_F(PlainTextLineReaderTest, uncompressed_test_empty) { + LocalFileReader file_reader("./be/test/exec/test_data/plain_text_line_reader/empty.txt", 0); + auto st = file_reader.open(); + ASSERT_TRUE(st.ok()); + + Decompressor* decompressor; + st = Decompressor::create_decompressor(CompressType::UNCOMPRESSED, &decompressor); + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(decompressor == nullptr); + + // set min length larger than 0 to test + PlainTextLineReader line_reader(&_profile, &file_reader, decompressor, 10, '\n'); + const uint8_t* ptr; + size_t size; + bool eof; + + st = line_reader.read_line(&ptr, &size, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(eof); +} + } // end namespace doris int main(int argc, char** argv) { diff --git a/be/test/exec/test_data/plain_text_line_reader/empty.txt b/be/test/exec/test_data/plain_text_line_reader/empty.txt new file mode 100644 index 00000000000000..e69de29bb2d1d6 diff --git a/be/test/http/http_client_test.cpp b/be/test/http/http_client_test.cpp index 343c60614466a8..e75a299142cf21 100644 --- a/be/test/http/http_client_test.cpp +++ b/be/test/http/http_client_test.cpp @@ -19,6 +19,7 @@ #include +#include "boost/algorithm/string.hpp" #include "common/logging.h" #include "http/ev_http_server.h" #include "http/http_channel.h" @@ -151,6 +152,20 @@ TEST_F(HttpClientTest, post_normal) { ASSERT_STREQ(response.c_str(), request_body.c_str()); } +TEST_F(HttpClientTest, post_failed) { + HttpClient client; + auto st = client.init("http://127.0.0.1:29386/simple_pos"); + ASSERT_TRUE(st.ok()); + client.set_method(POST); + client.set_basic_auth("test1", ""); + std::string response; + std::string request_body = "simple post body query"; + st = client.execute_post_request(request_body, &response); + ASSERT_FALSE(st.ok()); + std::string not_found = "404"; + ASSERT_TRUE(boost::algorithm::contains(st.get_error_msg(), not_found)); +} + } int main(int argc, char* argv[]) { diff --git a/be/test/http/message_body_sink_test.cpp b/be/test/http/message_body_sink_test.cpp index 145b71a6f69e6c..cf71f1344301de 100644 --- a/be/test/http/message_body_sink_test.cpp +++ b/be/test/http/message_body_sink_test.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "http/message_body_sink.h" +#include "runtime/message_body_sink.h" #include diff --git a/be/test/http/stream_load_test.cpp b/be/test/http/stream_load_test.cpp index a4086f29d7563e..3090ae42edd2b7 100644 --- a/be/test/http/stream_load_test.cpp +++ b/be/test/http/stream_load_test.cpp @@ -23,12 +23,13 @@ #include #include -#include "exec/schema_scanner/frontend_helper.h" +#include "exec/schema_scanner/schema_helper.h" #include "gen_cpp/HeartbeatService_types.h" #include "http/http_channel.h" #include "http/http_request.h" #include "runtime/exec_env.h" -#include "runtime/load_stream_mgr.h" +#include "runtime/stream_load/load_stream_mgr.h" +#include "runtime/stream_load/stream_load_executor.h" #include "runtime/thread_resource_mgr.h" #include "util/brpc_stub_cache.h" #include "util/cpu_info.h" @@ -81,6 +82,7 @@ class StreamLoadActionTest : public testing::Test { _env._master_info = new TMasterInfo(); _env._load_stream_mgr = new LoadStreamMgr(); _env._brpc_stub_cache = new BrpcStubCache(); + _env._stream_load_executor = new StreamLoadExecutor(&_env); _evhttp_req = evhttp_request_new(nullptr, nullptr); } @@ -93,6 +95,8 @@ class StreamLoadActionTest : public testing::Test { _env._master_info = nullptr; delete _env._thread_mgr; _env._thread_mgr = nullptr; + delete _env._stream_load_executor; + _env._stream_load_executor = nullptr; if (_evhttp_req != nullptr) { evhttp_request_free(_evhttp_req); diff --git a/be/test/runtime/CMakeLists.txt b/be/test/runtime/CMakeLists.txt index 09758663078ec2..dbb3068853d548 100644 --- a/be/test/runtime/CMakeLists.txt +++ b/be/test/runtime/CMakeLists.txt @@ -56,3 +56,5 @@ ADD_BE_TEST(tablet_writer_mgr_test) #ADD_BE_TEST(export_task_mgr_test) ADD_BE_TEST(snapshot_loader_test) ADD_BE_TEST(user_function_cache_test) +ADD_BE_TEST(kafka_consumer_pipe_test) +ADD_BE_TEST(routine_load_task_executor_test) diff --git a/be/test/runtime/decimal_value_test.cpp b/be/test/runtime/decimal_value_test.cpp index fa8242d2a8400a..6249583d7690e9 100755 --- a/be/test/runtime/decimal_value_test.cpp +++ b/be/test/runtime/decimal_value_test.cpp @@ -257,6 +257,14 @@ TEST_F(DecimalValueTest, mul) { std::cout << "mul_result2: " << mul_result2.get_debug_info() << std::endl; ASSERT_EQ(DecimalValue(std::string("0")), mul_result2); + { + // test when carry is needed + DecimalValue value1(std::string("3074062.5421333313")); + DecimalValue value2(std::string("2169.957745029689045693")); + DecimalValue mul_result = value1 * value2; + std::cout << "mul_result=" << mul_result.get_debug_info() << std::endl; + ASSERT_EQ(DecimalValue(std::string("6670585822.0078770603624547106640070909")), mul_result); + } } TEST_F(DecimalValueTest, div) { diff --git a/be/test/runtime/kafka_consumer_pipe_test.cpp b/be/test/runtime/kafka_consumer_pipe_test.cpp new file mode 100644 index 00000000000000..5cea24f863a904 --- /dev/null +++ b/be/test/runtime/kafka_consumer_pipe_test.cpp @@ -0,0 +1,78 @@ +// 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 "runtime/routine_load/kafka_consumer_pipe.h" + +#include + +namespace doris { + +class KafkaConsumerPipeTest : public testing::Test { +public: + KafkaConsumerPipeTest() { } + virtual ~ KafkaConsumerPipeTest() { } + + void SetUp() override { + + + } + + void TearDown() override { + + } + +private: + +}; + +TEST_F(KafkaConsumerPipeTest, append_read) { + KafkaConsumerPipe k_pipe(1024 * 1024, 64 * 1024); + + std::string msg1 = "i have a dream"; + std::string msg2 = "This is from kafka"; + + Status st; + st = k_pipe.append_with_line_delimiter(msg1.c_str(), msg1.length()); + ASSERT_TRUE(st.ok()); + st = k_pipe.append_with_line_delimiter(msg2.c_str(), msg2.length()); + ASSERT_TRUE(st.ok()); + st = k_pipe.finish(); + ASSERT_TRUE(st.ok()); + + char buf[1024]; + size_t data_size = 1024; + bool eof = false; + st = k_pipe.read((uint8_t*) buf, &data_size, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(data_size, msg1.length() + msg2.length() + 2); + ASSERT_EQ(eof, false); + + data_size = 1024; + st = k_pipe.read((uint8_t*) buf, &data_size, &eof); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(data_size, 0); + ASSERT_EQ(eof, true); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + doris::CpuInfo::init(); + return RUN_ALL_TESTS(); +} + diff --git a/be/test/runtime/routine_load_task_executor_test.cpp b/be/test/runtime/routine_load_task_executor_test.cpp new file mode 100644 index 00000000000000..6cf0fa7bf32bbc --- /dev/null +++ b/be/test/runtime/routine_load_task_executor_test.cpp @@ -0,0 +1,134 @@ +// 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 "runtime/routine_load/routine_load_task_executor.h" + +#include "runtime/exec_env.h" +#include "runtime/stream_load/load_stream_mgr.h" +#include "runtime/stream_load/stream_load_executor.h" +#include "util/logging.h" + +#include + +#include "gen_cpp/BackendService_types.h" +#include "gen_cpp/FrontendService_types.h" +#include "gen_cpp/HeartbeatService_types.h" + +namespace doris { + +using namespace RdKafka; + +extern TLoadTxnBeginResult k_stream_load_begin_result; +extern TLoadTxnCommitResult k_stream_load_commit_result; +extern TLoadTxnRollbackResult k_stream_load_rollback_result; +extern TStreamLoadPutResult k_stream_load_put_result; + +class RoutineLoadTaskExecutorTest : public testing::Test { +public: + RoutineLoadTaskExecutorTest() { } + virtual ~RoutineLoadTaskExecutorTest() { } + + void SetUp() override { + k_stream_load_begin_result = TLoadTxnBeginResult(); + k_stream_load_commit_result = TLoadTxnCommitResult(); + k_stream_load_rollback_result = TLoadTxnRollbackResult(); + k_stream_load_put_result = TStreamLoadPutResult(); + + _env._master_info = new TMasterInfo(); + _env._load_stream_mgr = new LoadStreamMgr(); + _env._stream_load_executor = new StreamLoadExecutor(&_env); + } + + void TearDown() override { + delete _env._master_info; + _env._master_info = nullptr; + delete _env._load_stream_mgr; + _env._load_stream_mgr = nullptr; + delete _env._stream_load_executor; + _env._stream_load_executor = nullptr; + } + +private: + ExecEnv _env; +}; + +TEST_F(RoutineLoadTaskExecutorTest, exec_task) { + TRoutineLoadTask task; + task.type = TLoadSourceType::KAFKA; + task.job_id = 1L; + task.id = TUniqueId(); + task.txn_id = 4; + task.auth_code = 5; + task.__set_db("db1"); + task.__set_tbl("tbl1"); + task.__set_label("l1"); + task.__set_max_interval_s(5); + task.__set_max_batch_rows(10); + task.__set_max_batch_size(2048); + + TKafkaLoadInfo k_info; + k_info.brokers = "127.0.0.1:9092"; + k_info.topic = "test"; + + std::map part_off; + part_off[0] = 13L; + k_info.__set_partition_begin_offset(part_off); + + task.__set_kafka_load_info(k_info); + + RoutineLoadTaskExecutor executor(&_env); + + // submit task + Status st; + st = executor.submit_task(task); + ASSERT_TRUE(st.ok()); + + sleep(2); + k_info.brokers = "127.0.0.1:9092"; + task.__set_kafka_load_info(k_info); + st = executor.submit_task(task); + ASSERT_TRUE(st.ok()); + + sleep(2); + k_info.brokers = "192.0.0.2:9092"; + task.__set_kafka_load_info(k_info); + st = executor.submit_task(task); + ASSERT_TRUE(st.ok()); + + sleep(2); + k_info.brokers = "192.0.0.2:9092"; + task.__set_kafka_load_info(k_info); + st = executor.submit_task(task); + ASSERT_TRUE(st.ok()); + + sleep(2); +} + +} // end namespace + +int main(int argc, char* argv[]) { + std::string conffile = std::string(getenv("DORIS_HOME")) + "/conf/be.conf"; + if (!doris::config::init(conffile.c_str(), false)) { + fprintf(stderr, "error read config file. \n"); + return -1; + } + doris::init_glog("be-test"); + + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + diff --git a/be/test/runtime/stream_load_pipe_test.cpp b/be/test/runtime/stream_load_pipe_test.cpp index bed70392a1662e..175d5338632c88 100644 --- a/be/test/runtime/stream_load_pipe_test.cpp +++ b/be/test/runtime/stream_load_pipe_test.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "runtime/stream_load_pipe.h" +#include "runtime/stream_load/stream_load_pipe.h" #include diff --git a/be/test/util/uid_util_test.cpp b/be/test/util/uid_util_test.cpp index de00b776390487..2175fc917a652e 100644 --- a/be/test/util/uid_util_test.cpp +++ b/be/test/util/uid_util_test.cpp @@ -32,12 +32,12 @@ TEST_F(UidUtilTest, UniqueId) { { UniqueId id; std::string hex_str = id.to_string(); - ASSERT_STRNE("0000000000000000:0000000000000000", hex_str.c_str()); + ASSERT_STRNE("0000000000000000-0000000000000000", hex_str.c_str()); } { UniqueId id(123456789, 987654321); std::string hex_str = id.to_string(); - ASSERT_STREQ("00000000075bcd15:000000003ade68b1", hex_str.c_str()); + ASSERT_STREQ("00000000075bcd15-000000003ade68b1", hex_str.c_str()); } { PUniqueId puid; @@ -45,7 +45,7 @@ TEST_F(UidUtilTest, UniqueId) { puid.set_lo(98765432123456789); UniqueId id(puid); std::string hex_str = id.to_string(); - ASSERT_STREQ("002bdc546291f4b1:015ee2a321ce7d15", hex_str.c_str()); + ASSERT_STREQ("002bdc546291f4b1-015ee2a321ce7d15", hex_str.c_str()); } { TUniqueId tuid; @@ -53,7 +53,7 @@ TEST_F(UidUtilTest, UniqueId) { tuid.__set_lo(98765432123456789); UniqueId id(tuid); std::string hex_str = id.to_string(); - ASSERT_STREQ("002bdc546291f4b1:015ee2a321ce7d15", hex_str.c_str()); + ASSERT_STREQ("002bdc546291f4b1-015ee2a321ce7d15", hex_str.c_str()); } { TUniqueId tuid; @@ -61,7 +61,7 @@ TEST_F(UidUtilTest, UniqueId) { tuid.__set_lo(98765432123456789); std::stringstream ss; ss << UniqueId(tuid); - ASSERT_STREQ("002bdc546291f4b1:015ee2a321ce7d15", ss.str().c_str()); + ASSERT_STREQ("002bdc546291f4b1-015ee2a321ce7d15", ss.str().c_str()); } } diff --git a/build.sh b/build.sh index 76685f7e07bbea..0217675cb35a3d 100755 --- a/build.sh +++ b/build.sh @@ -141,10 +141,7 @@ fi # Build docs, should be built before Frontend echo "Build docs" cd ${DORIS_HOME}/docs -if [ ${CLEAN} -eq 1 ]; then - make clean -fi -make +make clean && make cd ${DORIS_HOME} # Clean and build Frontend diff --git a/conf/be.conf b/conf/be.conf index 3234a479f88c91..8807bca542e946 100644 --- a/conf/be.conf +++ b/conf/be.conf @@ -29,6 +29,6 @@ storage_root_path = /home/disk1/palo;/home/disk2/palo # sys_log_dir = ${DORIS_HOME}/log # sys_log_roll_mode = SIZE-MB-1024 # sys_log_roll_num = 10 -# sys_log_verbose_modules = +# sys_log_verbose_modules = * # log_buffer_level = -1 # palo_cgroups diff --git a/docs/documentation/cn/administrator-guide/load-data/routine-load-manual.md b/docs/documentation/cn/administrator-guide/load-data/routine-load-manual.md new file mode 100644 index 00000000000000..f6b20913b623ae --- /dev/null +++ b/docs/documentation/cn/administrator-guide/load-data/routine-load-manual.md @@ -0,0 +1,173 @@ +# 例行导入使用手册 + +例行导入(Routine Load)功能为用户提供了一种自动从指定数据源进行数据导入的功能。 + +本文档主要介绍该功能的实现原理、使用方式以及最佳实践。 + +## 名词解释 + +* FE:Frontend,Doris 的前端节点。负责元数据管理和请求接入。 +* BE:Backend,Doris 的后端节点。负责查询执行和数据存储。 +* RoutineLoadJob:用户提交的一个例行导入作业。 +* JobScheduler:例行导入作业调度器,用于调度和拆分一个 RoutineLoadJob 为多个 Task。 +* Task:RoutineLoadJob 被 JobScheduler 根据规则拆分的子任务。 +* TaskScheduler:任务调度器。用于调度 Task 的执行。 + +## 原理 + +``` + +---------+ + | Client | + +----+----+ + | ++-----------------------------+ +| FE | | +| +-----------v------------+ | +| | | | +| | Routine Load Job | | +| | | | +| +---+--------+--------+--+ | +| | | | | +| +---v--+ +---v--+ +---v--+ | +| | task | | task | | task | | +| +--+---+ +---+--+ +---+--+ | +| | | | | ++-----------------------------+ + | | | + v v v + +---+--+ +--+---+ ++-----+ + | BE | | BE | | BE | + +------+ +------+ +------+ + +``` + +如上图,Client 向 FE 提交一个例行导入作业。 + +FE 通过 JobScheduler 将一个导入作业拆分成若干个 Task。每个 Task 负责导入指定的一部分数据。Task 被 TaskScheduler 分配到指定的 BE 上执行。 + +在 BE 上,一个 Task 被视为一个普通的导入任务,通过 Stream Load 的导入机制进行导入。导入完成后,向 FE 汇报。 + +FE 中的 JobScheduler 根据汇报结果,继续生成后续新的 Task,或者对失败的 Task 进行重试。 + +整个例行导入作业通过不断的产生新的 Task,来完成数据不间断的导入。 + +## Kafka 例行导入 + +当前我们仅支持从 Kafka 系统进行例行导入。该部分会详细介绍 Kafka 例行导入使用方式和最佳实践。 + +### 使用限制 + +1. 仅支持无认证的 Kafka 访问。 +2. 支持的消息格式为 csv 文本格式。每一个 message 为一行,且行尾**不包含**换行符。 +3. 仅支持 Kafka 0.10.0.0(含) 以上版本。 + +### 创建例行导入任务 + +创建例行导入任务的的详细语法可以参照 [这里]()。或者连接到 Doris 后,执行 `HELP CREATE ROUTINE LOAD;` 查看语法帮助。这里主要详细介绍,创建作业时的注意事项。 + +* columns_mapping + + `columns_mapping` 主要用于指定表结构和 message 中的列映射关系,以及一些列的转换。如果不指定,Doris 会默认 message 中的列和表结构的列按顺序一一对应。虽然在正常情况下,如果源数据正好一一对应,则不指定也可以进行正常的数据导入。但是我们依然强烈建议用户**显式的指定列映射关系**。这样当表结构发生变化(比如增加一个 nullable 的列),或者源文件发生变化(比如增加了一列)时,导入任务依然可以继续进行。否则,当发生上述变动后,因为列映射关系不再一一对应,导入将报错。 + + 在 `columns_mapping` 中我们同样可以使用一些内置函数进行列的转换。但需要注意函数参数对应的实际列类型。举例说明: + + 假设用户需要导入只包含 `k1` 一列的表,列类型为 `int`。并且需要将源文件中的 null 值转换为 0。该功能可以通过 `ifnull` 函数实现。正确是的使用方式如下: + + `COLUMNS (xx, k1=ifnull(xx, "3"))` + + 注意这里我们使用 `"3"` 而不是 `3`,虽然 `k1` 的类型为 `int`。因为对于导入任务来说,源数据中的列类型都为 `varchar`,所以这里 `xx` 虚拟列的类型也为 `varchar`。所以我们需要使用 `"3"` 来进行对应的匹配,否则 `ifnull` 函数无法找到参数为 `(varchar, int)` 的函数签名,将出现错误。 + +* desired\_concurrent\_number + + `desired_concurrent_number` 用于指定一个例行作业期望的并发度。即一个作业,最多有多少 task 同时在执行。对于 Kafka 导入而言,当前的实际并发度计算如下: + + `Min(partition num, desired_concurrent_number, alive_backend_num, Config.max_routine_load_task_concurrrent_num)` + + 其中 `Config.max_routine_load_task_concurrrent_num` 是系统的一个默认的最大并发数限制。这是一个 FE 配置,可以通过改配置调整。默认为 5。 + + 其中 partition num 指订阅的 Kafka topic 的 partition 数量。`alive_backend_num` 是当前正常的 BE 节点数。 + +* max\_batch\_interval/max\_batch\_rows/max\_batch\_size + + 这三个参数用于控制单个任务的执行时间。其中任意一个阈值达到,则任务结束。其中 `max_batch_rows` 用于记录从 Kafka 中读取到的数据行数。`max_batch_size` 用于记录从 Kafka 中读取到的数据量,单位是字节。目前一个任务的消费速率大约为 5-10MB/s。 + + 那么假设一行数据 500B,用户希望每 100MB 或 10 秒为一个 task。100MB 的预期处理时间是 10-20 秒,对应的行数约为 200000 行。则一个合理的配置为: + + ``` + "max_batch_interval" = "10", + "max_batch_rows" = "200000", + "max_batch_size" = "104857600" + ``` + + 以上示例中的参数也是这些配置的默认参数。 + +* max\_error\_number + + `max_error_number` 用于控制错误率。在错误率过高的时候,作业会自动暂停。因为整个作业是面向数据流的,因为数据流的无边界性,我们无法像其他导入任务一样,通过一个错误比例来计算错误率。因此这里提供了一种新的计算方式,来计算数据流中的错误比例。 + + 我们设定了一个采样窗口。窗口的大小为 `max_batch_rows * 10`。在一个采样窗口内,如果错误行数超过 `max_error_number`,则作业被暂停。如果没有超过,则下一个窗口重新开始计算错误行数。 + + 我们假设 `max_batch_rows` 为 200000,则窗口大小为 2000000。设 `max_error_number` 为 20000,即用户预期每 2000000 行的错误行为 20000。即错误率为 1%。但是因为不是每批次任务正好消费 200000 行,所以窗口的实际范围是 [2000000, 2200000],即有 10% 的统计误差。 + + 错误行不包括通过 where 条件过滤掉的行。但是包括没有对应的 Doris 表中的分区的行。 + +* data\_source\_properties + + `data_source_properties` 中可以指定消费具体的 Kakfa partition。如果不指定,则默认消费所订阅的 topic 的所有 partition。 + + 注意,当显式的指定了 partition,则导入作业不会再动态的检测 Kafka partition 的变化。如果没有指定,则会根据 kafka partition 的变化,动态调整需要消费的 partition。 + +### 查看导入作业状态 + +查看**作业**状态的具体命令和示例可以通过 `HELP SHOW ROUTINE LOAD;` 命令查看。 + +查看**任务**运行状态的具体命令和示例可以通过 `HELP SHOW ROUTINE LOAD TASK;` 命令查看。 + +只能查看当前正在运行中的任务,已结束和未开始的任务无法查看。 + +### 作业控制 + +用户可以通过 `STOP/PAUSE/RESUME` 三个命令来控制作业的停止,暂停和重启。可以通过 `HELP STOP ROUTINE LOAD;`, `HELP PAUSE ROUTINE LOAD;` 以及 `HELP RESUME ROUTINE LOAD;` 三个命令查看帮助和示例。 + +## 其他说明 + +1. 例行导入作业和 ALTER TABLE 操作的关系 + + * 例行导入不会阻塞 SCHEMA CHANGE 和 ROLLUP 操作。但是注意如果 SCHEMA CHANGE 完成后,列映射关系无法匹配,则会导致作业的错误数据激增,最终导致作业暂停。建议通过在例行导入作业中显式指定列映射关系,以及通过增加 Nullable 列或带 Default 值的列来减少这类问题。 + * 删除表的 Partition 可能会导致导入数据无法找到对应的 Partition,作业进入暂停。 + +2. 例行导入作业和其他导入作业的关系(LOAD, DELETE, INSERT) + + * 例行导入和其他 LOAD 作业以及 INSERT 操作没有冲突。 + * 当执行 DELETE 操作时,对应表分区不能有任何正在执行的导入任务。所以在执行 DELETE 操作前,可能需要先暂停例行导入作业,并等待已下发的 task 全部完成后,才可以执行 DELETE。 + +3. 例行导入作业和 DROP DATABASE/TABLE 操作的关系 + + 当例行导入对应的 database 或 table 被删除后,作业会自动 CANCEL。 + +4. kafka 类型的例行导入作业和 kafka topic 的关系 + + 当用户在创建例行导入声明的 `kafka_topic` 在kafka集群中不存在时。 + + * 如果用户 kafka 集群的 broker 设置了 `auto.create.topics.enable = true`,则 `kafka_topic` 会先被自动创建,自动创建的 partition 个数是由**用户方的kafka集群**中的 broker 配置 `num.partitions` 决定的。例行作业会正常的不断读取该 topic 的数据。 + * 如果用户 kafka 集群的 broker 设置了 `auto.create.topics.enable = false`, 则 topic 不会被自动创建,例行作业会在没有读取任何数据之前就被暂停,状态为 `PAUSED`。 + + 所以,如果用户希望当 kafka topic 不存在的时候,被例行作业自动创建的话,只需要将**用户方的kafka集群**中的 broker 设置 `auto.create.topics.enable = true` 即可。 + +## 相关参数 + +一些系统配置参数会影响例行导入的使用。 + +1. max\_routine\_load\_task\_concurrent\_num + + FE 配置项,默认为 5,可以运行时修改。该参数限制了一个例行导入最大的子任务并发数。建议维持默认值。设置过大,可能导致同时并发的任务数过多,占用集群资源。 + +2. max\_consumer\_num\_per\_group + + BE 配置项,默认为 3。该参数表示一个子任务中最多生成几个 consumer 进行数据消费。对于 Kafka 数据源,一个 consumer 可能消费一个或多个 kafka partition。假设一个任务需要消费 6 个 kafka partition,则会生成 3 个 consumer,每个 consumer 消费 2 个 partition。如果只有 2 个 partition,则只会生成 2 个 consumer,每个 consumer 消费 1 个 partition。 + +3. push\_write\_mbytes\_per\_sec + + BE 配置项。默认为 10,即 10MB/s。该参数为导入通用参数,不限于例行导入作业。该参数限制了导入数据写入磁盘的速度。对于 SSD 等高性能存储设备,可以适当增加这个限速。 + + diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_extract.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_extract.md new file mode 100644 index 00000000000000..8cbf690eea9cb5 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_extract.md @@ -0,0 +1,27 @@ +# regexp_extract + +## Syntax + +`VARCHAR regexp_extract(VARCHAR str, VARCHAR pattern, int pos)` + +## Description + +对字符串 str 进行正则匹配,抽取符合 pattern 的第 pos 个匹配部分。需要 pattern 完全匹配 str 中的某部分,这样才能返回 pattern 部分中需匹配部分。如果没有匹配,返回空字符串。 + +## Examples + +``` +mysql> SELECT regexp_extract('AbCdE', '([[:lower:]]+)C([[:lower:]]+)', 1); ++-------------------------------------------------------------+ +| regexp_extract('AbCdE', '([[:lower:]]+)C([[:lower:]]+)', 1) | ++-------------------------------------------------------------+ +| b | ++-------------------------------------------------------------+ + +mysql> SELECT regexp_extract('AbCdE', '([[:lower:]]+)C([[:lower:]]+)', 2); ++-------------------------------------------------------------+ +| regexp_extract('AbCdE', '([[:lower:]]+)C([[:lower:]]+)', 2) | ++-------------------------------------------------------------+ +| d | ++-------------------------------------------------------------+ +``` diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_replace.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_replace.md new file mode 100644 index 00000000000000..c131996cdc4760 --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/regexp_replace.md @@ -0,0 +1,27 @@ +# regexp_replace + +## Syntax + +`VARCHAR regexp_replace(VARCHAR str, VARCHAR pattern, VARCHAR repl) + +## Description + +对字符串 str 进行正则匹配, 将命中 pattern 的部分使用 repl 来进行替换 + +## Examples + +``` +mysql> SELECT regexp_replace('a b c', " ", "-"); ++-----------------------------------+ +| regexp_replace('a b c', ' ', '-') | ++-----------------------------------+ +| a-b-c | ++-----------------------------------+ + +mysql> SELECT regexp_replace('a b c','(b)','<\\1>'); ++----------------------------------------+ +| regexp_replace('a b c', '(b)', '<\1>') | ++----------------------------------------+ +| a c | ++----------------------------------------+ +``` diff --git a/docs/documentation/cn/sql-reference/sql-functions/string-functions/repeat.md b/docs/documentation/cn/sql-reference/sql-functions/string-functions/repeat.md new file mode 100644 index 00000000000000..4ef1eeb2f70acb --- /dev/null +++ b/docs/documentation/cn/sql-reference/sql-functions/string-functions/repeat.md @@ -0,0 +1,27 @@ +# repeat + +## Syntax + +`VARCHAR repeat(VARCHAR str, INT count) + +## Description + +将字符串 str 重复 count 次输出,count 小于1时返回空串,str,count 任一为NULL时,返回 NULL + +## Examples + +``` +mysql> SELECT repeat("a", 3); ++----------------+ +| repeat('a', 3) | ++----------------+ +| aaa | ++----------------+ + +mysql> SELECT repeat("a", -1); ++-----------------+ +| repeat('a', -1) | ++-----------------+ +| | ++-----------------+ +``` diff --git a/docs/help/Contents/Account Management/help.md b/docs/help/Contents/Account Management/help.md index f25f7a0bc9b266..4a70ffe89636d9 100644 --- a/docs/help/Contents/Account Management/help.md +++ b/docs/help/Contents/Account Management/help.md @@ -144,7 +144,7 @@ user_identity: 3. 授予指定库表的权限给角色 - GRANT LOAD_PRIV ON db1.* TO ROLE my_role; + GRANT LOAD_PRIV ON db1.* TO ROLE 'my_role'; ## keyword GRANT @@ -279,7 +279,7 @@ user_identity: 1. 查看已创建的角色: - SHOW ROELS; + SHOW ROLES; ## keyword SHOW,ROLES diff --git a/docs/help/Contents/Data Definition/ddl_stmt.md b/docs/help/Contents/Data Definition/ddl_stmt.md index b1daca6e51b338..a9d39c48011fb3 100644 --- a/docs/help/Contents/Data Definition/ddl_stmt.md +++ b/docs/help/Contents/Data Definition/ddl_stmt.md @@ -580,8 +580,7 @@ ORDER BY (k3,k1,k2,v2,v1) FROM example_rollup_index; 11. 修改表的 bloom filter 列 - ALTER TABLE example_db.my_table - PROPERTIES ("bloom_filter_columns"="k1,k2,k3"); + ALTER TABLE example_db.my_table set ("bloom_filter_columns"="k1,k2,k3"); 也可以合并到上面的 schema change 操作中 ALTER TABLE example_db.my_table @@ -723,7 +722,7 @@ ALTER DATABASE example_db SET DATA QUOTA 200M; - 2. 将数据库额 example_db 重命名为 example_db2 + 2. 将数据库 example_db 重命名为 example_db2 ALTER DATABASE example_db RENAME example_db2; ## keyword diff --git a/docs/help/Contents/Data Manipulation/routine_load.md b/docs/help/Contents/Data Manipulation/routine_load.md new file mode 100644 index 00000000000000..2733c842a8a87d --- /dev/null +++ b/docs/help/Contents/Data Manipulation/routine_load.md @@ -0,0 +1,386 @@ +# ROUTINE LOAD +## description + + 例行导入(Routine Load)功能,支持用户提交一个常驻的导入任务,通过不断的从指定的数据源读取数据,将数据导入到 Doris 中。 + 目前仅支持通过无认证的方式,从 Kakfa 导入文本格式(CSV)的数据。 + +语法: + + CREATE ROUTINE LOAD [db.]job_name ON tbl_name + [load_properties] + [job_properties] + FROM data_source + [data_source_properties] + + 1. [db.]job_name + + 导入作业的名称,在同一个 database 内,相同名称只能有一个 job 在运行。 + + 2. tbl_name + + 指定需要导入的表的名称。 + + 3. load_properties + + 用于描述导入数据。语法: + + [column_separator], + [columns_mapping], + [where_predicates], + [partitions] + + 1. column_separator: + + 指定列分隔符,如: + + COLUMNS TERMINATED BY "," + + 默认为:\t + + 2. columns_mapping: + + 指定源数据中列的映射关系,以及定义衍生列的生成方式。 + + 1. 映射列: + + 按顺序指定,源数据中各个列,对应目的表中的哪些列。对于希望跳过的列,可以指定一个不存在的列名。 + 假设目的表有三列 k1, k2, v1。源数据有4列,其中第1、2、4列分别对应 k2, k1, v1。则书写如下: + + COLUMNS (k2, k1, xxx, v1) + + 其中 xxx 为不存在的一列,用于跳过源数据中的第三列。 + + 2. 衍生列: + + 以 col_name = expr 的形式表示的列,我们称为衍生列。即支持通过 expr 计算得出目的表中对应列的值。 + 衍生列通常排列在映射列之后,虽然这不是强制的规定,但是 Doris 总是先解析映射列,再解析衍生列。 + 接上一个示例,假设目的表还有第4列 v2,v2 由 k1 和 k2 的和产生。则可以书写如下: + + COLUMNS (k2, k1, xxx, v1, v2 = k1 + k2); + + 3. where_predicates + + 用于指定过滤条件,以过滤掉不需要的列。过滤列可以是映射列或衍生列。 + 例如我们只希望导入 k1 大于 100 并且 k2 等于 1000 的列,则书写如下: + + WHERE k1 > 100 and k2 = 1000 + + 4. partitions + + 指定导入目的表的哪些 partition 中。如果不指定,则会自动导入到对应的 partition 中。 + 示例: + + PARTITION(p1, p2, p3) + + 4. job_properties + + 用于指定例行导入作业的通用参数。 + 语法: + + PROPERTIES ( + "key1" = "val1", + "key2" = "val2" + ) + + 目前我们支持以下参数: + + 1. desired_concurrent_number + + 期望的并发度。一个例行导入作业会被分成多个子任务执行。这个参数指定一个作业最多有多少任务可以同时执行。必须大于0。默认为3。 + 这个并发度并不是实际的并发度,实际的并发度,会通过集群的节点数、负载情况,以及数据源的情况综合考虑。 + 例: + + "desired_concurrent_number" = "3" + + 2. max_batch_interval/max_batch_rows/max_batch_size + + 这三个参数分别表示: + 1)每个子任务最大执行时间,单位是秒。范围为 5 到 60。默认为10。 + 2)每个子任务最多读取的行数。必须大于等于200000。默认是200000。 + 3)每个子任务最多读取的字节数。单位是字节,范围是 100MB 到 1GB。默认是 100MB。 + + 这三个参数,用于控制一个子任务的执行时间和处理量。当任意一个达到阈值,则任务结束。 + 例: + + "max_batch_interval" = "20", + "max_batch_rows" = "300000", + "max_batch_size" = "209715200" + + 3. max_error_number + + 采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。 + 采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。 + 被 where 条件过滤掉的行不算错误行。 + + 5. data_source + + 数据源的类型。当前支持: + + KAFKA + + 6. data_source_properties + + 指定数据源相关的信息。 + 语法: + + ( + "key1" = "val1", + "key2" = "val2" + ) + + 1. KAFKA 数据源 + + 1. kafka_broker_list + + Kafka 的 broker 连接信息。格式为 ip:host。多个broker之间以逗号分隔。 + 示例: + + "kafka_broker_list" = "broker1:9092,broker2:9092" + + 2. kafka_topic + + 指定要订阅的 Kafka 的 topic。 + 示例: + + "kafka_topic" = "my_topic" + + 3. kafka_partitions/kafka_offsets + + 指定需要订阅的 kafka partition,以及对应的每个 partition 的起始 offset。 + + offset 可以指定从大于等于 0 的具体 offset,或者: + 1) OFFSET_BEGINNING: 从有数据的位置开始订阅。 + 2) OFFSET_END: 从末尾开始订阅。 + + 如果没有指定,则默认从 OFFSET_END 开始订阅 topic 下的所有 partition。 + 示例: + + "kafka_partitions" = "0,1,2,3", + "kafka_offsets" = "101,0,OFFSET_BEGINNING,OFFSET_END" + + + 7. 导入数据格式样例 + + 整型类(TINYINT/SMALLINT/INT/BIGINT/LARGEINT):1, 1000, 1234 + 浮点类(FLOAT/DOUBLE/DECIMAL):1.1, 0.23, .356 + 日期类(DATE/DATETIME):2017-10-03, 2017-06-13 12:34:03。 + 字符串类(CHAR/VARCHAR)(无引号):I am a student, a + NULL值:\N + +## example + + 1. 为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务。 + + CREATE ROUTINE LOAD example_db.test1 ON example_tbl + COLUMNS(k1, k2, k3, v1, v2, v3 = k1 * 100), + WHERE k1 > 100 and k2 like "%doris%" + PROPERTIES + ( + "desired_concurrent_number"="3", + "max_batch_interval" = "20", + "max_batch_rows" = "300000", + "max_batch_size" = "209715200" + ) + FROM KAFKA + ( + "kafka_broker_list" = "broker1:9092,broker2:9092,broker3:9092", + "kafka_topic" = "my_topic", + "kafka_partitions" = "0,1,2,3", + "kafka_offsets" = "101,0,0,200" + ); + +## keyword + CREATE,ROUTINE,LOAD + +# PAUSE ROUTINE LOAD + + 该语句用于暂停一个指定的例行导入作业。 + +语法: + + PAUSE ROUTINE LOAD [db.]name; + +## example + +1. 暂停名称为 test1 的例行导入作业。 + + PAUSE ROUTINE LOAD test1; + +## keyword + PAUSE,ROUTINE,LOAD + +# RESUME ROUTINE LOAD + + 该语句用于恢复一个被暂停的例行导入作业。 + +语法: + + RESUME ROUTINE LOAD [db.]name; + +## example + +1. 恢复名称为 test1 的例行导入作业。 + + RESUME ROUTINE LOAD test1; + +## keyword + RESUME,ROUTINE,LOAD + +# STOP ROUTINE LOAD + + 该语句用于停止一个被暂停的例行导入作业。 + +语法: + + STOP ROUTINE LOAD [db.]name; + + 被停止的作业无法再恢复运行。 + +## example + +1. 停止名称为 test1 的例行导入作业。 + + STOP ROUTINE LOAD test1; + +## keyword + STOP,ROUTINE,LOAD + +# SHOW ROUTINE LOAD + + 该语句用于展示指定名称的例行导入作业的详细信息。 + +语法: + + SHOW [ALL] ROUTINE LOAD [[db.]name] + +展示结果包括如下信息: + + 1. Id:作业id。 + 2. Name:作业的名称。 + 3. CreateTime:作业创建时间。 + 4. PauseTime:作业暂停时间。 + 5. EndTime:作业结束时间。 + 6. DdName:数据库名称。 + 7. TableName:目的表名称。 + 8. State:作业状态。 + + NEED_SCHEDULE:等待被调度。 + RUNNING:运行中。 + PAUSE:暂停中。 + STOPPED:作业由用户停止。 + CANCELLED:作业因失败停止。 + + 9. DataSourceType:数据源类型。 + + KAFKA + + 10. CurrentTaskNum:当前正在运行的子任务的个数 + 11. JobProperties:作业相关配置信息,对应创建语句中的 load_properties 和 job_properties。以 json 格式表示。 + + { + "partitions": "*", // 目的表的分区,星号表示没有指定。 + "columnToColumnExpr": "k1,yyy,v1,v2,v3,v4,v5,v6,k2=`k1` + 1", + "maxBatchIntervalS": "10", + "whereExpr": "`k1` > 100", + "maxBatchSizeBytes": "104857600", + "columnSeparator": "\t", + "maxErrorNum": "0", + "currentTaskConcurrentNum": "3", // 当前例行作业的子任务并发数 + "maxBatchRows": "200000" + } + + 12. Statistic:作业运行状态的统计信息。以 json 格式表示。 + + { + "errorRows": 0, // 总的错误行数 + "loadedRows": 6584959, // 总导入的行数 + "unselectedRows": 2392, // 被 where 条件过滤的行数 + "totalRows": 6587351, // 总消费的行数,totalRows = errorRows + loadedRows + unselectedRows + "loadRowsRate": 91000, // 导入速率(rows/s) + "receivedBytes": 861626324, // 总消费的字节数 + "receivedBytesRate": 11915000, // 消费速率 (Bytes/s) + "committedTaskNum": 33, // 提交成功的子任务数 + "abortedTaskNum": 2, // 失败的子任务数 + "taskExecuteTaskMs": 72312 // 子任务执行时间,单位毫秒 + } + + 13. Progress:作业进度。以 json 格式表示。 + + 如果数据源是 Kafka,则显示每个 kafka partition,当前已经被消费的 offset。 + + { + "0": 2199288, + "1": 2194329, + "2": 2193731 + } + + OFFSET_BEGINNING: 表示用户指定了从头开始消费,并且还未开始消费。 + OFFSET_END: 表示用户指定了从末尾开始消费,并且还未开始消费。 + OFFSET_ZERO: 表示用户指定了从 0 开始消费,并且还未开始消费。 + +## example + +1. 展示名称为 test1 的所有例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 + + SHOW ALL ROUTINE LOAD test1; + +2. 展示名称为 test1 的当前正在运行的例行导入作业 + + SHOW ROUTINE LOAD test1; + +3. 显示 example_db 下,所有的例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 + + use example_db; + SHOW ALL ROUTINE LOAD; + +4. 显示 example_db 下,所有正在运行的例行导入作业 + + use example_db; + SHOW ROUTINE LOAD; + +5. 显示 example_db 下,名称为 test1 的当前正在运行的例行导入作业 + + SHOW ROUTINE LOAD example_db.test1; + +6. 显示 example_db 下,名称为 test1 的所有例行导入作业(包括已停止或取消的作业)。结果为一行或多行。 + + SHOW ALL ROUTINE LOAD example_db.test1; + +## keyword + SHOW,ROUTINE,LOAD + +# SHOW ROUTINE LOAD TASK + + 该语句用于展示指定例行导入作业,当前正在运行的子任务信息。 + +语法: + + SHOW ROUTINE LOAD TASK [FROM db] WHERE JobName = "name"; + +展示结果包括如下信息: + + TaskId:task id。 + TxnId:task 对应的事务id。 + JobId:作业id。 + CreateTime:任务创建时间。 + ExecuteStartTime:任务开始执行的时间。 + BeId:任务所在的 Backend id。 + DataSourceProperties: + + 任务的参数,以 json 格式展示。 + 当数据源为 Kafka 时,显示如下: + + { + "2":2193732 + } + + 表示该任务准备消费的 kafka partition 和起始 offset。 + +## example + +1. 展示名为 test1 的例行导入任务的子任务信息。 + + SHOW ROUTINE LOAD TASK WHERE JobName = "test1"; + +# keyword + SHOW,ROUTINE,LOAD,TASK diff --git a/docs/help/Contents/Data Manipulation/streaming.md b/docs/help/Contents/Data Manipulation/streaming.md index 5872eaf85a744b..3a279e2d0c94ac 100644 --- a/docs/help/Contents/Data Manipulation/streaming.md +++ b/docs/help/Contents/Data Manipulation/streaming.md @@ -35,7 +35,7 @@ where: 用于抽取部分数据。用户如果有需要将不需要的数据过滤掉,那么可以通过设定这个选项来达到。 例1: 只导入大于k1列等于20180601的数据,那么可以在导入时候指定-H "where: k1 = 20180601" - max_filter_ratio:最大容忍可过滤(数据不规范等原因)的数据比例。默认零容忍。 + max_filter_ratio:最大容忍可过滤(数据不规范等原因)的数据比例。默认零容忍。数据不规范不包括通过 where 条件过滤掉的行。 partitions: 用于指定这次导入所设计的partition。如果用户能够确定数据对应的partition,推荐指定该项。不满足这些分区的数据将被过滤掉。 比如指定导入到p1, p2分区,-H "partitions: p1, p2" @@ -49,8 +49,10 @@ 用户需要通过get label state命令来确定后续的操作 其他:此次导入失败,用户可以指定Label重试此次作业 Message: 导入状态详细的说明。失败时会返回具体的失败原因。 + NumberTotalRows: 从数据流中读取到的总行数 NumberLoadedRows: 此次导入的数据行数,只有在Success时有效 - NumberFilteredRows: 此次导入过滤掉的行数 + NumberFilteredRows: 此次导入过滤掉的行数,即数据质量不合格的行数 + NumberUnselectedRows: 此次导入,通过 where 条件被过滤掉的行数 LoadBytes: 此次导入的源文件数据量大小 LoadTimeMs: 此次导入所用的时间 ErrorURL: 被过滤数据的具体内容,仅保留前1000条 diff --git a/fe/src/main/cup/sql_parser.cup b/fe/src/main/cup/sql_parser.cup index 3e5c9b13ac1124..a453520145a24f 100644 --- a/fe/src/main/cup/sql_parser.cup +++ b/fe/src/main/cup/sql_parser.cup @@ -206,7 +206,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A KW_JOIN, KW_KEY, KW_KILL, KW_LABEL, KW_LARGEINT, KW_LAST, KW_LEFT, KW_LESS, KW_LEVEL, KW_LIKE, KW_LIMIT, KW_LINK, KW_LOAD, - KW_ROUTINE, KW_PAUSE, KW_RESUME, KW_STOP, + KW_ROUTINE, KW_PAUSE, KW_RESUME, KW_STOP, KW_TASK, KW_LOCAL, KW_LOCATION, KW_MAX, KW_MAX_VALUE, KW_MERGE, KW_MIN, KW_MIGRATE, KW_MIGRATIONS, KW_MODIFY, KW_NAME, KW_NAMES, KW_NEGATIVE, KW_NO, KW_NOT, KW_NULL, KW_NULLS, @@ -243,7 +243,8 @@ terminal String COMMENTED_PLAN_HINTS; // Statement that the result of this parser. nonterminal StatementBase query, stmt, show_stmt, show_param, help_stmt, load_stmt, - create_routine_load_stmt, pause_routine_load_stmt, resume_routine_load_stmt, stop_routine_load_stmt, show_routine_load_stmt, + create_routine_load_stmt, pause_routine_load_stmt, resume_routine_load_stmt, stop_routine_load_stmt, + show_routine_load_stmt, show_routine_load_task_stmt, describe_stmt, alter_stmt, use_stmt, kill_stmt, drop_stmt, recover_stmt, grant_stmt, revoke_stmt, create_stmt, set_stmt, sync_stmt, cancel_stmt, cancel_param, delete_stmt, link_stmt, migrate_stmt, enter_stmt, unsupported_stmt, export_stmt, admin_stmt, truncate_stmt, import_columns_stmt, import_where_stmt; @@ -374,6 +375,7 @@ nonterminal AccessPrivilege privilege_type; nonterminal DataDescription data_desc; nonterminal List data_desc_list; nonterminal LabelName job_label; +nonterminal LabelName opt_job_label; nonterminal String opt_system; nonterminal String opt_cluster; nonterminal BrokerDesc opt_broker; @@ -386,7 +388,6 @@ nonterminal TablePattern tbl_pattern; nonterminal String ident_or_star; // Routine load -nonterminal LoadColumnsInfo load_columns_info; nonterminal ParseNode load_property; nonterminal List opt_load_property_list; @@ -449,7 +450,7 @@ query ::= ; import_columns_stmt ::= - KW_COLUMNS import_column_descs:columns + KW_COLUMNS LPAREN import_column_descs:columns RPAREN {: RESULT = new ImportColumnsStmt(columns); :} @@ -530,6 +531,8 @@ stmt ::= {: RESULT = stmt; :} | show_routine_load_stmt : stmt {: RESULT = stmt; :} + | show_routine_load_task_stmt : stmt + {: RESULT = stmt; :} | cancel_stmt : stmt {: RESULT = stmt; :} | delete_stmt : stmt @@ -1012,6 +1015,17 @@ load_stmt ::= :} ; +opt_job_label ::= + /* Empty */ + {: + RESULT = null; + :} + | job_label:jobLabel + {: + RESULT = jobLabel; + :} + ; + job_label ::= ident:label {: @@ -1158,12 +1172,12 @@ opt_cluster ::= // Routine load statement create_routine_load_stmt ::= - KW_CREATE KW_ROUTINE KW_LOAD ident:jobName KW_ON table_name:dbTableName + KW_CREATE KW_ROUTINE KW_LOAD job_label:jobLabel KW_ON ident:tableName opt_load_property_list:loadPropertyList opt_properties:properties KW_FROM ident:type LPAREN key_value_map:customProperties RPAREN {: - RESULT = new CreateRoutineLoadStmt(jobName, dbTableName, loadPropertyList, properties, type, customProperties); + RESULT = new CreateRoutineLoadStmt(jobLabel, tableName, loadPropertyList, properties, type, customProperties); :} ; @@ -1187,11 +1201,11 @@ load_property ::= {: RESULT = colSep; :} - | load_columns_info:columnsInfo + | import_columns_stmt:columnsInfo {: RESULT = columnsInfo; :} - | where_clause_without_null:wherePredicate + | import_where_stmt:wherePredicate {: RESULT = wherePredicate; :} @@ -1201,39 +1215,42 @@ load_property ::= :} ; -load_columns_info ::= - col_list:colList - opt_col_mapping_list:colMappingList - {: - RESULT = new LoadColumnsInfo(colList, colMappingList); - :} - ; - pause_routine_load_stmt ::= - KW_PAUSE KW_ROUTINE KW_LOAD ident:name + KW_PAUSE KW_ROUTINE KW_LOAD job_label:jobLabel {: - RESULT = new PauseRoutineLoadStmt(name); + RESULT = new PauseRoutineLoadStmt(jobLabel); :} ; resume_routine_load_stmt ::= - KW_RESUME KW_ROUTINE KW_LOAD ident:name + KW_RESUME KW_ROUTINE KW_LOAD job_label:jobLabel {: - RESULT = new ResumeRoutineLoadStmt(name); + RESULT = new ResumeRoutineLoadStmt(jobLabel); :} ; stop_routine_load_stmt ::= - KW_STOP KW_ROUTINE KW_LOAD ident:name + KW_STOP KW_ROUTINE KW_LOAD job_label:jobLabel {: - RESULT = new StopRoutineLoadStmt(name); + RESULT = new StopRoutineLoadStmt(jobLabel); :} ; show_routine_load_stmt ::= - KW_SHOW KW_ROUTINE KW_LOAD ident:name + KW_SHOW KW_ROUTINE KW_LOAD opt_job_label:jobLabel + {: + RESULT = new ShowRoutineLoadStmt(jobLabel, false); + :} + | KW_SHOW KW_ALL KW_ROUTINE KW_LOAD opt_job_label:jobLabel + {: + RESULT = new ShowRoutineLoadStmt(jobLabel, true); + :} + ; + +show_routine_load_task_stmt ::= + KW_SHOW KW_ROUTINE KW_LOAD KW_TASK opt_db:dbName opt_wild_where {: - RESULT = new ShowRoutineLoadStmt(name); + RESULT = new ShowRoutineLoadTaskStmt(dbName, parser.where); :} ; @@ -3842,6 +3859,8 @@ keyword ::= {: RESULT = id; :} | KW_LAST:id {: RESULT = id; :} + | KW_LEFT:id + {: RESULT = id; :} | KW_LESS:id {: RESULT = id; :} | KW_LEVEL:id @@ -3906,6 +3925,8 @@ keyword ::= {: RESULT = id; :} | KW_RETURNS:id {: RESULT = id; :} + | KW_RIGHT:id + {: RESULT = id; :} | KW_ROLLBACK:id {: RESULT = id; :} | KW_ROLLUP:id @@ -3957,26 +3978,36 @@ keyword ::= | KW_WORK:id {: RESULT = id; :} | KW_CLUSTER:id - {: RESULT = id; :} - | KW_CLUSTERS:id - {: RESULT = id; :} + {: RESULT = id; :} + | KW_CLUSTERS:id + {: RESULT = id; :} | KW_LINK:id - {: RESULT = id; :} + {: RESULT = id; :} | KW_MIGRATE:id - {: RESULT = id; :} - | KW_MIGRATIONS:id - {: RESULT = id; :} - | KW_COUNT:id - {: RESULT = id; :} - | KW_SUM:id - {: RESULT = id; :} - | KW_MIN:id - {: RESULT = id; :} - | KW_MAX:id - {: RESULT = id; :} - | KW_FREE:id - {: RESULT = id; :} - ; + {: RESULT = id; :} + | KW_MIGRATIONS:id + {: RESULT = id; :} + | KW_COUNT:id + {: RESULT = id; :} + | KW_SUM:id + {: RESULT = id; :} + | KW_MIN:id + {: RESULT = id; :} + | KW_MAX:id + {: RESULT = id; :} + | KW_FREE:id + {: RESULT = id; :} + | KW_TASK:id + {: RESULT = id; :} + | KW_ROUTINE:id + {: RESULT = id; :} + | KW_PAUSE:id + {: RESULT = id; :} + | KW_RESUME:id + {: RESULT = id; :} + | KW_STOP:id + {: RESULT = id; :} + ; // Identifier that contain keyword ident ::= diff --git a/fe/src/main/java/org/apache/doris/alter/AlterJob.java b/fe/src/main/java/org/apache/doris/alter/AlterJob.java index 5819fdf2cb1580..91ae084848276c 100644 --- a/fe/src/main/java/org/apache/doris/alter/AlterJob.java +++ b/fe/src/main/java/org/apache/doris/alter/AlterJob.java @@ -82,7 +82,7 @@ public String toString() { protected long tableId; protected long transactionId = -1; // not serialize it - protected boolean hasPreviousLoadFinished = false; + protected boolean isPreviousLoadFinished = false; protected AgentBatchTask batchClearAlterTask = null; protected long createTime; @@ -285,13 +285,13 @@ public abstract void handleFinishedReplica(AgentTask task, TTabletInfo finishTab public abstract void getJobInfo(List> jobInfos, OlapTable tbl); - public boolean checkPreviousLoadFinished() { - if (hasPreviousLoadFinished) { + public boolean isPreviousLoadFinished() { + if (isPreviousLoadFinished) { return true; } else { - hasPreviousLoadFinished = Catalog.getCurrentGlobalTransactionMgr() - .hasPreviousTransactionsFinished(transactionId, dbId); - return hasPreviousLoadFinished; + isPreviousLoadFinished = Catalog.getCurrentGlobalTransactionMgr() + .isPreviousTransactionsFinished(transactionId, dbId); + return isPreviousLoadFinished; } } diff --git a/fe/src/main/java/org/apache/doris/alter/RollupHandler.java b/fe/src/main/java/org/apache/doris/alter/RollupHandler.java index a201bca59ddf56..17f6f041c98f18 100644 --- a/fe/src/main/java/org/apache/doris/alter/RollupHandler.java +++ b/fe/src/main/java/org/apache/doris/alter/RollupHandler.java @@ -598,9 +598,9 @@ protected void runOneCycle() { break; } case FINISHING: { - // check previous load job finished - if (rollupJob.checkPreviousLoadFinished()) { - // if all previous load job finished, then send clear alter tasks to all related be + // check if previous load job finished + if (rollupJob.isPreviousLoadFinished()) { + // if all previous load jobs are finished, then send clear alter tasks to all related be int res = rollupJob.checkOrResendClearTasks(); if (res != 0) { if (res == -1) { @@ -609,6 +609,9 @@ protected void runOneCycle() { } finishedJobs.add(rollupJob); } + } else { + LOG.info("previous load jobs are not finished. can not finish rollup job: {}", + rollupJob.getTableId()); } break; } diff --git a/fe/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/src/main/java/org/apache/doris/alter/RollupJob.java index 04ca2ee44c4a94..8bc2a08fb68c34 100644 --- a/fe/src/main/java/org/apache/doris/alter/RollupJob.java +++ b/fe/src/main/java/org/apache/doris/alter/RollupJob.java @@ -118,7 +118,7 @@ private RollupJob() { this.finishedPartitionIds = new HashSet(); } - // yiguolei: every job has a transactionid to identify the occurrent time, for example + // yiguolei: every job has a transactionid to identify the current time, for example // a load job's transactionid is 10 and a rollup job's transaction id is 12, then we could // find load job is occurred before rollup job public RollupJob(long dbId, long tableId, long baseIndexId, long rollupIndexId, @@ -1139,7 +1139,7 @@ public String toString() { + baseIndexName + ", rollupIndexName=" + rollupIndexName + ", rollupSchema=" + rollupSchema + ", baseSchemaHash=" + baseSchemaHash + ", rollupSchemaHash=" + rollupSchemaHash + ", type=" + type + ", state=" + state + ", dbId=" + dbId + ", tableId=" + tableId + ", transactionId=" + transactionId - + ", hasPreviousLoadFinished=" + hasPreviousLoadFinished + ", createTime=" + createTime + + ", isPreviousLoadFinished=" + isPreviousLoadFinished + ", createTime=" + createTime + ", finishedTime=" + finishedTime + "]"; } } diff --git a/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 3f4caa58f62459..89c96523e9e528 100644 --- a/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -1140,8 +1140,8 @@ protected void runOneCycle() { break; } case FINISHING: { - // check previous load job finished - if (alterJob.checkPreviousLoadFinished()) { + // check if previous load job finished + if (alterJob.isPreviousLoadFinished()) { LOG.info("schema change job has finished, send clear tasks to all be {}", alterJob); // if all previous load job finished, then send clear alter tasks to all related be int res = schemaChangeJob.checkOrResendClearTasks(); @@ -1156,6 +1156,9 @@ protected void runOneCycle() { finishedJobs.add(alterJob); } + } else { + LOG.info("previous load jobs are not finished. can not finish schema change job: {}", + alterJob.getTableId()); } break; } diff --git a/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java index ae71cd4230f5d1..2e7ee376c90bc1 100644 --- a/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java +++ b/fe/src/main/java/org/apache/doris/alter/SchemaChangeJob.java @@ -1106,7 +1106,6 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { jobInfo.add(state.name()); // job state jobInfo.add("N/A"); // progress jobInfo.add(cancelMsg); - jobInfos.add(jobInfo); return; } @@ -1122,33 +1121,38 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { // calc progress and state for each table for (Long indexId : changedIndexIdToSchemaVersion.keySet()) { - int totalReplicaNum = 0; - int finishedReplicaNum = 0; - String idxState = IndexState.NORMAL.name(); - for (Partition partition : tbl.getPartitions()) { - MaterializedIndex index = partition.getIndex(indexId); - - if (state == JobState.RUNNING) { - int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); - int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); - Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); - Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, - tableFinishedReplicaNum + "/" + tableReplicaNum); - totalReplicaNum += tableReplicaNum; - finishedReplicaNum += tableFinishedReplicaNum; - } + if (tbl.getIndexNameById(indexId) == null) { + // this index may be dropped, and this should be a FINISHED job, just use a dummy info to show + indexState.put(indexId, IndexState.NORMAL.name()); + indexProgress.put(indexId, "100%"); + } else { + int totalReplicaNum = 0; + int finishedReplicaNum = 0; + String idxState = IndexState.NORMAL.name(); + for (Partition partition : tbl.getPartitions()) { + MaterializedIndex index = partition.getIndex(indexId); + if (state == JobState.RUNNING) { + int tableReplicaNum = getTotalReplicaNumByIndexId(indexId); + int tableFinishedReplicaNum = getFinishedReplicaNumByIndexId(indexId); + Preconditions.checkState(!(tableReplicaNum == 0 && tableFinishedReplicaNum == -1)); + Preconditions.checkState(tableFinishedReplicaNum <= tableReplicaNum, + tableFinishedReplicaNum + "/" + tableReplicaNum); + totalReplicaNum += tableReplicaNum; + finishedReplicaNum += tableFinishedReplicaNum; + } - if (index.getState() != IndexState.NORMAL) { - idxState = index.getState().name(); + if (index.getState() != IndexState.NORMAL) { + idxState = index.getState().name(); + } } - } - indexState.put(indexId, idxState); + indexState.put(indexId, idxState); - if (Catalog.getInstance().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { - indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); - } else { - indexProgress.put(indexId, "0%"); + if (Catalog.getInstance().isMaster() && state == JobState.RUNNING && totalReplicaNum != 0) { + indexProgress.put(indexId, (finishedReplicaNum * 100 / totalReplicaNum) + "%"); + } else { + indexProgress.put(indexId, "0%"); + } } } @@ -1159,7 +1163,7 @@ public void getJobInfo(List> jobInfos, OlapTable tbl) { jobInfo.add(tbl.getName()); jobInfo.add(TimeUtils.longToTimeString(createTime)); jobInfo.add(TimeUtils.longToTimeString(finishedTime)); - jobInfo.add(tbl.getIndexNameById(indexId)); // index name + jobInfo.add(tbl.getIndexNameById(indexId) == null ? "N/A" : tbl.getIndexNameById(indexId)); // index name jobInfo.add(indexId); // index schema version and schema hash jobInfo.add(changedIndexIdToSchemaVersion.get(indexId) + "-" + changedIndexIdToSchemaHash.get(indexId)); @@ -1358,7 +1362,7 @@ public boolean equals(Object obj) { @Override public String toString() { return "SchemaChangeJob [tableName=" + tableName + ", type=" + type + ", state=" + state + ", dbId=" + dbId - + ", tableId=" + tableId + ", transactionId=" + transactionId + ", hasPreviousLoadFinished=" - + hasPreviousLoadFinished + ", createTime=" + createTime + ", finishedTime=" + finishedTime + "]"; + + ", tableId=" + tableId + ", transactionId=" + transactionId + ", isPreviousLoadFinished=" + + isPreviousLoadFinished + ", createTime=" + createTime + ", finishedTime=" + finishedTime + "]"; } } diff --git a/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaDistributionStmt.java b/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaDistributionStmt.java index d8bd929f27ddec..deae50ffbec457 100644 --- a/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaDistributionStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaDistributionStmt.java @@ -92,4 +92,13 @@ public ShowResultSetMetaData getMetaData() { } return builder.build(); } + + @Override + public RedirectStatus getRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } } diff --git a/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaStatusStmt.java b/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaStatusStmt.java index 8b1daa7a9e41a8..53d343aec0953e 100644 --- a/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaStatusStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/AdminShowReplicaStatusStmt.java @@ -160,4 +160,12 @@ public ShowResultSetMetaData getMetaData() { return builder.build(); } + @Override + public RedirectStatus getRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } } diff --git a/fe/src/main/java/org/apache/doris/analysis/AlterLoadErrorUrlClause.java b/fe/src/main/java/org/apache/doris/analysis/AlterLoadErrorUrlClause.java index d88e65bdd99f71..633ed85e4290f8 100644 --- a/fe/src/main/java/org/apache/doris/analysis/AlterLoadErrorUrlClause.java +++ b/fe/src/main/java/org/apache/doris/analysis/AlterLoadErrorUrlClause.java @@ -29,8 +29,7 @@ import java.util.Map; // FORMAT: -// ALTER SYSTEM SET GLOBAL LOAD_ERROR_URL= "mysql://user:password@host:port[/database[/table]]" -// ALTER SYSTEM SET GLOBAL LOAD_ERROR_URL= "broker://" +// ALTER SYSTEM SET LOAD ERRORS HUB properties("type" = "xxx"); public class AlterLoadErrorUrlClause extends AlterClause { private static final Logger LOG = LogManager.getLogger(AlterLoadErrorUrlClause.class); diff --git a/fe/src/main/java/org/apache/doris/analysis/BinaryPredicate.java b/fe/src/main/java/org/apache/doris/analysis/BinaryPredicate.java index 76915a370ca057..fcdbebd1e9b779 100644 --- a/fe/src/main/java/org/apache/doris/analysis/BinaryPredicate.java +++ b/fe/src/main/java/org/apache/doris/analysis/BinaryPredicate.java @@ -125,6 +125,8 @@ public Operator converse() { } public boolean isEquivalence() { return this == EQ; }; + + public boolean isUnequivalence() { return this == NE; } } private Operator op; diff --git a/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java index e84fdc34cc2003..7d30d9975e020d 100644 --- a/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java @@ -17,26 +17,33 @@ package org.apache.doris.analysis; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableSet; -import org.apache.doris.load.routineload.LoadDataSourceType; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.Util; import org.apache.doris.load.RoutineLoadDesc; -import org.apache.doris.qe.ConnectContext; +import org.apache.doris.load.routineload.KafkaProgress; +import org.apache.doris.load.routineload.LoadDataSourceType; +import org.apache.doris.load.routineload.RoutineLoadJob; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Predicate; import java.util.regex.Pattern; /* Create routine Load statement, continually load data from a streaming app syntax: - CREATE ROUTINE LOAD name ON database.table + CREATE ROUTINE LOAD [database.]name on table [load properties] [PROPERTIES ( @@ -57,16 +64,16 @@ load property [[,] load property] ... load property: - column separator | columns | partitions | where + column separator | columns_mapping | partitions | where column separator: COLUMNS TERMINATED BY xxx - columns: - COLUMNS (c1, c2, c3) set (c1, c2, c3=c1+c2) + columns_mapping: + COLUMNS (c1, c2, c3 = c1 + c2) partitions: PARTITIONS (p1, p2, p3) where: - WHERE xxx + WHERE c1 > 1 type of routine load: KAFKA @@ -76,75 +83,93 @@ public class CreateRoutineLoadStmt extends DdlStmt { public static final String DESIRED_CONCURRENT_NUMBER_PROPERTY = "desired_concurrent_number"; // max error number in ten thousand records public static final String MAX_ERROR_NUMBER_PROPERTY = "max_error_number"; + // the following 3 properties limit the time and batch size of a single routine load task + public static final String MAX_BATCH_INTERVAL_SEC_PROPERTY = "max_batch_interval"; + public static final String MAX_BATCH_ROWS_PROPERTY = "max_batch_rows"; + public static final String MAX_BATCH_SIZE_PROPERTY = "max_batch_size"; // kafka type properties - public static final String KAFKA_ENDPOINT_PROPERTY = "kafka_endpoint"; + public static final String KAFKA_BROKER_LIST_PROPERTY = "kafka_broker_list"; public static final String KAFKA_TOPIC_PROPERTY = "kafka_topic"; // optional public static final String KAFKA_PARTITIONS_PROPERTY = "kafka_partitions"; + public static final String KAFKA_OFFSETS_PROPERTY = "kafka_offsets"; private static final String NAME_TYPE = "ROUTINE LOAD NAME"; private static final String ENDPOINT_REGEX = "[-A-Za-z0-9+&@#/%?=~_|!:,.;]+[-A-Za-z0-9+&@#/%=~_|]"; - private static final String EMPTY_STRING = ""; private static final ImmutableSet PROPERTIES_SET = new ImmutableSet.Builder() .add(DESIRED_CONCURRENT_NUMBER_PROPERTY) .add(MAX_ERROR_NUMBER_PROPERTY) + .add(MAX_BATCH_INTERVAL_SEC_PROPERTY) + .add(MAX_BATCH_ROWS_PROPERTY) + .add(MAX_BATCH_SIZE_PROPERTY) .build(); private static final ImmutableSet KAFKA_PROPERTIES_SET = new ImmutableSet.Builder() - .add(KAFKA_ENDPOINT_PROPERTY) + .add(KAFKA_BROKER_LIST_PROPERTY) .add(KAFKA_TOPIC_PROPERTY) .add(KAFKA_PARTITIONS_PROPERTY) + .add(KAFKA_OFFSETS_PROPERTY) .build(); - private final String name; - private final TableName dbTableName; + private final LabelName labelName; + private final String tableName; private final List loadPropertyList; - private final Map properties; + private final Map jobProperties; private final String typeName; - private final Map customProperties; + private final Map dataSourceProperties; - - // those load properties will be initialized after analyze + // the following variables will be initialized after analyze + // -1 as unset, the default value will set in RoutineLoadJob + private String name; + private String dbName; private RoutineLoadDesc routineLoadDesc; - private int desiredConcurrentNum; - private int maxErrorNum; - private String kafkaEndpoint; + private int desiredConcurrentNum = 1; + private long maxErrorNum = -1; + private long maxBatchIntervalS = -1; + private long maxBatchRows = -1; + private long maxBatchSizeBytes = -1; + + // kafka related properties + private String kafkaBrokerList; private String kafkaTopic; - private List kafkaPartitions; - - public CreateRoutineLoadStmt(String name, TableName dbTableName, List loadPropertyList, - Map properties, - String typeName, Map customProperties) { - this.name = name; - this.dbTableName = dbTableName; + // pair + private List> kafkaPartitionOffsets = Lists.newArrayList(); + + private static final Predicate DESIRED_CONCURRENT_NUMBER_PRED = (v) -> { return v > 0L; }; + private static final Predicate MAX_ERROR_NUMBER_PRED = (v) -> { return v >= 0L; }; + private static final Predicate MAX_BATCH_INTERVAL_PRED = (v) -> { return v >= 5 && v <= 60; }; + private static final Predicate MAX_BATCH_ROWS_PRED = (v) -> { return v >= 200000; }; + private static final Predicate MAX_BATCH_SIZE_PRED = (v) -> { return v >= 100 * 1024 * 1024 && v <= 1024 * 1024 * 1024; }; + + public CreateRoutineLoadStmt(LabelName labelName, String tableName, List loadPropertyList, + Map jobProperties, + String typeName, Map dataSourceProperties) { + this.labelName = labelName; + this.tableName = tableName; this.loadPropertyList = loadPropertyList; - this.properties = properties; - this.typeName = typeName; - this.customProperties = customProperties; + this.jobProperties = jobProperties == null ? Maps.newHashMap() : jobProperties; + this.typeName = typeName.toUpperCase(); + this.dataSourceProperties = dataSourceProperties; } public String getName() { return name; } - public TableName getDBTableName() { - return dbTableName; + public String getDBName() { + return dbName; } - public Map getProperties() { - return properties; + public String getTableName() { + return tableName; } public String getTypeName() { return typeName; } - public Map getCustomProperties() { - return customProperties; - } - public RoutineLoadDesc getRoutineLoadDesc() { return routineLoadDesc; } @@ -153,58 +178,66 @@ public int getDesiredConcurrentNum() { return desiredConcurrentNum; } - public int getMaxErrorNum() { + public long getMaxErrorNum() { return maxErrorNum; } - public String getKafkaEndpoint() { - return kafkaEndpoint; + public long getMaxBatchIntervalS() { + return maxBatchIntervalS; + } + + public long getMaxBatchRows() { + return maxBatchRows; + } + + public long getMaxBatchSize() { + return maxBatchSizeBytes; + } + + public String getKafkaBrokerList() { + return kafkaBrokerList; } public String getKafkaTopic() { return kafkaTopic; } - public List getKafkaPartitions() { - return kafkaPartitions; + public List> getKafkaPartitionOffsets() { + return kafkaPartitionOffsets; } @Override - public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + // check dbName and tableName + checkDBTable(analyzer); // check name FeNameFormat.checkCommonName(NAME_TYPE, name); - // check dbName and tableName - checkDBTableName(); // check load properties include column separator etc. checkLoadProperties(analyzer); - // check routine load properties include desired concurrent number etc. - checkRoutineLoadProperties(); - // check custom properties - checkCustomProperties(); + // check routine load job properties include desired concurrent number etc. + checkJobProperties(); + // check data source properties + checkDataSourceProperties(); } - private void checkDBTableName() throws AnalysisException { - if (Strings.isNullOrEmpty(dbTableName.getDb())) { - String dbName = ConnectContext.get().getDatabase(); - if (Strings.isNullOrEmpty(dbName)) { - throw new AnalysisException("please choose a database first"); - } - dbTableName.setDb(dbName); - } - if (Strings.isNullOrEmpty(dbTableName.getTbl())) { - throw new AnalysisException("empty table name in create routine load statement"); + public void checkDBTable(Analyzer analyzer) throws AnalysisException { + labelName.analyze(analyzer); + dbName = labelName.getDbName(); + name = labelName.getLabelName(); + if (Strings.isNullOrEmpty(tableName)) { + throw new AnalysisException("Table name should not be null"); } } - private void checkLoadProperties(Analyzer analyzer) throws AnalysisException { + public void checkLoadProperties(Analyzer analyzer) throws UserException { if (loadPropertyList == null) { return; } ColumnSeparator columnSeparator = null; - LoadColumnsInfo columnsInfo = null; - Expr wherePredicate = null; - PartitionNames partitionNames = null; + ImportColumnsStmt importColumnsStmt = null; + ImportWhereStmt importWhereStmt = null; + List partitionNames = null; for (ParseNode parseNode : loadPropertyList) { if (parseNode instanceof ColumnSeparator) { // check column separator @@ -212,63 +245,62 @@ private void checkLoadProperties(Analyzer analyzer) throws AnalysisException { throw new AnalysisException("repeat setting of column separator"); } columnSeparator = (ColumnSeparator) parseNode; - columnSeparator.analyze(analyzer); - } else if (parseNode instanceof LoadColumnsInfo) { + columnSeparator.analyze(null); + } else if (parseNode instanceof ImportColumnsStmt) { // check columns info - if (columnsInfo != null) { + if (importColumnsStmt != null) { throw new AnalysisException("repeat setting of columns info"); } - columnsInfo = (LoadColumnsInfo) parseNode; - columnsInfo.analyze(analyzer); - } else if (parseNode instanceof Expr) { + importColumnsStmt = (ImportColumnsStmt) parseNode; + } else if (parseNode instanceof ImportWhereStmt) { // check where expr - if (wherePredicate != null) { + if (importWhereStmt != null) { throw new AnalysisException("repeat setting of where predicate"); } - wherePredicate = (Expr) parseNode; - wherePredicate.analyze(analyzer); + importWhereStmt = (ImportWhereStmt) parseNode; } else if (parseNode instanceof PartitionNames) { // check partition names if (partitionNames != null) { throw new AnalysisException("repeat setting of partition names"); } - partitionNames = (PartitionNames) parseNode; - partitionNames.analyze(analyzer); + PartitionNames partitionNamesNode = (PartitionNames) parseNode; + partitionNamesNode.analyze(null); + partitionNames = partitionNamesNode.getPartitionNames(); } } - routineLoadDesc = new RoutineLoadDesc(columnSeparator, columnsInfo, wherePredicate, - partitionNames.getPartitionNames()); + routineLoadDesc = new RoutineLoadDesc(columnSeparator, importColumnsStmt, importWhereStmt, + partitionNames); } - private void checkRoutineLoadProperties() throws AnalysisException { - Optional optional = properties.keySet().parallelStream() - .filter(entity -> !PROPERTIES_SET.contains(entity)).findFirst(); + private void checkJobProperties() throws AnalysisException { + Optional optional = jobProperties.keySet().parallelStream().filter( + entity -> !PROPERTIES_SET.contains(entity)).findFirst(); if (optional.isPresent()) { throw new AnalysisException(optional.get() + " is invalid property"); } - // check desired concurrent number - final String desiredConcurrentNumberString = properties.get(DESIRED_CONCURRENT_NUMBER_PROPERTY); - if (desiredConcurrentNumberString != null) { - desiredConcurrentNum = getIntegerValueFromString(desiredConcurrentNumberString, - DESIRED_CONCURRENT_NUMBER_PROPERTY); - if (desiredConcurrentNum <= 0) { - throw new AnalysisException(DESIRED_CONCURRENT_NUMBER_PROPERTY + " must be greater then 0"); - } - } - - // check max error number - final String maxErrorNumberString = properties.get(MAX_ERROR_NUMBER_PROPERTY); - if (maxErrorNumberString != null) { - maxErrorNum = getIntegerValueFromString(maxErrorNumberString, MAX_ERROR_NUMBER_PROPERTY); - if (maxErrorNum < 0) { - throw new AnalysisException(MAX_ERROR_NUMBER_PROPERTY + " must be greater then or equal to 0"); - } - - } + desiredConcurrentNum = ((Long) Util.getLongPropertyOrDefault(jobProperties.get(DESIRED_CONCURRENT_NUMBER_PROPERTY), + Config.max_routine_load_task_concurrent_num, DESIRED_CONCURRENT_NUMBER_PRED, + DESIRED_CONCURRENT_NUMBER_PROPERTY + " should > 0")).intValue(); + + maxErrorNum = Util.getLongPropertyOrDefault(jobProperties.get(MAX_ERROR_NUMBER_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_ERROR_NUM, MAX_ERROR_NUMBER_PRED, + MAX_ERROR_NUMBER_PROPERTY + " should >= 0"); + + maxBatchIntervalS = Util.getLongPropertyOrDefault(jobProperties.get(MAX_BATCH_INTERVAL_SEC_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_INTERVAL_SECOND, MAX_BATCH_INTERVAL_PRED, + MAX_BATCH_INTERVAL_SEC_PROPERTY + " should between 5 and 60"); + + maxBatchRows = Util.getLongPropertyOrDefault(jobProperties.get(MAX_BATCH_ROWS_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_BATCH_ROWS, MAX_BATCH_ROWS_PRED, + MAX_BATCH_ROWS_PROPERTY + " should > 200000"); + + maxBatchSizeBytes = Util.getLongPropertyOrDefault(jobProperties.get(MAX_BATCH_SIZE_PROPERTY), + RoutineLoadJob.DEFAULT_MAX_BATCH_SIZE, MAX_BATCH_SIZE_PRED, + MAX_BATCH_SIZE_PROPERTY + " should between 100MB and 1GB"); } - private void checkCustomProperties() throws AnalysisException { + private void checkDataSourceProperties() throws AnalysisException { LoadDataSourceType type; try { type = LoadDataSourceType.valueOf(typeName); @@ -277,53 +309,95 @@ private void checkCustomProperties() throws AnalysisException { } switch (type) { case KAFKA: - checkKafkaCustomProperties(); + checkKafkaProperties(); break; default: break; } } - private void checkKafkaCustomProperties() throws AnalysisException { - Optional optional = customProperties.keySet().parallelStream() + private void checkKafkaProperties() throws AnalysisException { + Optional optional = dataSourceProperties.keySet().parallelStream() .filter(entity -> !KAFKA_PROPERTIES_SET.contains(entity)).findFirst(); if (optional.isPresent()) { throw new AnalysisException(optional.get() + " is invalid kafka custom property"); } - // check endpoint - kafkaEndpoint = customProperties.get(KAFKA_ENDPOINT_PROPERTY); - if (Strings.isNullOrEmpty(kafkaEndpoint)) { - throw new AnalysisException(KAFKA_ENDPOINT_PROPERTY + " is required property"); + + // check broker list + kafkaBrokerList = Strings.nullToEmpty(dataSourceProperties.get(KAFKA_BROKER_LIST_PROPERTY)).replaceAll(" ", ""); + if (Strings.isNullOrEmpty(kafkaBrokerList)) { + throw new AnalysisException(KAFKA_BROKER_LIST_PROPERTY + " is a required property"); } - if (!Pattern.matches(ENDPOINT_REGEX, kafkaEndpoint)) { - throw new AnalysisException(KAFKA_ENDPOINT_PROPERTY + " not match pattern " + ENDPOINT_REGEX); + String[] kafkaBrokerList = this.kafkaBrokerList.split(","); + for (String broker : kafkaBrokerList) { + if (!Pattern.matches(ENDPOINT_REGEX, broker)) { + throw new AnalysisException(KAFKA_BROKER_LIST_PROPERTY + ":" + broker + + " not match pattern " + ENDPOINT_REGEX); + } } + // check topic - kafkaTopic = customProperties.get(KAFKA_TOPIC_PROPERTY); + kafkaTopic = Strings.nullToEmpty(dataSourceProperties.get(KAFKA_TOPIC_PROPERTY)).replaceAll(" ", ""); if (Strings.isNullOrEmpty(kafkaTopic)) { - throw new AnalysisException(KAFKA_TOPIC_PROPERTY + " is required property"); + throw new AnalysisException(KAFKA_TOPIC_PROPERTY + " is a required property"); } + // check partitions - final String kafkaPartitionsString = customProperties.get(KAFKA_PARTITIONS_PROPERTY); + final String kafkaPartitionsString = dataSourceProperties.get(KAFKA_PARTITIONS_PROPERTY); if (kafkaPartitionsString != null) { - kafkaPartitions = new ArrayList<>(); - if (kafkaPartitionsString.equals(EMPTY_STRING)) { + kafkaPartitionsString.replaceAll(" ", ""); + if (kafkaPartitionsString.isEmpty()) { throw new AnalysisException(KAFKA_PARTITIONS_PROPERTY + " could not be a empty string"); } String[] kafkaPartionsStringList = kafkaPartitionsString.split(","); for (String s : kafkaPartionsStringList) { try { - kafkaPartitions.add(getIntegerValueFromString(s, KAFKA_PARTITIONS_PROPERTY)); + kafkaPartitionOffsets.add(Pair.create(getIntegerValueFromString(s, KAFKA_PARTITIONS_PROPERTY), + KafkaProgress.OFFSET_END_VAL)); } catch (AnalysisException e) { throw new AnalysisException(KAFKA_PARTITIONS_PROPERTY + " must be a number string with comma-separated"); } } } + + // check offset + final String kafkaOffsetsString = dataSourceProperties.get(KAFKA_OFFSETS_PROPERTY); + if (kafkaOffsetsString != null) { + kafkaOffsetsString.replaceAll(" ", ""); + if (kafkaOffsetsString.isEmpty()) { + throw new AnalysisException(KAFKA_OFFSETS_PROPERTY + " could not be a empty string"); + } + String[] kafkaOffsetsStringList = kafkaOffsetsString.split(","); + if (kafkaOffsetsStringList.length != kafkaPartitionOffsets.size()) { + throw new AnalysisException("Partitions number should be equals to offsets number"); + } + + for (int i = 0; i < kafkaOffsetsStringList.length; i++) { + // defined in librdkafka/rdkafkacpp.h + // OFFSET_BEGINNING: -2 + // OFFSET_END: -1 + try { + kafkaPartitionOffsets.get(i).second = getLongValueFromString(kafkaOffsetsStringList[i], + KAFKA_OFFSETS_PROPERTY); + if (kafkaPartitionOffsets.get(i).second < 0) { + throw new AnalysisException("Cannot specify offset smaller than 0"); + } + } catch (AnalysisException e) { + if (kafkaOffsetsStringList[i].equalsIgnoreCase(KafkaProgress.OFFSET_BEGINNING)) { + kafkaPartitionOffsets.get(i).second = KafkaProgress.OFFSET_BEGINNING_VAL; + } else if (kafkaOffsetsStringList[i].equalsIgnoreCase(KafkaProgress.OFFSET_END)) { + kafkaPartitionOffsets.get(i).second = KafkaProgress.OFFSET_END_VAL; + } else { + throw e; + } + } + } + } } private int getIntegerValueFromString(String valueString, String propertyName) throws AnalysisException { - if (valueString.equals(EMPTY_STRING)) { + if (valueString.isEmpty()) { throw new AnalysisException(propertyName + " could not be a empty string"); } int value; @@ -334,4 +408,17 @@ private int getIntegerValueFromString(String valueString, String propertyName) t } return value; } + + private long getLongValueFromString(String valueString, String propertyName) throws AnalysisException { + if (valueString.isEmpty()) { + throw new AnalysisException(propertyName + " could not be a empty string"); + } + long value; + try { + value = Long.valueOf(valueString); + } catch (NumberFormatException e) { + throw new AnalysisException(propertyName + " must be a integer"); + } + return value; + } } diff --git a/fe/src/main/java/org/apache/doris/analysis/ImportColumnDesc.java b/fe/src/main/java/org/apache/doris/analysis/ImportColumnDesc.java index 38a91fce551102..1274fa71cf958e 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ImportColumnDesc.java +++ b/fe/src/main/java/org/apache/doris/analysis/ImportColumnDesc.java @@ -21,20 +21,20 @@ * Created by zhaochun on 2018/4/23. */ public class ImportColumnDesc { - private String column; + private String columnName; private Expr expr; public ImportColumnDesc(String column) { - this.column = column; + this.columnName = column; } public ImportColumnDesc(String column, Expr expr) { - this.column = column; + this.columnName = column; this.expr = expr; } - public String getColumn() { - return column; + public String getColumnName() { + return columnName; } public Expr getExpr() { @@ -44,4 +44,14 @@ public Expr getExpr() { public boolean isColumn() { return expr == null; } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(columnName); + if (expr != null) { + sb.append("=").append(expr.toSql()); + } + return sb.toString(); + } } diff --git a/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java b/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java index a664ce8d814a19..5c27198b4609a0 100644 --- a/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/InsertStmt.java @@ -17,7 +17,6 @@ package org.apache.doris.analysis; -import com.google.common.base.Preconditions; import org.apache.doris.catalog.BrokerTable; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; @@ -40,9 +39,11 @@ import org.apache.doris.planner.OlapTableSink; import org.apache.doris.qe.ConnectContext; import org.apache.doris.rewrite.ExprRewriter; +import org.apache.doris.service.FrontendOptions; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TransactionState.LoadJobSourceType; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -238,9 +239,9 @@ public void analyze(Analyzer analyzer) throws UserException { String jobLabel = "insert_" + uuid; LoadJobSourceType sourceType = isStreaming ? LoadJobSourceType.INSERT_STREAMING : LoadJobSourceType.FRONTEND; + long timeoutSecond = ConnectContext.get().getSessionVariable().getQueryTimeoutS(); transactionId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction(db.getId(), - jobLabel, - "fe", sourceType); + jobLabel, "FE: " + FrontendOptions.getLocalHostAddress(), sourceType, timeoutSecond); if (isStreaming) { OlapTableSink sink = (OlapTableSink) dataSink; TUniqueId loadId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); diff --git a/fe/src/main/java/org/apache/doris/analysis/LoadColumnsInfo.java b/fe/src/main/java/org/apache/doris/analysis/LoadColumnsInfo.java index 07cc6385eb5a63..8fa4baea6ade81 100644 --- a/fe/src/main/java/org/apache/doris/analysis/LoadColumnsInfo.java +++ b/fe/src/main/java/org/apache/doris/analysis/LoadColumnsInfo.java @@ -17,25 +17,30 @@ package org.apache.doris.analysis; -import com.google.common.base.Joiner; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.Pair; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +/* + * LoadColumnsInfo saves all columns' mapping expression + */ public class LoadColumnsInfo implements ParseNode { private final List columnNames; private final List columnMappingList; + // the following maps are parsed from 'columnMappingList' + // col name -> (func name -> func args) private Map>> columnToFunction; private Map parsedExprMap; @@ -44,6 +49,10 @@ public LoadColumnsInfo(List columnNames, List columnMappingList) { this.columnMappingList = columnMappingList; } + public Map getParsedExprMap() { + return parsedExprMap; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException { checkColumnNames(); @@ -87,27 +96,28 @@ private void checkColumnMapping() throws AnalysisException { parsedExprMap = Maps.newHashMap(); for (Expr expr : columnMappingList) { if (!(expr instanceof BinaryPredicate)) { - throw new AnalysisException("Mapping function expr error. expr: " + expr.toSql()); + throw new AnalysisException("Mapping function should only be binary predicate: " + expr.toSql()); } BinaryPredicate predicate = (BinaryPredicate) expr; if (predicate.getOp() != BinaryPredicate.Operator.EQ) { - throw new AnalysisException("Mapping function operator error. op: " + predicate.getOp()); + throw new AnalysisException("Mapping function should only be binary predicate with EQ operator: " + + predicate.getOp()); } Expr child0 = predicate.getChild(0); if (!(child0 instanceof SlotRef)) { - throw new AnalysisException("Mapping column error. column: " + child0.toSql()); + throw new AnalysisException("Mapping function's left child should be a column name: " + child0.toSql()); } String column = ((SlotRef) child0).getColumnName(); if (columnToFunction.containsKey(column)) { - throw new AnalysisException("Duplicate column mapping: " + column); + throw new AnalysisException("Duplicate mapping for column: " + column); } Expr child1 = predicate.getChild(1); if (!(child1 instanceof FunctionCallExpr)) { - throw new AnalysisException("Mapping function error, function: " + child1.toSql()); + throw new AnalysisException("Mapping function's right child should be a funcation: " + child1.toSql()); } if (!child1.supportSerializable()) { diff --git a/fe/src/main/java/org/apache/doris/analysis/PauseRoutineLoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/PauseRoutineLoadStmt.java index a7227211ad428d..18f7633806c0c7 100644 --- a/fe/src/main/java/org/apache/doris/analysis/PauseRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/PauseRoutineLoadStmt.java @@ -25,26 +25,27 @@ Pause routine load by name syntax: - PAUSE ROUTINE LOAD name + PAUSE ROUTINE LOAD [database.]name */ public class PauseRoutineLoadStmt extends DdlStmt { - private final String name; + private final LabelName labelName; - public PauseRoutineLoadStmt(String name) { - this.name = name; + public PauseRoutineLoadStmt(LabelName labelName) { + this.labelName = labelName; } public String getName() { - return name; + return labelName.getLabelName(); + } + + public String getDbFullName(){ + return labelName.getDbName(); } @Override public void analyze(Analyzer analyzer) throws AnalysisException, UserException { super.analyze(analyzer); - - if (Strings.isNullOrEmpty(name)) { - throw new AnalysisException("routine load name could not be empty or null"); - } + labelName.analyze(analyzer); } } diff --git a/fe/src/main/java/org/apache/doris/analysis/ResumeRoutineLoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/ResumeRoutineLoadStmt.java index 28bd4ae6833840..dc670c91a36085 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ResumeRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ResumeRoutineLoadStmt.java @@ -25,25 +25,27 @@ Resume routine load job by name syntax: - RESUME ROUTINE LOAD name + RESUME ROUTINE LOAD [database.]name */ public class ResumeRoutineLoadStmt extends DdlStmt{ - private final String name; + private final LabelName labelName; - public ResumeRoutineLoadStmt(String name) { - this.name = name; + public ResumeRoutineLoadStmt(LabelName labelName) { + this.labelName = labelName; } public String getName() { - return name; + return labelName.getLabelName(); + } + + public String getDbFullName() { + return labelName.getDbName(); } @Override public void analyze(Analyzer analyzer) throws AnalysisException, UserException { super.analyze(analyzer); - if (Strings.isNullOrEmpty(name)) { - throw new AnalysisException("routine load name could not be empty or null"); - } + labelName.analyze(analyzer); } } diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java index d6868b3b9ae8a5..1bba28f30e2a6c 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java @@ -18,8 +18,8 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.ScalarType; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowProcStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowProcStmt.java index 10a5657d379f49..561fbd05049194 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowProcStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowProcStmt.java @@ -79,4 +79,13 @@ public ShowResultSetMetaData getMetaData() { } return builder.build(); } + + @Override + public RedirectStatus getRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } } diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java index 3ff0d283ae2f50..2043dee5385f19 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java @@ -18,50 +18,115 @@ package org.apache.doris.analysis; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableList; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.qe.ShowResultSetMetaData; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; + +import java.util.List; + /* Show routine load progress by routine load name syntax: - SHOW ROUTINE LOAD name + SHOW [ALL] ROUTINE LOAD [database.][name] + + without ALL: only show job which is not final + with ALL: show all of job include history job + + without name: show all of routine load job in database with different name + with name: show all of job named ${name} in database + + without on db: show all of job in connection db + if user does not choose db before, return error + with on db: show all of job in ${db} + + example: + show routine load named test in database1 + SHOW ROUTINE LOAD database1.test; + + show routine load in database1 + SHOW ROUTINE LOAD database1; + + show routine load in database1 include history + use database1; + SHOW ALL ROUTINE LOAD; + + show routine load in all of database + please use show proc */ public class ShowRoutineLoadStmt extends ShowStmt { private static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() - .add("id") - .add("name") - .add("db_id") - .add("table_id") - .add("partitions") - .add("state") - .add(CreateRoutineLoadStmt.DESIRED_CONCURRENT_NUMBER_PROPERTY) - .add("progress") + .add("Id") + .add("Name") + .add("CreateTime") + .add("PauseTime") + .add("EndTime") + .add("DbName") + .add("TableName") + .add("State") + .add("DataSourceType") + .add("CurrentTaskNum") + .add("JobProperties") + .add("DataSourceProperties") + .add("Statistic") + .add("Progress") + .add("ReasonOfStateChanged") + .add("ErrorLogUrls") .build(); - private final String name; + private final LabelName labelName; + private String dbFullName; // optional + private String name; // optional + private boolean includeHistory = false; - public ShowRoutineLoadStmt(String name) { - this.name = name; + + public ShowRoutineLoadStmt(LabelName labelName, boolean includeHistory) { + this.labelName = labelName; + this.includeHistory = includeHistory; + } + + public String getDbFullName() { + return dbFullName; } public String getName() { return name; } + public boolean isIncludeHistory() { + return includeHistory; + } + @Override - public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); - if (Strings.isNullOrEmpty(name)) { - throw new AnalysisException("routine load name could not be empty or null"); + checkLabelName(analyzer); + } + + private void checkLabelName(Analyzer analyzer) throws AnalysisException { + String dbName = labelName == null ? null : labelName.getDbName(); + if (Strings.isNullOrEmpty(dbName)) { + dbFullName = analyzer.getContext().getDatabase(); + if (Strings.isNullOrEmpty(dbFullName)) { + throw new AnalysisException("please choose a database firstly " + + "such as use db, show routine load db.name etc."); + } + } else { + dbFullName = ClusterNamespace.getFullName(getClusterName(), dbName); } + name = labelName == null ? null : labelName.getLabelName(); + } + + public static List getTitleNames() { + return TITLE_NAMES; } @Override @@ -73,4 +138,9 @@ public ShowResultSetMetaData getMetaData() { } return builder.build(); } + + @Override + public RedirectStatus getRedirectStatus() { + return RedirectStatus.FORWARD_NO_SYNC; + } } diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadTaskStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadTaskStmt.java new file mode 100644 index 00000000000000..cf8fb645013108 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadTaskStmt.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.analysis; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.qe.ShowResultSetMetaData; + +import java.util.Arrays; +import java.util.List; + +/* + show all of task belong to job + SHOW ROUTINE LOAD TASK FROM DB where expr; + + where expr: JobName=xxx + */ +public class ShowRoutineLoadTaskStmt extends ShowStmt { + private static final List supportColumn = Arrays.asList("jobname"); + private static final ImmutableList TITLE_NAMES = + new ImmutableList.Builder() + .add("TaskId") + .add("TxnId") + .add("JobId") + .add("CreateTime") + .add("ExecuteStartTime") + .add("BeId") + .add("DataSourceProperties") + .build(); + + private final String dbName; + private final Expr jobNameExpr; + + private String jobName; + private String dbFullName; + + public ShowRoutineLoadTaskStmt(String dbName, Expr jobNameExpr) { + this.dbName = dbName; + this.jobNameExpr = jobNameExpr; + } + + public String getJobName() { + return jobName; + } + + public String getDbFullName() { + return dbFullName; + } + + @Override + public void analyze(Analyzer analyzer) throws UserException { + super.analyze(analyzer); + checkDB(analyzer); + checkJobNameExpr(analyzer); + } + + private void checkDB(Analyzer analyzer) throws AnalysisException { + if (Strings.isNullOrEmpty(dbName)) { + if (Strings.isNullOrEmpty(analyzer.getDefaultDb())) { + throw new AnalysisException("please designate a database in show stmt"); + } + dbFullName = analyzer.getDefaultDb(); + } else { + dbFullName = ClusterNamespace.getFullName(analyzer.getClusterName(), dbName); + } + } + + private void checkJobNameExpr(Analyzer analyzer) throws AnalysisException { + if (jobNameExpr == null) { + throw new AnalysisException("please designate a jobName in where expr such as JobName=\"ILoveDoris\""); + } + + boolean valid = true; + CHECK: + { + // check predicate + if (!(jobNameExpr instanceof BinaryPredicate)) { + valid = false; + break CHECK; + } + BinaryPredicate binaryPredicate = (BinaryPredicate) jobNameExpr; + if (binaryPredicate.getOp() != BinaryPredicate.Operator.EQ) { + valid = false; + break CHECK; + } + + // check child(0) + if (!(binaryPredicate.getChild(0) instanceof SlotRef)) { + valid = false; + break CHECK; + } + SlotRef slotRef = (SlotRef) binaryPredicate.getChild(0); + if (!supportColumn.stream().anyMatch(entity -> entity.equals(slotRef.getColumnName().toLowerCase()))) { + valid = false; + break CHECK; + } + + // check child(1) + if (!(binaryPredicate.getChild(1) instanceof StringLiteral)) { + valid = false; + break CHECK; + } + StringLiteral stringLiteral = (StringLiteral) binaryPredicate.getChild(1); + jobName = stringLiteral.getValue().toLowerCase(); + } + + if (!valid) { + throw new AnalysisException("show routine load job only support one equal expr which is sames like JobName=\"ILoveDoris\""); + } + } + + @Override + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + public static List getTitleNames() { + return TITLE_NAMES; + } +} diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java index 3083580cf9bc79..9de02136341ca0 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java @@ -119,4 +119,13 @@ public ShowResultSetMetaData getMetaData() { } return builder.build(); } + + @Override + public RedirectStatus getRedirectStatus() { + if (ConnectContext.get().getSessionVariable().getForwardToMaster()) { + return RedirectStatus.FORWARD_NO_SYNC; + } else { + return RedirectStatus.NO_FORWARD; + } + } } diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java index b78a884a43f4fe..f0b2f2ad1609a1 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowVariablesStmt.java @@ -18,15 +18,15 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.InfoSchemaDb; +import org.apache.doris.catalog.ScalarType; import org.apache.doris.qe.ShowResultSetMetaData; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.LogManager; - import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + // Show variables statement. public class ShowVariablesStmt extends ShowStmt { private static final Logger LOG = LogManager.getLogger(ShowVariablesStmt.class); @@ -100,7 +100,7 @@ public SelectStmt toSelectStmt(Analyzer analyzer) { selectStmt = new SelectStmt(selectList, new FromClause(Lists.newArrayList(new TableRef(tableName, null))), where, null, null, null, LimitElement.NO_LIMIT); - LOG.info("select Stmt is {}", selectStmt.toSql()); + LOG.debug("select stmt is {}", selectStmt.toSql()); // DB: type // table: thread id diff --git a/fe/src/main/java/org/apache/doris/analysis/StopRoutineLoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/StopRoutineLoadStmt.java index a8f15e4150082a..96bcce2882c68c 100644 --- a/fe/src/main/java/org/apache/doris/analysis/StopRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/StopRoutineLoadStmt.java @@ -26,25 +26,27 @@ Stop routine load job by name syntax: - STOP ROUTINE LOAD name + STOP ROUTINE LOAD [database.]name */ public class StopRoutineLoadStmt extends DdlStmt { - private final String name; + private final LabelName labelName; - public StopRoutineLoadStmt(String name) { - this.name = name; + public StopRoutineLoadStmt(LabelName labelName) { + this.labelName = labelName; } public String getName() { - return name; + return labelName.getLabelName(); + } + + public String getDbFullName() { + return labelName.getDbName(); } @Override public void analyze(Analyzer analyzer) throws AnalysisException, UserException { super.analyze(analyzer); - if (Strings.isNullOrEmpty(name)) { - throw new AnalysisException("routine load name could not be empty or null"); - } + labelName.analyze(analyzer); } } diff --git a/fe/src/main/java/org/apache/doris/analysis/TupleDescriptor.java b/fe/src/main/java/org/apache/doris/analysis/TupleDescriptor.java index b1b3983ce72acc..7b1036a7f0ece8 100644 --- a/fe/src/main/java/org/apache/doris/analysis/TupleDescriptor.java +++ b/fe/src/main/java/org/apache/doris/analysis/TupleDescriptor.java @@ -17,21 +17,22 @@ package org.apache.doris.analysis; -import java.util.ArrayList; -import java.util.List; - -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.LogManager; - import org.apache.doris.catalog.ColumnStats; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.Table; import org.apache.doris.thrift.TTupleDescriptor; + import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.List; + public class TupleDescriptor { private static final Logger LOG = LogManager.getLogger(TupleDescriptor.class); private final TupleId id; @@ -216,7 +217,7 @@ public void computeMemLayout() { } this.byteSize = offset; - LOG.debug("tuple is {}", byteSize); + // LOG.debug("tuple is {}", byteSize); } /** diff --git a/fe/src/main/java/org/apache/doris/catalog/BrokerMgr.java b/fe/src/main/java/org/apache/doris/catalog/BrokerMgr.java index 8ef0e5d4dc7d4c..4600f9d1a5af49 100644 --- a/fe/src/main/java/org/apache/doris/catalog/BrokerMgr.java +++ b/fe/src/main/java/org/apache/doris/catalog/BrokerMgr.java @@ -119,10 +119,14 @@ public FsBroker getBroker(String brokerName, String host) throws AnalysisExcepti throw new AnalysisException("Unknown broker name(" + brokerName + ")"); } List brokers = brokerAddsMap.get(host); - if (brokers.isEmpty()) { - brokers = brokerListMap.get(brokerName); + for (FsBroker fsBroker : brokers) { + if (fsBroker.isAlive) { + return fsBroker; + } } + // not find, get an arbitrary one + brokers = brokerListMap.get(brokerName); Collections.shuffle(brokers); for (FsBroker fsBroker : brokers) { if (fsBroker.isAlive) { @@ -130,7 +134,7 @@ public FsBroker getBroker(String brokerName, String host) throws AnalysisExcepti } } - throw new AnalysisException("failed to find alive broker"); + throw new AnalysisException("failed to find alive broker: " + brokerName); } finally { lock.unlock(); } diff --git a/fe/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/src/main/java/org/apache/doris/catalog/Catalog.java index 01dcea209a8d7d..da9d5253b4d740 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/src/main/java/org/apache/doris/catalog/Catalog.java @@ -133,6 +133,8 @@ import org.apache.doris.load.LoadJob; import org.apache.doris.load.LoadJob.JobState; import org.apache.doris.load.routineload.RoutineLoadManager; +import org.apache.doris.load.routineload.RoutineLoadScheduler; +import org.apache.doris.load.routineload.RoutineLoadTaskScheduler; import org.apache.doris.master.Checkpoint; import org.apache.doris.master.MetaHelper; import org.apache.doris.meta.MetaContext; @@ -349,6 +351,10 @@ public class Catalog { private TabletChecker tabletChecker; + private RoutineLoadScheduler routineLoadScheduler; + + private RoutineLoadTaskScheduler routineLoadTaskScheduler; + public List getFrontends(FrontendNodeType nodeType) { if (nodeType == null) { // get all @@ -466,6 +472,9 @@ private Catalog() { this.stat = new TabletSchedulerStat(); this.tabletScheduler = new TabletScheduler(this, systemInfo, tabletInvertedIndex, stat); this.tabletChecker = new TabletChecker(this, systemInfo, tabletScheduler, stat); + + this.routineLoadScheduler = new RoutineLoadScheduler(routineLoadManager); + this.routineLoadTaskScheduler = new RoutineLoadTaskScheduler(routineLoadManager); } public static void destroyCheckpoint() { @@ -649,7 +658,6 @@ public void initialize(String[] args) throws Exception { // the clear threads runs every min(transaction_clean_interval_second,stream_load_default_timeout_second)/10 txnCleaner.setInterval(Math.min(Config.transaction_clean_interval_second, Config.stream_load_default_timeout_second) * 100L); - } private void getClusterIdAndRole() throws IOException { @@ -1116,6 +1124,11 @@ private void transferToMaster() throws IOException { domainResolver.start(); tabletStatMgr.start(); + + // start routine load scheduler + routineLoadScheduler.start(); + routineLoadTaskScheduler.start(); + MetricRepo.init(); } @@ -1280,14 +1293,11 @@ public void loadImage(String imageDir) throws IOException, DdlException { try { checksum = loadHeader(dis, checksum); checksum = loadMasterInfo(dis, checksum); - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_22) { - checksum = loadFrontends(dis, checksum); - } + checksum = loadFrontends(dis, checksum); checksum = Catalog.getCurrentSystemInfo().loadBackends(dis, checksum); checksum = loadDb(dis, checksum); // ATTN: this should be done after load Db, and before loadAlterJob recreateTabletInvertIndex(); - checksum = loadLoadJob(dis, checksum); checksum = loadAlterJob(dis, checksum); checksum = loadBackupAndRestoreJob_D(dis, checksum); @@ -1299,10 +1309,9 @@ public void loadImage(String imageDir) throws IOException, DdlException { checksum = loadExportJob(dis, checksum); checksum = loadBackupHandler(dis, checksum); checksum = loadPaloAuth(dis, checksum); - if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_45) { - checksum = loadTransactionState(dis, checksum); - } + checksum = loadTransactionState(dis, checksum); checksum = loadColocateTableIndex(dis, checksum); + checksum = loadRoutineLoadJobs(dis, checksum); long remoteChecksum = dis.readLong(); Preconditions.checkState(remoteChecksum == checksum, remoteChecksum + " vs. " + checksum); @@ -1385,24 +1394,27 @@ public long loadMasterInfo(DataInputStream dis, long checksum) throws IOExceptio } public long loadFrontends(DataInputStream dis, long checksum) throws IOException { - int size = dis.readInt(); - long newChecksum = checksum ^ size; - for (int i = 0; i < size; i++) { - Frontend fe = Frontend.read(dis); - replayAddFrontend(fe); - } - - size = dis.readInt(); - newChecksum ^= size; - for (int i = 0; i < size; i++) { - if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_41) { + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_22) { + int size = dis.readInt(); + long newChecksum = checksum ^ size; + for (int i = 0; i < size; i++) { Frontend fe = Frontend.read(dis); - removedFrontends.add(fe.getNodeName()); - } else { - removedFrontends.add(Text.readString(dis)); + replayAddFrontend(fe); + } + + size = dis.readInt(); + newChecksum ^= size; + for (int i = 0; i < size; i++) { + if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_41) { + Frontend fe = Frontend.read(dis); + removedFrontends.add(fe.getNodeName()); + } else { + removedFrontends.add(Text.readString(dis)); + } } + return newChecksum; } - return newChecksum; + return checksum; } public long loadDb(DataInputStream dis, long checksum) throws IOException, DdlException { @@ -1688,10 +1700,13 @@ public long loadAccessService(DataInputStream dis, long checksum) throws IOExcep } public long loadTransactionState(DataInputStream dis, long checksum) throws IOException { - int size = dis.readInt(); - long newChecksum = checksum ^ size; - globalTransactionMgr.readFields(dis); - return newChecksum; + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_45) { + int size = dis.readInt(); + long newChecksum = checksum ^ size; + globalTransactionMgr.readFields(dis); + return newChecksum; + } + return checksum; } public long loadRecycleBin(DataInputStream dis, long checksum) throws IOException { @@ -1712,6 +1727,13 @@ public long loadColocateTableIndex(DataInputStream dis, long checksum) throws IO return checksum; } + public long loadRoutineLoadJobs(DataInputStream dis, long checksum) throws IOException { + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_49) { + Catalog.getCurrentCatalog().getRoutineLoadManager().readFields(dis); + } + return checksum; + } + // Only called by checkpoint thread public void saveImage() throws IOException { // Write image.ckpt @@ -1756,6 +1778,7 @@ public void saveImage(File curFile, long replayedJournalId) throws IOException { checksum = savePaloAuth(dos, checksum); checksum = saveTransactionState(dos, checksum); checksum = saveColocateTableIndex(dos, checksum); + checksum = saveRoutineLoadJobs(dos, checksum); dos.writeLong(checksum); } finally { dos.close(); @@ -1987,6 +2010,11 @@ public long saveColocateTableIndex(DataOutputStream dos, long checksum) throws I return checksum; } + public long saveRoutineLoadJobs(DataOutputStream dos, long checksum) throws IOException { + Catalog.getCurrentCatalog().getRoutineLoadManager().write(dos); + return checksum; + } + // global variable persistence public long loadGlobalVariable(DataInputStream in, long checksum) throws IOException, DdlException { if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_22) { @@ -3918,7 +3946,8 @@ public static void getDdlStmt(Table table, List createTableStmt, List tableNames) // erase db with same name eraseDatabaseWithSameName(db.getFullName()); - // recylce db + // recycle db RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames); idToDatabase.put(db.getId(), databaseInfo); idToRecycleTime.put(db.getId(), System.currentTimeMillis()); - LOG.info("recycle db[{}]", db.getId()); + LOG.info("recycle db[{}-{}]", db.getId(), db.getFullName()); return true; } @@ -93,11 +93,11 @@ public synchronized boolean recycleTable(long dbId, Table table) { // erase table with same name eraseTableWithSameName(dbId, table.getName()); - // recylce table + // recycle table RecycleTableInfo tableInfo = new RecycleTableInfo(dbId, table); idToRecycleTime.put(table.getId(), System.currentTimeMillis()); idToTable.put(table.getId(), tableInfo); - LOG.info("recycle table[{}]", table.getId()); + LOG.info("recycle table[{}-{}]", table.getId(), table.getName()); return true; } @@ -112,12 +112,12 @@ public synchronized boolean recyclePartition(long dbId, long tableId, Partition // erase partition with same name erasePartitionWithSameName(dbId, tableId, partition.getName()); - // recylce partition + // recycle partition RecyclePartitionInfo partitionInfo = new RecyclePartitionInfo(dbId, tableId, partition, range, dataProperty, replicationNum); idToRecycleTime.put(partition.getId(), System.currentTimeMillis()); idToPartition.put(partition.getId(), partitionInfo); - LOG.info("recycle partition[{}]", partition.getId()); + LOG.info("recycle partition[{}-{}]", partition.getId(), partition.getName()); return true; } diff --git a/fe/src/main/java/org/apache/doris/catalog/DiskInfo.java b/fe/src/main/java/org/apache/doris/catalog/DiskInfo.java index 5f2059c1497363..c4f4162db651f1 100644 --- a/fe/src/main/java/org/apache/doris/catalog/DiskInfo.java +++ b/fe/src/main/java/org/apache/doris/catalog/DiskInfo.java @@ -86,6 +86,10 @@ public void setAvailableCapacityB(long availableCapacityB) { this.diskAvailableCapacityB = availableCapacityB; } + public double getUsedPct() { + return (totalCapacityB - diskAvailableCapacityB) / (double) (totalCapacityB <= 0 ? 1 : totalCapacityB); + } + public DiskState getState() { return state; } diff --git a/fe/src/main/java/org/apache/doris/catalog/EsTable.java b/fe/src/main/java/org/apache/doris/catalog/EsTable.java index fdcb7c843adfc5..708ba4d992b39f 100644 --- a/fe/src/main/java/org/apache/doris/catalog/EsTable.java +++ b/fe/src/main/java/org/apache/doris/catalog/EsTable.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.zip.Adler32; +import org.apache.doris.external.EsNodeInfo; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -39,6 +40,8 @@ import com.google.common.base.Strings; import com.google.common.collect.Maps; +import javax.xml.soap.Node; + public class EsTable extends Table { private static final Logger LOG = LogManager.getLogger(EsTable.class); @@ -47,6 +50,10 @@ public class EsTable extends Table { public static final String PASSWORD = "password"; public static final String INDEX = "index"; public static final String TYPE = "type"; + public static final String TRANSPORT = "transport"; + + public static final String TRANSPORT_HTTP = "http"; + public static final String TRANSPORT_THRIFT = "thrift"; private String hosts; private String[] seeds; @@ -54,6 +61,7 @@ public class EsTable extends Table { private String passwd = ""; private String indexName; private String mappingType = "_doc"; + private String transport = "http"; // only save the partition definition, save the partition key, // partition list is got from es cluster dynamically and is saved in esTableState private PartitionInfo partitionInfo; @@ -63,7 +71,7 @@ public EsTable() { super(TableType.ELASTICSEARCH); } - public EsTable(long id, String name, List schema, + public EsTable(long id, String name, List schema, Map properties, PartitionInfo partitionInfo) throws DdlException { super(id, name, TableType.ELASTICSEARCH, schema); @@ -85,7 +93,7 @@ private void validate(Map properties) throws DdlException { hosts = properties.get(HOSTS).trim(); seeds = hosts.split(","); - if (!Strings.isNullOrEmpty(properties.get(USER)) + if (!Strings.isNullOrEmpty(properties.get(USER)) && !Strings.isNullOrEmpty(properties.get(USER).trim())) { userName = properties.get(USER).trim(); } @@ -106,8 +114,16 @@ private void validate(Map properties) throws DdlException { && !Strings.isNullOrEmpty(properties.get(TYPE).trim())) { mappingType = properties.get(TYPE).trim(); } + if (!Strings.isNullOrEmpty(properties.get(TRANSPORT)) + && !Strings.isNullOrEmpty(properties.get(TRANSPORT).trim())) { + transport = properties.get(TRANSPORT).trim(); + if (!(TRANSPORT_HTTP.equals(transport) || TRANSPORT_THRIFT.equals(transport))) { + throw new DdlException("transport of ES table must be http(recommend) or thrift(reserved inner usage)," + + " but value is " + transport); + } + } } - + public TTableDescriptor toThrift() { TEsTable tEsTable = new TEsTable(); TTableDescriptor tTableDescriptor = new TTableDescriptor(getId(), TTableType.ES_TABLE, @@ -137,7 +153,8 @@ public int getSignature(int signatureVersion) { adler32.update(indexName.getBytes(charsetName)); // mysql table adler32.update(mappingType.getBytes(charsetName)); - + // transport + adler32.update(transport.getBytes(charsetName)); } catch (UnsupportedEncodingException e) { LOG.error("encoding error", e); return -1; @@ -156,6 +173,7 @@ public void write(DataOutput out) throws IOException { Text.writeString(out, mappingType); Text.writeString(out, partitionInfo.getType().name()); partitionInfo.write(out); + Text.writeString(out, transport); } @Override @@ -175,12 +193,13 @@ public void readFields(DataInput in) throws IOException { } else { throw new IOException("invalid partition type: " + partType); } + transport = Text.readString(in); } public String getHosts() { return hosts; } - + public String[] getSeeds() { return seeds; } @@ -201,6 +220,10 @@ public String getMappingType() { return mappingType; } + public String getTransport() { + return transport; + } + public PartitionInfo getPartitionInfo() { return partitionInfo; } diff --git a/fe/src/main/java/org/apache/doris/catalog/Partition.java b/fe/src/main/java/org/apache/doris/catalog/Partition.java index 942e2bf3202255..791cfdd36ae915 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Partition.java +++ b/fe/src/main/java/org/apache/doris/catalog/Partition.java @@ -239,6 +239,10 @@ public long getDataSize() { return dataSize; } + public boolean hasData() { + return !(visibleVersion == PARTITION_INIT_VERSION && visibleVersionHash == PARTITION_INIT_VERSION_HASH); + } + public static Partition read(DataInput in) throws IOException { Partition partition = new Partition(); partition.readFields(in); diff --git a/fe/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/src/main/java/org/apache/doris/catalog/Tablet.java index 91ddd9845734ea..2a4fe448e9d5e8 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Tablet.java +++ b/fe/src/main/java/org/apache/doris/catalog/Tablet.java @@ -160,10 +160,22 @@ public Set getBackendIds() { return beIds; } - public List getBackendIdsList() { + public Set getAvailableBackendIds() { + Set beIds = Sets.newHashSet(); + for (Replica replica : replicas) { + if (Catalog.getCurrentSystemInfo().checkBackendAvailable(replica.getBackendId())) { + beIds.add(replica.getBackendId()); + } + } + return beIds; + } + + public List getAvailableBackendIdsList() { List beIds = Lists.newArrayList(); for (Replica replica : replicas) { - beIds.add(replica.getBackendId()); + if (Catalog.getCurrentSystemInfo().checkBackendAvailable(replica.getBackendId())) { + beIds.add(replica.getBackendId()); + } } return beIds; } diff --git a/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java b/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java index f3d74fe21459d8..1c13c042b3e65e 100644 --- a/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java +++ b/fe/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java @@ -141,6 +141,11 @@ public void tabletReport(long backendId, Map backendTablets, replica.setPathHash(backendTabletInfo.getPath_hash()); } + if (backendTabletInfo.isSetSchema_hash() + && replica.getSchemaHash() != backendTabletInfo.getSchema_hash()) { + replica.setSchemaHash(backendTabletInfo.getSchema_hash()); + } + if (needRecover(replica, tabletMeta.getOldSchemaHash(), backendTabletInfo)) { LOG.warn("replica {} of tablet {} on backend {} need recovery. " + "replica in FE: {}, report version {}-{}, report schema hash: {}," diff --git a/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java index f5caa75716efd0..e5177e2bbc361c 100644 --- a/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java +++ b/fe/src/main/java/org/apache/doris/clone/TabletSchedCtx.java @@ -498,8 +498,8 @@ public void chooseDestReplicaForVersionIncomplete(Map backendsWo continue; } - if (replica.getLastFailedVersion() <= 0 && replica.getVersion() == visibleVersion - && replica.getVersionHash() == visibleVersionHash) { + if (replica.getLastFailedVersion() <= 0 && ((replica.getVersion() == visibleVersion + && replica.getVersionHash() == visibleVersionHash) || replica.getVersion() > visibleVersion)) { // skip healthy replica continue; } diff --git a/fe/src/main/java/org/apache/doris/common/Config.java b/fe/src/main/java/org/apache/doris/common/Config.java index 7454e7022b34f8..2071232828816c 100644 --- a/fe/src/main/java/org/apache/doris/common/Config.java +++ b/fe/src/main/java/org/apache/doris/common/Config.java @@ -339,6 +339,18 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static int stream_load_default_timeout_second = 300; // 300s + /* + * Max stream load timeout + */ + @ConfField(mutable = true, masterOnly = true) + public static int max_stream_load_timeout_second = 172800; // 2days + + /* + * Min stream load timeout + */ + @ConfField(mutable = true, masterOnly = true) + public static int min_stream_load_timeout_second = 1; // 1s + /* * Default hadoop load timeout */ @@ -763,5 +775,11 @@ public class Config extends ConfigBase { * If set to true, metric collector will be run as a daemon timer to collect metrics at fix interval */ @ConfField public static boolean enable_metric_calculator = false; + + /* + * the max concurrent task num of a routine load task + */ + @ConfField(mutable = true, masterOnly = true) + public static int max_routine_load_task_concurrent_num = 5; } diff --git a/fe/src/main/java/org/apache/doris/common/FeConstants.java b/fe/src/main/java/org/apache/doris/common/FeConstants.java index 1fdf7c8b18cfda..21c40ea14fd31f 100644 --- a/fe/src/main/java/org/apache/doris/common/FeConstants.java +++ b/fe/src/main/java/org/apache/doris/common/FeConstants.java @@ -35,5 +35,5 @@ public class FeConstants { // general model // Current meta data version. Use this version to write journals and image - public static int meta_version = FeMetaVersion.VERSION_48; + public static int meta_version = FeMetaVersion.VERSION_49; } diff --git a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java index 651750480434ae..1e23b1febdad81 100644 --- a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -66,7 +66,7 @@ public final class FeMetaVersion { // persist LoadJob's execMemLimit public static final int VERSION_34 = 34; - // update the BE in cluster, because of forgeting + // update the BE in cluster, because of forgetting // to remove backend in cluster when drop backend or // decommission in latest versions. public static final int VERSION_35 = 35; @@ -107,4 +107,6 @@ public final class FeMetaVersion { // replica schema hash public static final int VERSION_48 = 48; + // routine load job + public static final int VERSION_49 = 49; } diff --git a/fe/src/main/java/org/apache/doris/common/LoadException.java b/fe/src/main/java/org/apache/doris/common/LoadException.java index 7e269302bb5d2c..759a2684488b4b 100644 --- a/fe/src/main/java/org/apache/doris/common/LoadException.java +++ b/fe/src/main/java/org/apache/doris/common/LoadException.java @@ -20,7 +20,10 @@ /** * Exception for load */ -public class LoadException extends Exception { +public class LoadException extends UserException { + + private static final long serialVersionUID = 1L; + public LoadException(String msg) { super(msg); } diff --git a/fe/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java index 6d0ace0e2a0e3c..a9b487b0a994be 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java @@ -51,7 +51,8 @@ public class BackendsProcDir implements ProcDirInterface { .add("BackendId").add("Cluster").add("IP").add("HostName").add("HeartbeatPort") .add("BePort").add("HttpPort").add("BrpcPort").add("LastStartTime").add("LastHeartbeat").add("Alive") .add("SystemDecommissioned").add("ClusterDecommissioned").add("TabletNum") - .add("DataUsedCapacity").add("AvailCapacity").add("TotalCapacity").add("UsedPct").add("ErrMsg") + .add("DataUsedCapacity").add("AvailCapacity").add("TotalCapacity").add("UsedPct") + .add("MaxDiskUsedPct").add("ErrMsg") .build(); public static final int IP_INDEX = 2; @@ -165,6 +166,7 @@ public static List> getClusterBackendInfos(String clusterName) { used = (double) (totalB - availB) * 100 / totalB; } backendInfo.add(String.format("%.2f", used) + " %"); + backendInfo.add(String.format("%.2f", backend.getMaxDiskUsedPct() * 100) + " %"); backendInfo.add(backend.getHeartbeatErrMsg()); diff --git a/fe/src/main/java/org/apache/doris/common/proc/ProcService.java b/fe/src/main/java/org/apache/doris/common/proc/ProcService.java index 0197a2d7e5c53e..4b61725db46731 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/ProcService.java +++ b/fe/src/main/java/org/apache/doris/common/proc/ProcService.java @@ -43,11 +43,12 @@ private ProcService() { root.register("frontends", new FrontendsProcNode(Catalog.getInstance())); root.register("brokers", Catalog.getInstance().getBrokerMgr().getProcNode()); root.register("load_error_hub", new LoadErrorHubProcNode(Catalog.getInstance())); - root.register("transactions", new TransDbProcDir(Catalog.getInstance())); + root.register("transactions", new TransDbProcDir()); root.register("monitor", new MonitorProcDir()); root.register("current_queries", new CurrentQueryStatisticsProcDir()); root.register("current_backend_instances", new CurrentQueryBackendInstanceProcDir()); root.register("cluster_balance", new ClusterBalanceProcDir()); + root.register("routine_loads", new RoutineLoadsProcDir()); } // 通过指定的路径获得对应的PROC Node diff --git a/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadProcNode.java b/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadProcNode.java new file mode 100644 index 00000000000000..3af6f1594b22a6 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadProcNode.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.common.proc; + +import org.apache.doris.analysis.ShowRoutineLoadTaskStmt; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.load.routineload.RoutineLoadManager; + +/* + SHOW RPOC "/routine_loads/{jobName}/{jobId}" + show routine load task info belong to job + + RESULT: + show result is sames as show routine load task + */ +public class RoutineLoadProcNode implements ProcNodeInterface { + + private final long jobId; + + public RoutineLoadProcNode(long jobId) { + this.jobId = jobId; + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + // check job id + RoutineLoadManager routineLoadManager = Catalog.getCurrentCatalog().getRoutineLoadManager(); + RoutineLoadJob routineLoadJob = routineLoadManager.getJob(jobId); + if (routineLoadJob == null) { + throw new AnalysisException("Job[" + jobId + "] does not exist"); + } + + BaseProcResult result = new BaseProcResult(); + result.setNames(ShowRoutineLoadTaskStmt.getTitleNames()); + result.setRows(routineLoadJob.getTasksShowInfo()); + return result; + } +} diff --git a/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadsNameProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadsNameProcDir.java new file mode 100644 index 00000000000000..06a7b981906cf9 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadsNameProcDir.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.common.proc; + +import com.google.common.base.Preconditions; +import org.apache.doris.analysis.ShowRoutineLoadStmt; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.load.routineload.RoutineLoadManager; + +import java.util.List; + +/* + SHOW PROC "/routine_loads/{jobName}" + show all of routine load named job name in all of db + + RESULT + show result is sames as show routine load {jobName} + */ +public class RoutineLoadsNameProcDir implements ProcDirInterface { + + private final String jobName; + + public RoutineLoadsNameProcDir(String jobName) { + this.jobName = jobName; + } + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String jobIdStr) throws AnalysisException { + Preconditions.checkNotNull(jobIdStr); + + long jobId; + try { + jobId = Long.valueOf(jobIdStr); + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid job id format: " + jobIdStr); + } + + return new RoutineLoadProcNode(jobId); + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + BaseProcResult baseProcResult = new BaseProcResult(); + baseProcResult.setNames(ShowRoutineLoadStmt.getTitleNames()); + + // find all of job named routine load + RoutineLoadManager routineLoadManager = Catalog.getCurrentCatalog().getRoutineLoadManager(); + List routineLoadJobList = routineLoadManager.getJobByName(jobName); + for (RoutineLoadJob routineLoadJob : routineLoadJobList) { + baseProcResult.addRow(routineLoadJob.getShowInfo()); + } + + return baseProcResult; + } +} diff --git a/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadsProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadsProcDir.java new file mode 100644 index 00000000000000..214519f500b677 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/proc/RoutineLoadsProcDir.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.common.proc; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.load.routineload.RoutineLoadManager; + +import java.util.List; + +/* + SHOW PROC "/routine_loads" + show statistic of all of running loads + + RESULT: + Name | Id | DbName | JobStatistic | TaskStatistic + */ +public class RoutineLoadsProcDir implements ProcDirInterface { + + private static final ImmutableList TITLE_NAMES = + new ImmutableList.Builder() + .add("Name") + .add("Id") + .add("DbName") + .add("Statistic") + .add("TaskStatistic") + .build(); + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String jobName) throws AnalysisException { + if (Strings.isNullOrEmpty(jobName)) { + throw new IllegalArgumentException("job name could not be empty of null"); + } + return new RoutineLoadsNameProcDir(jobName); + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + BaseProcResult baseProcResult = new BaseProcResult(); + baseProcResult.setNames(TITLE_NAMES); + RoutineLoadManager routineLoadManager = Catalog.getCurrentCatalog().getRoutineLoadManager(); + try { + List routineLoadJobList = routineLoadManager.getJob(null, null, true); + for (RoutineLoadJob routineLoadJob : routineLoadJobList) { + baseProcResult.addRow(routineLoadJob.getShowStatistic()); + } + } catch (MetaNotFoundException e) { + throw new AnalysisException("failed to get all of routine load job"); + } + return baseProcResult; + } +} diff --git a/fe/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java index 06a5c7a15a4f4c..75d22727237eda 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/TransDbProcDir.java @@ -18,11 +18,10 @@ package org.apache.doris.common.proc; import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; -import org.apache.doris.transaction.GlobalTransactionMgr; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.ListComparator; -import com.google.common.base.Preconditions; +import org.apache.doris.transaction.GlobalTransactionMgr; + import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -40,18 +39,15 @@ public class TransDbProcDir implements ProcDirInterface { .add("DbId") .add("DbName") .build(); - private Catalog catalog; - public TransDbProcDir(Catalog catalog) { - this.catalog = catalog; + public TransDbProcDir() { } @Override public ProcResult fetchResult() throws AnalysisException { - Preconditions.checkNotNull(catalog); BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); - GlobalTransactionMgr transactionMgr = catalog.getCurrentGlobalTransactionMgr(); + GlobalTransactionMgr transactionMgr = Catalog.getCurrentGlobalTransactionMgr(); List> infos = transactionMgr.getDbInfo(); // order by dbId, asc ListComparator> comparator = new ListComparator>(0); @@ -74,7 +70,7 @@ public boolean register(String name, ProcNodeInterface node) { @Override public ProcNodeInterface lookup(String dbIdStr) throws AnalysisException { - if (catalog == null || Strings.isNullOrEmpty(dbIdStr)) { + if (Strings.isNullOrEmpty(dbIdStr)) { throw new AnalysisException("Db id is null"); } long dbId = -1L; @@ -84,11 +80,6 @@ public ProcNodeInterface lookup(String dbIdStr) throws AnalysisException { throw new AnalysisException("Invalid db id format: " + dbIdStr); } - Database db = catalog.getDb(dbId); - if (db == null) { - throw new AnalysisException("Database[" + dbId + "] does not exist."); - } - - return new TransProcDir(dbId); + return new TransStateProcDir(dbId); } } diff --git a/fe/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java b/fe/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java index 1db9124f98baeb..e9a0f4f9829533 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java +++ b/fe/src/main/java/org/apache/doris/common/proc/TransPartitionProcNode.java @@ -18,12 +18,9 @@ package org.apache.doris.common.proc; import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.Table; -import org.apache.doris.transaction.GlobalTransactionMgr; import org.apache.doris.common.AnalysisException; -import com.google.common.base.Preconditions; +import org.apache.doris.transaction.GlobalTransactionMgr; + import com.google.common.collect.ImmutableList; import java.util.ArrayList; @@ -37,30 +34,22 @@ public class TransPartitionProcNode implements ProcNodeInterface { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("PartitionId") - .add("PartitionName") .add("CommittedVersion") .add("CommittedVersionHash") - .add("State") .build(); private long tid; - private Database db; - private OlapTable olapTable; + private long tableId; - public TransPartitionProcNode(long tid, Database db, OlapTable olapTable) { + public TransPartitionProcNode(long tid, long tableId) { this.tid = tid; - this.db = db; - this.olapTable = olapTable; + this.tableId = tableId; } @Override public ProcResult fetchResult() throws AnalysisException { - Preconditions.checkNotNull(db); - Preconditions.checkNotNull(olapTable); - Preconditions.checkState(olapTable.getType() == Table.TableType.OLAP); - Catalog catalog = Catalog.getInstance(); - GlobalTransactionMgr transactionMgr = catalog.getCurrentGlobalTransactionMgr(); - List> partitionInfos = transactionMgr.getPartitionTransInfo(tid, db, olapTable); + GlobalTransactionMgr transactionMgr = Catalog.getCurrentGlobalTransactionMgr(); + List> partitionInfos = transactionMgr.getPartitionTransInfo(tid, tableId); // set result BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); diff --git a/fe/src/main/java/org/apache/doris/common/proc/TransProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/TransProcDir.java index f18edd03fe8a76..2f9e6649467b5d 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/TransProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/TransProcDir.java @@ -18,10 +18,10 @@ package org.apache.doris.common.proc; import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; -import org.apache.doris.transaction.GlobalTransactionMgr; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.ListComparator; +import org.apache.doris.transaction.GlobalTransactionMgr; + import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -46,20 +46,26 @@ public class TransProcDir implements ProcDirInterface { .add("FinishTime") .add("Reason") .add("ErrorReplicasCount") + .add("ListenerId") + .add("TimeoutMs") .build(); + + public static final int MAX_SHOW_ENTRIES = 2000; + private long dbId; + private String state; - public TransProcDir(long dbId) { + public TransProcDir(long dbId, String state) { this.dbId = dbId; + this.state = state; } @Override public ProcResult fetchResult() throws AnalysisException { BaseProcResult result = new BaseProcResult(); result.setNames(TITLE_NAMES); - Catalog catalog = Catalog.getInstance(); GlobalTransactionMgr transactionMgr = Catalog.getCurrentGlobalTransactionMgr(); - List> infos = transactionMgr.getDbTransInfo(dbId); + List> infos = transactionMgr.getDbTransInfo(dbId, state.equals("running"), MAX_SHOW_ENTRIES); // order by transactionId, asc ListComparator> comparator = new ListComparator>(0); Collections.sort(infos, comparator); @@ -90,11 +96,6 @@ public ProcNodeInterface lookup(String tidStr) throws AnalysisException { } catch (NumberFormatException e) { throw new AnalysisException("Invalid transaction id format: " + tid); } - Database db = Catalog.getInstance().getDb(dbId); - if (db == null) { - throw new AnalysisException("Database[" + dbId + "] does not exist."); - } - - return new TransTablesProcDir(db, tid); + return new TransTablesProcDir(tid); } } diff --git a/fe/src/main/java/org/apache/doris/common/proc/TransStateProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/TransStateProcDir.java new file mode 100644 index 00000000000000..b1d5c6dada42e9 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/proc/TransStateProcDir.java @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.common.proc; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.transaction.GlobalTransactionMgr; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; + +/** + * author: chenmingyu + * date: 19/4/25 10:43 + * project: doris + */ +public class TransStateProcDir implements ProcDirInterface { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("State").add("Number") + .build(); + + private long dbId; + + public TransStateProcDir(Long dbId) { + this.dbId = dbId; + } + + @Override + public ProcResult fetchResult() throws AnalysisException { + BaseProcResult result = new BaseProcResult(); + result.setNames(TITLE_NAMES); + GlobalTransactionMgr transactionMgr = Catalog.getCurrentGlobalTransactionMgr(); + result.setRows(transactionMgr.getDbTransStateInfo(dbId)); + return result; + } + + @Override + public boolean register(String name, ProcNodeInterface node) { + return false; + } + + @Override + public ProcNodeInterface lookup(String state) throws AnalysisException { + if (Strings.isNullOrEmpty(state)) { + throw new AnalysisException("State is not set"); + } + + if (!state.equals("running") && !state.equals("finished")) { + throw new AnalysisException("State is invalid"); + } + + return new TransProcDir(dbId, state); + } +} diff --git a/fe/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java b/fe/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java index 265f63b8dcea52..0696366179baa1 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java +++ b/fe/src/main/java/org/apache/doris/common/proc/TransTablesProcDir.java @@ -18,13 +18,10 @@ package org.apache.doris.common.proc; import org.apache.doris.catalog.Catalog; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.Table; -import org.apache.doris.transaction.GlobalTransactionMgr; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.ListComparator; -import com.google.common.base.Preconditions; +import org.apache.doris.transaction.GlobalTransactionMgr; + import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -40,60 +37,25 @@ public class TransTablesProcDir implements ProcDirInterface { public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() .add("TableId") - .add("TableName") - .add("PartitionNum") - .add("State") + .add("CommittedPartitionIds") .build(); - private Database db; private long tid; - public TransTablesProcDir(Database db, long tid) { - this.db = db; + public TransTablesProcDir(long tid) { this.tid = tid; } - @Override public boolean register(String name, ProcNodeInterface node) { return false; } - @Override - public ProcNodeInterface lookup(String tableIdStr) throws AnalysisException { - Preconditions.checkNotNull(db); - if (Strings.isNullOrEmpty(tableIdStr)) { - throw new AnalysisException("TableIdStr is null"); - } - - long tableId = -1L; - try { - tableId = Long.valueOf(tableIdStr); - } catch (NumberFormatException e) { - throw new AnalysisException("Invalid table id format: " + tableIdStr); - } - - Table table = null; - db.readLock(); - try { - table = db.getTable(tableId); - } finally { - db.readUnlock(); - } - if (table == null) { - throw new AnalysisException("Table[" + tableId + "] does not exist"); - } - - return new TransPartitionProcNode(tid, db, (OlapTable) table); - } - @Override public ProcResult fetchResult() throws AnalysisException { - Preconditions.checkNotNull(db); - // get info GlobalTransactionMgr transactionMgr = Catalog.getCurrentGlobalTransactionMgr(); - List> tableInfos = transactionMgr.getTableTransInfo(tid, db); + List> tableInfos = transactionMgr.getTableTransInfo(tid); // sort by table id ListComparator> comparator = new ListComparator>(0); Collections.sort(tableInfos, comparator); @@ -112,4 +74,20 @@ public ProcResult fetchResult() throws AnalysisException { return result; } + + @Override + public ProcNodeInterface lookup(String tableIdStr) throws AnalysisException { + if (Strings.isNullOrEmpty(tableIdStr)) { + throw new AnalysisException("TableIdStr is null"); + } + + long tableId = -1L; + try { + tableId = Long.valueOf(tableIdStr); + } catch (NumberFormatException e) { + throw new AnalysisException("Invalid table id format: " + tableIdStr); + } + + return new TransPartitionProcNode(tid, tableId); + } } diff --git a/fe/src/main/java/org/apache/doris/common/util/Daemon.java b/fe/src/main/java/org/apache/doris/common/util/Daemon.java index d1419650dc51f3..6840b11bd471f0 100644 --- a/fe/src/main/java/org/apache/doris/common/util/Daemon.java +++ b/fe/src/main/java/org/apache/doris/common/util/Daemon.java @@ -106,8 +106,8 @@ public void run() { while (!isStop.get()) { try { runOneCycle(); - } catch (Exception e) { - LOG.error("exception: ", e); + } catch (Throwable e) { + LOG.error("daemon thread got exception: ", e); } try { diff --git a/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java b/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java index 5e239fcfdf2bd8..783c06d88b3068 100644 --- a/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java +++ b/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java @@ -24,6 +24,7 @@ import java.io.PrintWriter; import java.io.StringWriter; import java.text.DecimalFormat; +import java.util.UUID; public class DebugUtil { public static final DecimalFormat DECIMAL_FORMAT_SCALE_3 = new DecimalFormat("#.000"); @@ -120,13 +121,20 @@ public static Pair getByteUint(long value) { public static String printId(final TUniqueId id) { StringBuilder builder = new StringBuilder(); - builder.append(Long.toHexString(id.hi)).append(":").append(Long.toHexString(id.lo)); + builder.append(Long.toHexString(id.hi)).append("-").append(Long.toHexString(id.lo)); + return builder.toString(); + } + + public static String printId(final UUID id) { + TUniqueId tUniqueId = new TUniqueId(id.getMostSignificantBits(), id.getLeastSignificantBits()); + StringBuilder builder = new StringBuilder(); + builder.append(Long.toHexString(tUniqueId.hi)).append("-").append(Long.toHexString(tUniqueId.lo)); return builder.toString(); } public static String printId(final PUniqueId id) { StringBuilder builder = new StringBuilder(); - builder.append(Long.toHexString(id.hi)).append(":").append(Long.toHexString(id.lo)); + builder.append(Long.toHexString(id.hi)).append("-").append(Long.toHexString(id.lo)); return builder.toString(); } diff --git a/fe/src/main/java/org/apache/doris/common/util/LogBuilder.java b/fe/src/main/java/org/apache/doris/common/util/LogBuilder.java new file mode 100644 index 00000000000000..434fa002df8b40 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/util/LogBuilder.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.common.util; + +import com.google.common.collect.Lists; +import org.apache.doris.thrift.TUniqueId; + +import java.util.Iterator; +import java.util.List; +import java.util.UUID; + +public class LogBuilder { + + private final StringBuffer stringBuffer; + private final List entries; + + public LogBuilder(String identifier) { + stringBuffer = new StringBuffer(identifier).append("-"); + entries = Lists.newLinkedList(); + } + + public LogBuilder(LogKey key, Long identifier) { + stringBuffer = new StringBuffer().append(key.name()).append("=").append(identifier).append(", "); + entries = Lists.newLinkedList(); + } + + public LogBuilder(LogKey key, UUID identifier) { + TUniqueId tUniqueId = new TUniqueId(identifier.getMostSignificantBits(), identifier.getLeastSignificantBits()); + stringBuffer = new StringBuffer().append(key.name()).append("=").append(DebugUtil.printId(tUniqueId)).append(", "); + entries = Lists.newLinkedList(); + } + + public LogBuilder(LogKey key, String identifier) { + stringBuffer = new StringBuffer().append(key.name()).append("=").append(identifier).append(", "); + entries = Lists.newLinkedList(); + } + + + public LogBuilder add(String key, long value) { + entries.add(new LogEntry(key, String.valueOf(value))); + return this; + } + + public LogBuilder add(String key, int value) { + entries.add(new LogEntry(key, String.valueOf(value))); + return this; + } + + public LogBuilder add(String key, float value) { + entries.add(new LogEntry(key, String.valueOf(value))); + return this; + } + + public LogBuilder add(String key, boolean value) { + entries.add(new LogEntry(key, String.valueOf(value))); + return this; + } + + public LogBuilder add(String key, String value) { + entries.add(new LogEntry(key, String.valueOf(value))); + return this; + } + + public LogBuilder add(String key, Object value) { + if (value == null) { + entries.add(new LogEntry(key, "null")); + } else { + entries.add(new LogEntry(key, value.toString())); + } + return this; + } + + public String build() { + Iterator it = entries.iterator(); + while (it.hasNext()) { + LogEntry logEntry = it.next(); + stringBuffer.append(logEntry.key).append("={").append(logEntry.value).append("}"); + if (it.hasNext()) { + stringBuffer.append(", "); + } + } + return stringBuffer.toString(); + } + + private class LogEntry { + String key; + String value; + + public LogEntry(String key, String value) { + this.key = key; + this.value = value; + } + } + + @Override + public String toString() { + return build(); + } +} diff --git a/fe/src/main/java/org/apache/doris/common/util/LogKey.java b/fe/src/main/java/org/apache/doris/common/util/LogKey.java new file mode 100644 index 00000000000000..a474b79885a770 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/util/LogKey.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.common.util; + +public enum LogKey{ + ROUTINE_LOAD_JOB, + ROUINTE_LOAD_TASK +} diff --git a/fe/src/main/java/org/apache/doris/common/util/Util.java b/fe/src/main/java/org/apache/doris/common/util/Util.java index a857b1cbe7249d..ed2f85f1e49600 100644 --- a/fe/src/main/java/org/apache/doris/common/util/Util.java +++ b/fe/src/main/java/org/apache/doris/common/util/Util.java @@ -19,7 +19,9 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.common.AnalysisException; +import com.google.common.base.Strings; import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; @@ -40,6 +42,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.function.Predicate; import java.util.zip.Adler32; public class Util { @@ -364,5 +367,29 @@ public static String getResultForUrl(String urlStr, String encodedAuthInfo, int LOG.debug("get result from url {}: {}", urlStr, sb.toString()); return sb.toString(); } + + public static long getLongPropertyOrDefault(String valStr, long defaultVal, Predicate pred, + String hintMsg) throws AnalysisException { + if (Strings.isNullOrEmpty(valStr)) { + return defaultVal; + } + + long result = defaultVal; + try { + result = Long.valueOf(valStr); + } catch (NumberFormatException e) { + throw new AnalysisException(hintMsg); + } + + if (pred == null) { + return result; + } + + if (!pred.test(result)) { + throw new AnalysisException(hintMsg); + } + + return result; + } } diff --git a/fe/src/main/java/org/apache/doris/external/EsIndexState.java b/fe/src/main/java/org/apache/doris/external/EsIndexState.java index bcb692511ae9ef..4a1201e70dbc33 100644 --- a/fe/src/main/java/org/apache/doris/external/EsIndexState.java +++ b/fe/src/main/java/org/apache/doris/external/EsIndexState.java @@ -19,6 +19,7 @@ import java.util.List; import java.util.Map; +import java.util.Random; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -55,6 +56,27 @@ public EsIndexState(String indexName) { this.partitionDesc = null; this.partitionKey = null; } + + + public void addHttpAddress(Map nodesInfo) { + for (Map.Entry> entry : shardRoutings.entrySet()) { + List shardRoutings = entry.getValue(); + for (EsShardRouting shardRouting : shardRoutings) { + String nodeId = shardRouting.getNodeId(); + if (nodesInfo.containsKey(nodeId)) { + shardRouting.setHttpAddress(nodesInfo.get(nodeId).getPublishAddress()); + } else { + shardRouting.setHttpAddress(randomAddress(nodesInfo)); + } + } + } + } + + public TNetworkAddress randomAddress(Map nodesInfo) { + int seed = new Random().nextInt() % nodesInfo.size(); + EsNodeInfo[] nodeInfos = (EsNodeInfo[]) nodesInfo.values().toArray(); + return nodeInfos[seed].getPublishAddress(); + } public static EsIndexState parseIndexStateV55(String indexName, JSONObject indicesRoutingMap, JSONObject nodesMap, diff --git a/fe/src/main/java/org/apache/doris/external/EsMajorVersion.java b/fe/src/main/java/org/apache/doris/external/EsMajorVersion.java new file mode 100644 index 00000000000000..b71db8ae0fd6b4 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/external/EsMajorVersion.java @@ -0,0 +1,103 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.external; + + +/** + * Elasticsearch major version information, useful to check client's query compatibility with the Rest API. + * + * reference es-hadoop: + * + */ +public class EsMajorVersion { + public static final EsMajorVersion V_5_X = new EsMajorVersion((byte) 5, "5.x"); + public static final EsMajorVersion V_6_X = new EsMajorVersion((byte) 6, "6.x"); + public static final EsMajorVersion V_7_X = new EsMajorVersion((byte) 7, "7.x"); + public static final EsMajorVersion LATEST = V_7_X; + + public final byte major; + private final String version; + + private EsMajorVersion(byte major, String version) { + this.major = major; + this.version = version; + } + + public boolean after(EsMajorVersion version) { + return version.major < major; + } + + public boolean on(EsMajorVersion version) { + return version.major == major; + } + + public boolean notOn(EsMajorVersion version) { + return !on(version); + } + + public boolean onOrAfter(EsMajorVersion version) { + return version.major <= major; + } + + public boolean before(EsMajorVersion version) { + return version.major > major; + } + + public boolean onOrBefore(EsMajorVersion version) { + return version.major >= major; + } + + public static EsMajorVersion parse(String version) throws Exception { + if (version.startsWith("5.")) { + return new EsMajorVersion((byte) 5, version); + } + if (version.startsWith("6.")) { + return new EsMajorVersion((byte) 6, version); + } + if (version.startsWith("7.")) { + return new EsMajorVersion((byte) 7, version); + } + throw new Exception("Unsupported/Unknown Elasticsearch version [" + version + "]." + + "Highest supported version is [" + LATEST.version + "]. You may need to upgrade ES-Hadoop."); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + EsMajorVersion version = (EsMajorVersion) o; + + return major == version.major && + version.equals(version.version); + } + + @Override + public int hashCode() { + return major; + } + + @Override + public String toString() { + return version; + } +} diff --git a/fe/src/main/java/org/apache/doris/external/EsNodeInfo.java b/fe/src/main/java/org/apache/doris/external/EsNodeInfo.java new file mode 100644 index 00000000000000..61b513bbfacd9e --- /dev/null +++ b/fe/src/main/java/org/apache/doris/external/EsNodeInfo.java @@ -0,0 +1,205 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.external; + +import org.apache.doris.thrift.TNetworkAddress; + +import java.util.List; +import java.util.Map; + +/** + * This class represents one node with the http and potential thrift publish address + */ +public class EsNodeInfo { + private final String id; + private final String name; + private final String host; + private final String ip; + private TNetworkAddress publishAddress; + private final boolean hasHttp; + private final boolean isClient; + private final boolean isData; + private final boolean isIngest; + private boolean hasThrift; + private TNetworkAddress thriftAddress; + + public EsNodeInfo(String id, Map map) throws Exception { + this.id = id; + EsMajorVersion version = EsMajorVersion.parse((String) map.get("version")); + this.name = (String) map.get("name"); + this.host = (String) map.get("host"); + this.ip = (String) map.get("ip"); + if (version.before(EsMajorVersion.V_5_X)) { + Map attributes = (Map) map.get("attributes"); + if (attributes == null) { + this.isClient = false; + this.isData = true; + } else { + String data = (String) attributes.get("data"); + this.isClient = data == null ? true : !Boolean.parseBoolean(data); + this.isData = data == null ? true : Boolean.parseBoolean(data); + } + this.isIngest = false; + } else { + List roles = (List) map.get("roles"); + this.isClient = roles.contains("data") == false; + this.isData = roles.contains("data"); + this.isIngest = roles.contains("ingest"); + } + Map httpMap = (Map) map.get("http"); + if (httpMap != null) { + String address = (String) httpMap.get("publish_address"); + if (address != null) { + String[] scratch = address.split(":"); + this.publishAddress = new TNetworkAddress(scratch[0], Integer.valueOf(scratch[1])); + this.hasHttp = true; + } else { + this.publishAddress = null; + this.hasHttp = false; + } + } else { + this.publishAddress = null; + this.hasHttp = false; + } + + Map attributesMap = (Map) map.get("attributes"); + if (attributesMap != null) { + String thriftPortStr = (String) attributesMap.get("thrift_port"); + if (thriftPortStr != null) { + try { + int thriftPort = Integer.valueOf(thriftPortStr); + hasThrift = true; + thriftAddress = new TNetworkAddress(this.ip, thriftPort); + } catch (Exception e) { + hasThrift = false; + } + } else { + hasThrift = false; + } + } else { + hasThrift = false; + } + } + + public boolean hasHttp() { + return hasHttp; + } + + public boolean isClient() { + return isClient; + } + + public boolean isData() { + return isData; + } + + public boolean isIngest() { + return isIngest; + } + + public String getId() { + return id; + } + + public String getName() { + return name; + } + + public String getHost() { + return host; + } + + public TNetworkAddress getPublishAddress() { + return publishAddress; + } + + public boolean isHasThrift() { + return hasThrift; + } + + public TNetworkAddress getThriftAddress() { + return thriftAddress; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + EsNodeInfo nodeInfo = (EsNodeInfo) o; + + if (hasHttp != nodeInfo.hasHttp) { + return false; + } + if (isClient != nodeInfo.isClient) { + return false; + } + if (isData != nodeInfo.isData) { + return false; + } + if (!id.equals(nodeInfo.id)) { + return false; + } + if (!name.equals(nodeInfo.name)) { + return false; + } + if (!host.equals(nodeInfo.host)) { + return false; + } + if (!ip.equals(nodeInfo.ip)) { + return false; + } + if (hasThrift != nodeInfo.hasThrift) { + return false; + } + return (publishAddress != null ? publishAddress.equals(nodeInfo.publishAddress) : nodeInfo.publishAddress == null) + && (thriftAddress != null ? thriftAddress.equals(nodeInfo.thriftAddress) : nodeInfo.thriftAddress == null); + } + + @Override + public int hashCode() { + int result = id.hashCode(); + result = 31 * result + name.hashCode(); + result = 31 * result + host.hashCode(); + result = 31 * result + ip.hashCode(); + result = 31 * result + (publishAddress != null ? publishAddress.hashCode() : 0); + result = 31 * result + (thriftAddress != null ? thriftAddress.hashCode() : 0); + result = 31 * result + (hasHttp ? 1 : 0); + result = 31 * result + (hasThrift ? 1 : 0); + result = 31 * result + (isClient ? 1 : 0); + result = 31 * result + (isData ? 1 : 0); + return result; + } + + @Override + public String toString() { + return "EsNodeInfo{" + + "id='" + id + '\'' + + ", name='" + name + '\'' + + ", host='" + host + '\'' + + ", ip='" + ip + '\'' + + ", publishAddress=" + publishAddress + + ", hasHttp=" + hasHttp + + ", isClient=" + isClient + + ", isData=" + isData + + ", isIngest=" + isIngest + + ", hasThrift=" + hasThrift + + ", thriftAddress=" + thriftAddress + + '}'; + } +} diff --git a/fe/src/main/java/org/apache/doris/external/EsRestClient.java b/fe/src/main/java/org/apache/doris/external/EsRestClient.java new file mode 100644 index 00000000000000..1159cadc646f30 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/external/EsRestClient.java @@ -0,0 +1,141 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.external; + +import okhttp3.Credentials; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.util.Strings; +import org.codehaus.jackson.JsonParser; +import org.codehaus.jackson.map.DeserializationConfig; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.SerializationConfig; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class EsRestClient { + private static final Logger LOG = LogManager.getLogger(EsRestClient.class); + private ObjectMapper mapper; + + { + mapper = new ObjectMapper(); + mapper.configure(DeserializationConfig.Feature.USE_ANNOTATIONS, false); + mapper.configure(SerializationConfig.Feature.USE_ANNOTATIONS, false); + } + + private static OkHttpClient networkClient = new OkHttpClient.Builder() + .readTimeout(10, TimeUnit.SECONDS) + .build(); + + private String basicAuth; + + private int nextClient = 0; + private String[] nodes; + private String currentNode; + + public EsRestClient(String[] nodes, String authUser, String authPassword) { + this.nodes = nodes; + if (!Strings.isEmpty(authUser) && !Strings.isEmpty(authPassword)) { + basicAuth = Credentials.basic(authUser, authPassword); + } + selectNextNode(); + } + + private boolean selectNextNode() { + if (nextClient >= nodes.length) { + return false; + } + currentNode = nodes[nextClient++]; + return true; + } + + public Map getHttpNodes() throws Exception { + Map> nodesData = get("_nodes/http", "nodes"); + if (nodesData == null) { + return Collections.emptyMap(); + } + Map nodes = new HashMap<>(); + for (Map.Entry> entry : nodesData.entrySet()) { + EsNodeInfo node = new EsNodeInfo(entry.getKey(), entry.getValue()); + if (node.hasHttp()) { + nodes.put(node.getId(), node); + } + } + return nodes; + } + + public String getIndexMetaData(String indexName) { + String path = "_cluster/state?indices=" + indexName + + "&metric=routing_table,nodes,metadata&expand_wildcards=open"; + return execute(path); + + } + + /** + * execute request for specific path + * @param path the path must not leading with '/' + * @return + */ + private String execute(String path) { + selectNextNode(); + boolean nextNode; + do { + Request request = new Request.Builder() + .get() + .addHeader("Authorization", basicAuth) + .url(currentNode + "/" + path) + .build(); + try { + Response response = networkClient.newCall(request).execute(); + if (response.isSuccessful()) { + return response.body().string(); + } + } catch (IOException e) { + LOG.warn("request node [{}] [{}] failures {}, try next nodes", currentNode, path, e); + } + nextNode = selectNextNode(); + if (!nextNode) { + LOG.error("try all nodes [{}],no other nodes left", nodes); + } + } while (nextNode); + return null; + } + + public T get(String q, String key) { + return parseContent(execute(q), key); + } + + private T parseContent(String response, String key) { + Map map = Collections.emptyMap(); + try { + JsonParser jsonParser = mapper.getJsonFactory().createJsonParser(response); + map = mapper.readValue(jsonParser, Map.class); + } catch (IOException ex) { + LOG.error("parse es response failure: [{}]", response); + } + return (T) (key != null ? map.get(key) : map); + } + +} diff --git a/fe/src/main/java/org/apache/doris/external/EsShardRouting.java b/fe/src/main/java/org/apache/doris/external/EsShardRouting.java index 721edc9a26bae7..5f9e090a607583 100644 --- a/fe/src/main/java/org/apache/doris/external/EsShardRouting.java +++ b/fe/src/main/java/org/apache/doris/external/EsShardRouting.java @@ -28,12 +28,16 @@ public class EsShardRouting { private final int shardId; private final boolean isPrimary; private final TNetworkAddress address; + + private TNetworkAddress httpAddress; + private final String nodeId; - public EsShardRouting(String indexName, int shardId, boolean isPrimary, TNetworkAddress address) { + public EsShardRouting(String indexName, int shardId, boolean isPrimary, TNetworkAddress address, String nodeId) { this.indexName = indexName; this.shardId = shardId; this.isPrimary = isPrimary; this.address = address; + this.nodeId = nodeId; } public static EsShardRouting parseShardRoutingV55(String indexName, String shardKey, @@ -45,8 +49,8 @@ public static EsShardRouting parseShardRoutingV55(String indexName, String shard String thriftPort = nodeInfo.getJSONObject("attributes").getString("thrift_port"); TNetworkAddress addr = new TNetworkAddress(transportAddr[0], Integer.valueOf(thriftPort)); boolean isPrimary = shardInfo.getBoolean("primary"); - return new EsShardRouting(indexName, Integer.valueOf(shardKey), - isPrimary, addr); + return new EsShardRouting(indexName, Integer.valueOf(shardKey), + isPrimary, addr, nodeId); } public int getShardId() { @@ -64,4 +68,28 @@ public TNetworkAddress getAddress() { public String getIndexName() { return indexName; } + + public TNetworkAddress getHttpAddress() { + return httpAddress; + } + + public void setHttpAddress(TNetworkAddress httpAddress) { + this.httpAddress = httpAddress; + } + + public String getNodeId() { + return nodeId; + } + + @Override + public String toString() { + return "EsShardRouting{" + + "indexName='" + indexName + '\'' + + ", shardId=" + shardId + + ", isPrimary=" + isPrimary + + ", address=" + address + + ", httpAddress=" + httpAddress + + ", nodeId='" + nodeId + '\'' + + '}'; + } } diff --git a/fe/src/main/java/org/apache/doris/external/EsStateStore.java b/fe/src/main/java/org/apache/doris/external/EsStateStore.java index 3a4822ce219dbf..8bab6180fc6bce 100644 --- a/fe/src/main/java/org/apache/doris/external/EsStateStore.java +++ b/fe/src/main/java/org/apache/doris/external/EsStateStore.java @@ -86,10 +86,22 @@ public void deRegisterTable(long tableId) { protected void runOneCycle() { for (EsTable esTable : esTables.values()) { try { - EsTableState esTableState = loadEsIndexMetadataV55(esTable); - if (esTableState != null) { - esTable.setEsTableState(esTableState); + EsRestClient client = new EsRestClient(esTable.getSeeds(), + esTable.getUserName(), esTable.getPasswd()); +// EsTableState esTableState = loadEsIndexMetadataV55(esTable); + String indexMetaData = client.getIndexMetaData(esTable.getIndexName()); + if (indexMetaData == null) { + continue; } + EsTableState esTableState = parseClusterState55(indexMetaData, esTable); + if (esTableState == null) { + continue; + } + if (EsTable.TRANSPORT_HTTP.equals(esTable.getTransport())) { + Map nodesInfo = client.getHttpNodes(); + esTableState.addHttpAddress(nodesInfo); + } + esTable.setEsTableState(esTableState); } catch (Throwable e) { LOG.error("errors while load table {} state from es", esTable.getName()); } diff --git a/fe/src/main/java/org/apache/doris/external/EsTableState.java b/fe/src/main/java/org/apache/doris/external/EsTableState.java index a7620cfd04a9cb..59b69aa2678c2b 100644 --- a/fe/src/main/java/org/apache/doris/external/EsTableState.java +++ b/fe/src/main/java/org/apache/doris/external/EsTableState.java @@ -18,9 +18,11 @@ package org.apache.doris.external; import java.util.Map; +import java.util.Random; import org.apache.doris.catalog.PartitionInfo; import com.google.common.collect.Maps; +import org.apache.doris.thrift.TNetworkAddress; /** * save the dynamic info parsed from es cluster state such as shard routing, partition info @@ -38,6 +40,22 @@ public EsTableState() { partitionedIndexStates = Maps.newHashMap(); unPartitionedIndexStates = Maps.newHashMap(); } + + public void addHttpAddress(Map nodesInfo) { + for (EsIndexState indexState : partitionedIndexStates.values()) { + indexState.addHttpAddress(nodesInfo); + } + for (EsIndexState indexState : unPartitionedIndexStates.values()) { + indexState.addHttpAddress(nodesInfo); + } + + } + + public TNetworkAddress randomAddress(Map nodesInfo) { + int seed = new Random().nextInt() % nodesInfo.size(); + EsNodeInfo[] nodeInfos = (EsNodeInfo[]) nodesInfo.values().toArray(); + return nodeInfos[seed].getPublishAddress(); + } public PartitionInfo getPartitionInfo() { return partitionInfo; diff --git a/fe/src/main/java/org/apache/doris/http/BaseAction.java b/fe/src/main/java/org/apache/doris/http/BaseAction.java index 0f072b6a5eb36f..9ff453a4c361a5 100644 --- a/fe/src/main/java/org/apache/doris/http/BaseAction.java +++ b/fe/src/main/java/org/apache/doris/http/BaseAction.java @@ -47,18 +47,20 @@ import io.netty.channel.ChannelProgressiveFutureListener; import io.netty.channel.DefaultFileRegion; import io.netty.handler.codec.base64.Base64; -import io.netty.handler.codec.http.Cookie; import io.netty.handler.codec.http.DefaultFullHttpResponse; import io.netty.handler.codec.http.DefaultHttpResponse; import io.netty.handler.codec.http.FullHttpResponse; import io.netty.handler.codec.http.HttpChunkedInput; -import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpHeaderValues; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; import io.netty.handler.codec.http.LastHttpContent; -import io.netty.handler.codec.http.ServerCookieEncoder; +import io.netty.handler.codec.http.cookie.Cookie; +import io.netty.handler.codec.http.cookie.ServerCookieEncoder; import io.netty.handler.ssl.SslHandler; import io.netty.handler.stream.ChunkedFile; import io.netty.util.CharsetUtil; @@ -93,7 +95,7 @@ public void handleRequest(BaseRequest request) throws Exception { } catch (Exception e) { LOG.warn("fail to process url: {}", request.getRequest().uri(), e); if (e instanceof UnauthorizedException) { - response.updateHeader(HttpHeaders.Names.WWW_AUTHENTICATE, "Basic realm=\"\""); + response.updateHeader(HttpHeaderNames.WWW_AUTHENTICATE.toString(), "Basic realm=\"\""); writeResponse(request, response, HttpResponseStatus.UNAUTHORIZED); } else { writeResponse(request, response, HttpResponseStatus.NOT_FOUND); @@ -123,17 +125,17 @@ protected void writeResponse(BaseRequest request, BaseResponse response, HttpRes checkDefaultContentTypeHeader(response, responseObj); if (!method.equals(HttpMethod.HEAD)) { - response.updateHeader(HttpHeaders.Names.CONTENT_LENGTH, + response.updateHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(responseObj.content().readableBytes())); } writeCustomHeaders(response, responseObj); writeCookies(response, responseObj); - boolean keepAlive = HttpHeaders.isKeepAlive(request.getRequest()); + boolean keepAlive = HttpUtil.isKeepAlive(request.getRequest()); if (!keepAlive) { request.getContext().write(responseObj).addListener(ChannelFutureListener.CLOSE); } else { - responseObj.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); + responseObj.headers().set(HttpHeaderNames.CONNECTION.toString(), HttpHeaderValues.KEEP_ALIVE.toString()); request.getContext().write(responseObj); } } @@ -142,8 +144,8 @@ protected void writeFileResponse(BaseRequest request, BaseResponse response, Htt File resFile) { HttpResponse responseObj = new DefaultHttpResponse(HttpVersion.HTTP_1_1, status); - if (HttpHeaders.isKeepAlive(request.getRequest())) { - response.updateHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); + if (HttpUtil.isKeepAlive(request.getRequest())) { + response.updateHeader(HttpHeaderNames.CONNECTION.toString(), HttpHeaderValues.KEEP_ALIVE.toString()); } ChannelFuture sendFileFuture; @@ -155,7 +157,7 @@ protected void writeFileResponse(BaseRequest request, BaseResponse response, Htt rafFile = new RandomAccessFile(resFile, "r"); long fileLength = 0; fileLength = rafFile.length(); - response.updateHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(fileLength)); + response.updateHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(fileLength)); writeCookies(response, responseObj); writeCustomHeaders(response, responseObj); @@ -205,7 +207,7 @@ public void operationComplete(ChannelProgressiveFuture future) { }); // Decide whether to close the connection or not. - boolean keepAlive = HttpHeaders.isKeepAlive(request.getRequest()); + boolean keepAlive = HttpUtil.isKeepAlive(request.getRequest()); if (!keepAlive) { // Close the connection when the whole content is written out. lastContentFuture.addListener(ChannelFutureListener.CLOSE); @@ -215,9 +217,9 @@ public void operationComplete(ChannelProgressiveFuture future) { // Set 'CONTENT_TYPE' header if it havn't been set. protected void checkDefaultContentTypeHeader(BaseResponse response, Object responseOj) { if (!Strings.isNullOrEmpty(response.getContentType())) { - response.updateHeader(HttpHeaders.Names.CONTENT_TYPE, response.getContentType()); + response.updateHeader(HttpHeaderNames.CONTENT_TYPE.toString(), response.getContentType()); } else { - response.updateHeader(HttpHeaders.Names.CONTENT_TYPE, "text/html"); + response.updateHeader(HttpHeaderNames.CONTENT_TYPE.toString(), "text/html"); } } @@ -229,7 +231,7 @@ protected void writeCustomHeaders(BaseResponse response, HttpResponse responseOb protected void writeCookies(BaseResponse response, HttpResponse responseObj) { for (Cookie cookie : response.getCookies()) { - responseObj.headers().add(HttpHeaders.Names.SET_COOKIE, ServerCookieEncoder.encode(cookie)); + responseObj.headers().add(HttpHeaderNames.SET_COOKIE.toString(), ServerCookieEncoder.LAX.encode(cookie)); } } diff --git a/fe/src/main/java/org/apache/doris/http/BaseResponse.java b/fe/src/main/java/org/apache/doris/http/BaseResponse.java index 97b643ea91a9c8..297c8c7f8c9474 100644 --- a/fe/src/main/java/org/apache/doris/http/BaseResponse.java +++ b/fe/src/main/java/org/apache/doris/http/BaseResponse.java @@ -25,7 +25,7 @@ import java.util.Map; import java.util.Set; -import io.netty.handler.codec.http.Cookie; +import io.netty.handler.codec.http.cookie.Cookie; public class BaseResponse { private String contentType; diff --git a/fe/src/main/java/org/apache/doris/http/action/SystemAction.java b/fe/src/main/java/org/apache/doris/http/action/SystemAction.java index 8f120d77d42e34..4966d4fa52377e 100644 --- a/fe/src/main/java/org/apache/doris/http/action/SystemAction.java +++ b/fe/src/main/java/org/apache/doris/http/action/SystemAction.java @@ -17,6 +17,8 @@ package org.apache.doris.http.action; +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Catalog; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.proc.ProcDirInterface; import org.apache.doris.common.proc.ProcNodeInterface; @@ -25,10 +27,17 @@ import org.apache.doris.http.BaseRequest; import org.apache.doris.http.BaseResponse; import org.apache.doris.http.IllegalArgException; +import org.apache.doris.mysql.privilege.PaloAuth; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.MasterOpExecutor; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.system.SystemInfoService; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import java.util.List; +import java.util.stream.Collectors; import io.netty.handler.codec.http.HttpMethod; @@ -47,50 +56,73 @@ public void executeGet(BaseRequest request, BaseResponse response) { getPageHeader(request, response.getContent()); String currentPath = request.getSingleParameter("path"); - ProcNodeInterface node = null; - // root path is default path if (Strings.isNullOrEmpty(currentPath)) { currentPath = "/"; } - - node = getProcNode(currentPath); - appendSystemInfo(response.getContent(), node, currentPath); + appendSystemInfo(response.getContent(), currentPath, currentPath); getPageFooter(response.getContent()); writeResponse(request, response); } - private void appendSystemInfo(StringBuilder buffer, ProcNodeInterface procNode, String path) { + private void appendSystemInfo(StringBuilder buffer, String procPath, String path) { buffer.append("

System Info

"); buffer.append("

This page lists the system info, like /proc in Linux.

"); buffer.append("

Current path: " + path + "

"); + ProcNodeInterface procNode = getProcNode(procPath); if (procNode == null) { - buffer.append("

No such proc path[" - + path - + "]

"); + buffer.append("

No such proc path[" + path + "]

"); return; } + boolean isDir = (procNode instanceof ProcDirInterface); + + List columnNames = null; + List> rows = null; + if (!Catalog.getCurrentCatalog().isMaster()) { + // forward to master + String showProcStmt = "SHOW PROC \"" + procPath + "\""; + ConnectContext context = new ConnectContext(null); + context.setCatalog(Catalog.getCurrentCatalog()); + context.setCluster(SystemInfoService.DEFAULT_CLUSTER); + context.setQualifiedUser(PaloAuth.ADMIN_USER); + MasterOpExecutor masterOpExecutor = new MasterOpExecutor(showProcStmt, context, + RedirectStatus.FORWARD_NO_SYNC); + try { + masterOpExecutor.execute(); + } catch (Exception e) { + buffer.append("

Failed to forward request to master

"); + return; + } - boolean isDir = false; - if (procNode instanceof ProcDirInterface) { - isDir = true; - } + ShowResultSet resultSet = masterOpExecutor.getProxyResultSet(); + if (resultSet == null) { + buffer.append("

Failed to get result from master

"); + return; + } - ProcResult result; - try { - result = procNode.fetchResult(); - } catch (AnalysisException e) { - buffer.append("

The result is null, " - + "maybe haven't be implemented completely[" + e.getMessage() + "], please check.

"); - buffer.append("

" - + "INFO: ProcNode type is [" + procNode.getClass().getName() - + "]

"); - return; + columnNames = resultSet.getMetaData().getColumns().stream().map(c -> c.getName()).collect( + Collectors.toList()); + rows = resultSet.getResultRows(); + } else { + ProcResult result; + try { + result = procNode.fetchResult(); + } catch (AnalysisException e) { + buffer.append("

The result is null, " + + "maybe haven't be implemented completely[" + e.getMessage() + "], please check.

"); + buffer.append("

" + + "INFO: ProcNode type is [" + procNode.getClass().getName() + + "]

"); + return; + } + + columnNames = result.getColumnNames(); + rows = result.getRows(); } - List columnNames = result.getColumnNames(); - List> rows = result.getRows(); + Preconditions.checkNotNull(columnNames); + Preconditions.checkNotNull(rows); appendBackButton(buffer, path); appendTableHeader(buffer, columnNames); diff --git a/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java b/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java index 5e7cefb7b59a5e..709a88a8db288a 100644 --- a/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java +++ b/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java @@ -17,21 +17,22 @@ package org.apache.doris.http.rest; -import java.io.File; -import java.util.Set; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - import org.apache.doris.common.Config; import org.apache.doris.http.ActionController; import org.apache.doris.http.BaseRequest; import org.apache.doris.http.BaseResponse; import org.apache.doris.http.IllegalArgException; + import com.google.common.base.Strings; import com.google.common.collect.Sets; -import io.netty.handler.codec.http.HttpHeaders; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.File; +import java.util.Set; + +import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpResponseStatus; @@ -82,7 +83,7 @@ public void execute(BaseRequest request, BaseResponse response) { writeFileResponse(request, response, HttpResponseStatus.OK, logFile); } else if (method.equals(HttpMethod.HEAD)) { long fileLength = logFile.length(); - response.updateHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(fileLength)); + response.updateHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(fileLength)); writeResponse(request, response, HttpResponseStatus.OK); } else { response.appendContent(new RestBaseResult("HTTP method is not allowed.").toJson()); diff --git a/fe/src/main/java/org/apache/doris/http/rest/ShowProcAction.java b/fe/src/main/java/org/apache/doris/http/rest/ShowProcAction.java index e15d93bb193b3f..00690e953cb336 100644 --- a/fe/src/main/java/org/apache/doris/http/rest/ShowProcAction.java +++ b/fe/src/main/java/org/apache/doris/http/rest/ShowProcAction.java @@ -17,6 +17,8 @@ package org.apache.doris.http.rest; +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Catalog; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.proc.ProcNodeInterface; import org.apache.doris.common.proc.ProcResult; @@ -27,6 +29,10 @@ import org.apache.doris.http.IllegalArgException; import org.apache.doris.http.UnauthorizedException; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.MasterOpExecutor; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.system.SystemInfoService; import com.google.common.base.Strings; import com.google.gson.Gson; @@ -54,8 +60,9 @@ public static void registerAction(ActionController controller) throws IllegalArg @Override public void execute(BaseRequest request, BaseResponse response) { // check authority + AuthorizationInfo authInfo; try { - AuthorizationInfo authInfo = getAuthorizationInfo(request); + authInfo = getAuthorizationInfo(request); checkGlobalAuth(authInfo, PrivPredicate.ADMIN); } catch (UnauthorizedException e) { response.appendContent("Authentication Failed. " + e.getMessage()); @@ -64,33 +71,73 @@ public void execute(BaseRequest request, BaseResponse response) { } String path = request.getSingleParameter("path"); - ProcNodeInterface procNode = null; - ProcService instance = ProcService.getInstance(); - try { - if (Strings.isNullOrEmpty(path)) { - procNode = instance.open("/"); - } else { - procNode = instance.open(path); - } - } catch (AnalysisException e) { - LOG.warn(e.getMessage()); - response.getContent().append("[]"); + String forward = request.getSingleParameter("forward"); + boolean isForward = false; + if (!Strings.isNullOrEmpty(forward) && forward.equals("true")) { + isForward = true; } - if (procNode != null) { - ProcResult result; + // forward to master if necessary + if (!Catalog.getCurrentCatalog().isMaster() && isForward) { + String showProcStmt = "SHOW PROC \"" + path + "\""; + ConnectContext context = new ConnectContext(null); + context.setCatalog(Catalog.getCurrentCatalog()); + context.setCluster(SystemInfoService.DEFAULT_CLUSTER); + context.setQualifiedUser(authInfo.fullUserName); + context.setRemoteIP(authInfo.remoteIp); + MasterOpExecutor masterOpExecutor = new MasterOpExecutor(showProcStmt, context, + RedirectStatus.FORWARD_NO_SYNC); + LOG.debug("need to transfer to Master. stmt: {}", context.getStmtId()); + try { - result = procNode.fetchResult(); - List> rows = result.getRows(); + masterOpExecutor.execute(); + } catch (Exception e) { + response.appendContent("Failed to forward stmt: " + e.getMessage()); + sendResult(request, response); + return; + } + + ShowResultSet resultSet = masterOpExecutor.getProxyResultSet(); + if (resultSet == null) { + response.appendContent("Failed to get result set"); + sendResult(request, response); + return; + } - Gson gson = new Gson(); - response.setContentType("application/json"); - response.getContent().append(gson.toJson(rows)); + Gson gson = new Gson(); + response.setContentType("application/json"); + response.getContent().append(gson.toJson(resultSet.getResultRows())); + + } else { + ProcNodeInterface procNode = null; + ProcService instance = ProcService.getInstance(); + try { + if (Strings.isNullOrEmpty(path)) { + procNode = instance.open("/"); + } else { + procNode = instance.open(path); + } } catch (AnalysisException e) { LOG.warn(e.getMessage()); response.getContent().append("[]"); } + + if (procNode != null) { + ProcResult result; + try { + result = procNode.fetchResult(); + List> rows = result.getRows(); + + Gson gson = new Gson(); + response.setContentType("application/json"); + response.getContent().append(gson.toJson(rows)); + } catch (AnalysisException e) { + LOG.warn(e.getMessage()); + response.getContent().append("[]"); + } + } } + sendResult(request, response); } } diff --git a/fe/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/src/main/java/org/apache/doris/journal/JournalEntity.java index 64f9f1de6776c7..e5f06e9df060a4 100644 --- a/fe/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -39,6 +39,7 @@ import org.apache.doris.load.ExportJob; import org.apache.doris.load.LoadErrorHub; import org.apache.doris.load.LoadJob; +import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.master.Checkpoint; import org.apache.doris.mysql.privilege.UserProperty; import org.apache.doris.mysql.privilege.UserPropertyInfo; @@ -60,6 +61,7 @@ import org.apache.doris.persist.PrivInfo; import org.apache.doris.persist.RecoverInfo; import org.apache.doris.persist.ReplicaPersistInfo; +import org.apache.doris.persist.RoutineLoadOperation; import org.apache.doris.persist.TableInfo; import org.apache.doris.persist.TablePropertyInfo; import org.apache.doris.persist.TruncateTableInfo; @@ -411,6 +413,17 @@ public void readFields(DataInput in) throws IOException { needRead = false; break; } + case OperationType.OP_CREATE_ROUTINE_LOAD_JOB: { + data = RoutineLoadJob.read(in); + needRead = false; + break; + } + case OperationType.OP_CHANGE_ROUTINE_LOAD_JOB: + case OperationType.OP_REMOVE_ROUTINE_LOAD_JOB: { + data = RoutineLoadOperation.read(in); + needRead = false; + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); diff --git a/fe/src/main/java/org/apache/doris/load/Load.java b/fe/src/main/java/org/apache/doris/load/Load.java index 27cf9bc76bd460..91aac372d55168 100644 --- a/fe/src/main/java/org/apache/doris/load/Load.java +++ b/fe/src/main/java/org/apache/doris/load/Load.java @@ -72,6 +72,7 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.persist.ReplicaPersistInfo; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.service.FrontendOptions; import org.apache.doris.system.Backend; import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentClient; @@ -3136,7 +3137,8 @@ public void delete(DeleteStmt stmt) throws DdlException { loadDeleteJob.setIdToTabletLoadInfo(idToTabletLoadInfo); loadDeleteJob.setState(JobState.LOADING); long transactionId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction(db.getId(), jobLabel, - "fe", LoadJobSourceType.FRONTEND); + "FE: " + FrontendOptions.getLocalHostAddress(), LoadJobSourceType.FRONTEND, + Config.stream_load_default_timeout_second); loadDeleteJob.setTransactionId(transactionId); // the delete job will be persist in editLog addLoadJob(loadDeleteJob, db); diff --git a/fe/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/src/main/java/org/apache/doris/load/LoadChecker.java index 9ac727effdd50f..5809c685715925 100644 --- a/fe/src/main/java/org/apache/doris/load/LoadChecker.java +++ b/fe/src/main/java/org/apache/doris/load/LoadChecker.java @@ -28,7 +28,7 @@ import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.Config; -import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.Daemon; import org.apache.doris.load.AsyncDeleteJob.DeleteState; import org.apache.doris.load.FailMsg.CancelType; @@ -52,7 +52,6 @@ import org.apache.doris.thrift.TTaskType; import org.apache.doris.transaction.GlobalTransactionMgr; import org.apache.doris.transaction.TabletCommitInfo; -import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TransactionStatus; @@ -330,7 +329,7 @@ private void tryCommitJob(LoadJob job, Database db) { tabletCommitInfos.add(new TabletCommitInfo(tabletId, replica.getBackendId())); } globalTransactionMgr.commitTransaction(job.getDbId(), job.getTransactionId(), tabletCommitInfos); - } catch (MetaNotFoundException | TransactionException e) { + } catch (UserException e) { LOG.warn("errors while commit transaction [{}], cancel the job {}, reason is {}", transactionState.getTransactionId(), job, e); load.cancelLoadJob(job, CancelType.UNKNOWN, transactionState.getReason()); diff --git a/fe/src/main/java/org/apache/doris/load/RoutineLoadDesc.java b/fe/src/main/java/org/apache/doris/load/RoutineLoadDesc.java index c0b8e8b8b19261..5ca44fb4c9f822 100644 --- a/fe/src/main/java/org/apache/doris/load/RoutineLoadDesc.java +++ b/fe/src/main/java/org/apache/doris/load/RoutineLoadDesc.java @@ -15,23 +15,23 @@ // specific language governing permissions and limitations // under the License. - package org.apache.doris.load; import org.apache.doris.analysis.ColumnSeparator; -import org.apache.doris.analysis.Expr; -import org.apache.doris.analysis.LoadColumnsInfo; +import org.apache.doris.analysis.ImportColumnsStmt; +import org.apache.doris.analysis.ImportWhereStmt; import java.util.List; public class RoutineLoadDesc { private final ColumnSeparator columnSeparator; - private final LoadColumnsInfo columnsInfo; - private final Expr wherePredicate; + private final ImportColumnsStmt columnsInfo; + private final ImportWhereStmt wherePredicate; + // nullable private final List partitionNames; - public RoutineLoadDesc(ColumnSeparator columnSeparator, LoadColumnsInfo columnsInfo, - Expr wherePredicate, List partitionNames) { + public RoutineLoadDesc(ColumnSeparator columnSeparator, ImportColumnsStmt columnsInfo, + ImportWhereStmt wherePredicate, List partitionNames) { this.columnSeparator = columnSeparator; this.columnsInfo = columnsInfo; this.wherePredicate = wherePredicate; @@ -42,14 +42,15 @@ public ColumnSeparator getColumnSeparator() { return columnSeparator; } - public LoadColumnsInfo getColumnsInfo() { + public ImportColumnsStmt getColumnsInfo() { return columnsInfo; } - public Expr getWherePredicate() { + public ImportWhereStmt getWherePredicate() { return wherePredicate; } + // nullable public List getPartitionNames() { return partitionNames; } diff --git a/fe/src/main/java/org/apache/doris/load/TxnStateChangeListener.java b/fe/src/main/java/org/apache/doris/load/TxnStateChangeListener.java deleted file mode 100644 index d394c51897316d..00000000000000 --- a/fe/src/main/java/org/apache/doris/load/TxnStateChangeListener.java +++ /dev/null @@ -1,50 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.load; - -import org.apache.doris.transaction.AbortTransactionException; -import org.apache.doris.transaction.TransactionState; - -public interface TxnStateChangeListener { - - /** - * update catalog of job which has related txn after transaction has been committed - * - * @param txnState - */ - void onCommitted(TransactionState txnState); - - /** - * this interface is executed before txn aborted, you can check if txn could be abort in this stage - * - * @param txnState - * @param txnStatusChangeReason maybe null - * @throws AbortTransactionException if transaction could not be abort or there are some exception before aborted, - * it will throw this exception - */ - void beforeAborted(TransactionState txnState, TransactionState.TxnStatusChangeReason txnStatusChangeReason) - throws AbortTransactionException; - - /** - * this interface is executed when transaction has been aborted - * - * @param txnState - * @param txnStatusChangeReason maybe null - */ - void onAborted(TransactionState txnState, TransactionState.TxnStatusChangeReason txnStatusChangeReason); -} diff --git a/fe/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java b/fe/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java index 9ac12e06ad3e63..8901f25869b535 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java @@ -17,53 +17,123 @@ package org.apache.doris.load.routineload; +import org.apache.doris.common.Pair; +import org.apache.doris.thrift.TKafkaRLTaskProgress; + import com.google.common.base.Joiner; import com.google.common.collect.Maps; -import org.apache.doris.common.io.Writable; -import org.apache.doris.thrift.TKafkaRLTaskProgress; +import com.google.gson.Gson; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; /** * this is description of kafka routine load progress - * the data before offset was already loaded in doris + * the data before offset was already loaded in Doris */ // {"partitionIdToOffset": {}} public class KafkaProgress extends RoutineLoadProgress { + public static final String OFFSET_BEGINNING = "OFFSET_BEGINNING"; // -2 + public static final String OFFSET_END = "OFFSET_END"; // -1 + // OFFSET_ZERO is just for show info, if user specified offset is 0 + public static final String OFFSET_ZERO = "OFFSET_ZERO"; + + public static final long OFFSET_BEGINNING_VAL = -2; + public static final long OFFSET_END_VAL = -1; - private Map partitionIdToOffset; + // (partition id, begin offset) + // the offset the next msg to be consumed + private Map partitionIdToOffset = Maps.newConcurrentMap(); public KafkaProgress() { - partitionIdToOffset = Maps.newHashMap(); + super(LoadDataSourceType.KAFKA); } public KafkaProgress(TKafkaRLTaskProgress tKafkaRLTaskProgress) { - this.partitionIdToOffset = tKafkaRLTaskProgress.getPartitionIdToOffset(); + super(LoadDataSourceType.KAFKA); + this.partitionIdToOffset = tKafkaRLTaskProgress.getPartitionCmtOffset(); + } + + public Map getPartitionIdToOffset(List partitionIds) { + Map result = Maps.newHashMap(); + for (Map.Entry entry : partitionIdToOffset.entrySet()) { + for (Integer partitionId : partitionIds) { + if (entry.getKey().equals(partitionId)) { + result.put(partitionId, entry.getValue()); + } + } + } + return result; } - public Map getPartitionIdToOffset() { - return partitionIdToOffset; + public void addPartitionOffset(Pair partitionOffset) { + partitionIdToOffset.put(partitionOffset.first, partitionOffset.second); } - public void setPartitionIdToOffset(Map partitionIdToOffset) { - this.partitionIdToOffset = partitionIdToOffset; + public Long getOffsetByPartition(int kafkaPartition) { + return partitionIdToOffset.get(kafkaPartition); + } + + public boolean containsPartition(Integer kafkaPartition) { + return partitionIdToOffset.containsKey(kafkaPartition); + } + + public boolean hasPartition() { + return !partitionIdToOffset.isEmpty(); + } + + // (partition id, end offset) + // OFFSET_ZERO: user set offset == 0, no committed msg + // OFFSET_END: user set offset = OFFSET_END, no committed msg + // OFFSET_BEGINNING: user set offset = OFFSET_BEGINNING, no committed msg + // other: current committed msg's offset + private void getReadableProgress(Map showPartitionIdToOffset) { + for (Map.Entry entry : partitionIdToOffset.entrySet()) { + if (entry.getValue() == 0) { + showPartitionIdToOffset.put(entry.getKey(), OFFSET_ZERO); + } else if (entry.getValue() == -1) { + showPartitionIdToOffset.put(entry.getKey(), OFFSET_END); + } else if (entry.getValue() == -2) { + showPartitionIdToOffset.put(entry.getKey(), OFFSET_BEGINNING); + } else { + showPartitionIdToOffset.put(entry.getKey(), "" + (entry.getValue() - 1)); + } + } + } + + @Override + public String toString() { + Map showPartitionIdToOffset = Maps.newHashMap(); + getReadableProgress(showPartitionIdToOffset); + return "KafkaProgress [partitionIdToOffset=" + + Joiner.on("|").withKeyValueSeparator("_").join(showPartitionIdToOffset) + "]"; + } + + @Override + public String toJsonString() { + Map showPartitionIdToOffset = Maps.newHashMap(); + getReadableProgress(showPartitionIdToOffset); + Gson gson = new Gson(); + return gson.toJson(showPartitionIdToOffset); } @Override public void update(RoutineLoadProgress progress) { KafkaProgress newProgress = (KafkaProgress) progress; - newProgress.getPartitionIdToOffset().entrySet().parallelStream() - .forEach(entity -> partitionIdToOffset.put(entity.getKey(), entity.getValue())); + // + 1 to point to the next msg offset to be consumed + newProgress.partitionIdToOffset.entrySet().stream() + .forEach(entity -> this.partitionIdToOffset.put(entity.getKey(), entity.getValue() + 1)); } @Override public void write(DataOutput out) throws IOException { + super.write(out); out.writeInt(partitionIdToOffset.size()); - for (Map.Entry entry : partitionIdToOffset.entrySet()) { + for (Map.Entry entry : partitionIdToOffset.entrySet()) { out.writeInt((Integer) entry.getKey()); out.writeLong((Long) entry.getValue()); } @@ -71,16 +141,11 @@ public void write(DataOutput out) throws IOException { @Override public void readFields(DataInput in) throws IOException { + super.readFields(in); int size = in.readInt(); partitionIdToOffset = new HashMap<>(); for (int i = 0; i < size; i++) { partitionIdToOffset.put(in.readInt(), in.readLong()); } } - - @Override - public String toString() { - return "KafkaProgress [partitionIdToOffset=" - + Joiner.on("|").withKeyValueSeparator("_").join(partitionIdToOffset) + "]"; - } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java index 2714b63a7521f5..4c172d1c3f1383 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java @@ -20,26 +20,38 @@ import org.apache.doris.analysis.CreateRoutineLoadStmt; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.Table; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.Config; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; -import org.apache.doris.load.RoutineLoadDesc; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; import org.apache.doris.system.SystemInfoService; -import org.apache.doris.transaction.BeginTransactionException; -import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.UUID; @@ -50,241 +62,337 @@ public class KafkaRoutineLoadJob extends RoutineLoadJob { private static final Logger LOG = LogManager.getLogger(KafkaRoutineLoadJob.class); - private static final String FE_GROUP_ID = "fe_fetch_partitions"; - private static final int FETCH_PARTITIONS_TIMEOUT = 10; + private static final int FETCH_PARTITIONS_TIMEOUT_SECOND = 5; - private String serverAddress; + private String brokerList; private String topic; // optional, user want to load partitions. - private List customKafkaPartitions; + private List customKafkaPartitions = Lists.newArrayList(); // current kafka partitions is the actually partition which will be fetched - private List currentKafkaPartitions; + private List currentKafkaPartitions = Lists.newArrayList(); // this is the kafka consumer which is used to fetch the number of partitions - private KafkaConsumer consumer; + private KafkaConsumer consumer; - public KafkaRoutineLoadJob(String name, long dbId, long tableId, String serverAddress, String topic) { - super(name, dbId, tableId, LoadDataSourceType.KAFKA); - this.serverAddress = serverAddress; + public KafkaRoutineLoadJob() { + // for serialization, id is dummy + super(-1, LoadDataSourceType.KAFKA); + } + + public KafkaRoutineLoadJob(Long id, String name, String clusterName, long dbId, long tableId, String brokerList, + String topic) { + super(id, name, clusterName, dbId, tableId, LoadDataSourceType.KAFKA); + this.brokerList = brokerList; this.topic = topic; this.progress = new KafkaProgress(); - this.customKafkaPartitions = new ArrayList<>(); - this.currentKafkaPartitions = new ArrayList<>(); setConsumer(); } - // TODO(ml): I will change it after ut. - @VisibleForTesting - public KafkaRoutineLoadJob(String id, String name, long dbId, long tableId, - RoutineLoadDesc routineLoadDesc, - int desireTaskConcurrentNum, int maxErrorNum, - String serverAddress, String topic, KafkaProgress kafkaProgress) { - super(id, name, dbId, tableId, routineLoadDesc, - desireTaskConcurrentNum, LoadDataSourceType.KAFKA, - maxErrorNum); - this.serverAddress = serverAddress; - this.topic = topic; - this.progress = kafkaProgress; - this.customKafkaPartitions = new ArrayList<>(); - this.currentKafkaPartitions = new ArrayList<>(); - setConsumer(); + public String getTopic() { + return topic; } - private void setCustomKafkaPartitions(List kafkaPartitions) throws LoadException { - writeLock(); - try { - if (this.customKafkaPartitions.size() != 0) { - throw new LoadException("Kafka partitions have been initialized"); - } - // check if custom kafka partition is valid - List allKafkaPartitions = getAllKafkaPartitions(); - outter: - for (Integer customkafkaPartition : kafkaPartitions) { - for (Integer kafkaPartition : allKafkaPartitions) { - if (kafkaPartition.equals(customkafkaPartition)) { - continue outter; - } - } - throw new LoadException("there is a custom kafka partition " + customkafkaPartition - + " which is invalid for topic " + topic); - } - this.customKafkaPartitions = kafkaPartitions; - } finally { - writeUnlock(); - } + public String getBrokerList() { + return brokerList; } @Override - public List divideRoutineLoadJob(int currentConcurrentTaskNum) { + public void divideRoutineLoadJob(int currentConcurrentTaskNum) throws UserException { List result = new ArrayList<>(); writeLock(); try { if (state == JobState.NEED_SCHEDULE) { // divide kafkaPartitions into tasks for (int i = 0; i < currentConcurrentTaskNum; i++) { - try { - KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(UUID.randomUUID().toString(), id); - routineLoadTaskInfoList.add(kafkaTaskInfo); - needScheduleTaskInfoList.add(kafkaTaskInfo); - result.add(kafkaTaskInfo); - } catch (UserException e) { - LOG.error("failed to begin txn for kafka routine load task, change job state to failed"); - state = JobState.CANCELLED; - // TODO(ml): edit log - break; + Map taskKafkaProgress = Maps.newHashMap(); + for (int j = 0; j < currentKafkaPartitions.size(); j++) { + if (j % currentConcurrentTaskNum == i) { + int kafkaPartition = currentKafkaPartitions.get(j); + taskKafkaProgress.put(kafkaPartition, + ((KafkaProgress) progress).getOffsetByPartition(kafkaPartition)); + } } + KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), id, clusterName, taskKafkaProgress); + routineLoadTaskInfoList.add(kafkaTaskInfo); + result.add(kafkaTaskInfo); } + // change job state to running if (result.size() != 0) { - for (int i = 0; i < currentKafkaPartitions.size(); i++) { - ((KafkaTaskInfo) routineLoadTaskInfoList.get(i % currentConcurrentTaskNum)) - .addKafkaPartition(currentKafkaPartitions.get(i)); - } - // change job state to running - // TODO(ml): edit log - state = JobState.RUNNING; + unprotectUpdateState(JobState.RUNNING, null, false); } } else { LOG.debug("Ignore to divide routine load job while job state {}", state); } + // save task into queue of needScheduleTasks + Catalog.getCurrentCatalog().getRoutineLoadTaskScheduler().addTasksInQueue(result); } finally { writeUnlock(); } - return result; } @Override public int calculateCurrentConcurrentTaskNum() throws MetaNotFoundException { - updateCurrentKafkaPartitions(); SystemInfoService systemInfoService = Catalog.getCurrentSystemInfo(); - Database db = Catalog.getCurrentCatalog().getDb(dbId); - if (db == null) { - LOG.warn("db {} is not exists from job {}", dbId, id); - throw new MetaNotFoundException("db " + dbId + " is not exists from job " + id); - } - int aliveBeNum = systemInfoService.getBackendIds(true).size(); + int aliveBeNum = systemInfoService.getClusterBackendIds(clusterName, true).size(); int partitionNum = currentKafkaPartitions.size(); if (desireTaskConcurrentNum == 0) { - desireTaskConcurrentNum = partitionNum; + desireTaskConcurrentNum = Config.max_routine_load_task_concurrent_num; + } + + LOG.info("current concurrent task number is min" + + "(partition num: {}, desire task concurrent num: {}, alive be num: {}, config: {})", + partitionNum, desireTaskConcurrentNum, aliveBeNum, Config.max_routine_load_task_concurrent_num); + currentTaskConcurrentNum = + Math.min(Math.min(partitionNum, Math.min(desireTaskConcurrentNum, aliveBeNum)), + Config.max_routine_load_task_concurrent_num); + return currentTaskConcurrentNum; + } + + // partitionIdToOffset must be not empty when loaded rows > 0 + // situation1: be commit txn but fe throw error when committing txn, + // fe rollback txn without partitionIdToOffset by itself + // this task should not be commit + // otherwise currentErrorNum and currentTotalNum is updated when progress is not updated + @Override + protected boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { + if (rlTaskTxnCommitAttachment.getLoadedRows() > 0 + && (!((KafkaProgress) rlTaskTxnCommitAttachment.getProgress()).hasPartition())) { + LOG.warn(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, DebugUtil.printId(rlTaskTxnCommitAttachment.getTaskId())) + .add("job_id", id) + .add("loaded_rows", rlTaskTxnCommitAttachment.getLoadedRows()) + .add("progress_partition_offset_size", 0) + .add("msg", "commit attachment info is incorrect")); + return false; } + return true; + } - LOG.info("current concurrent task number is min " - + "(current size of partition {}, desire task concurrent num {}, alive be num {})", - partitionNum, desireTaskConcurrentNum, aliveBeNum); - return Math.min(partitionNum, Math.min(desireTaskConcurrentNum, aliveBeNum)); + @Override + protected void updateProgress(RLTaskTxnCommitAttachment attachment) throws UserException { + super.updateProgress(attachment); + this.progress.update(attachment.getProgress()); } @Override - protected void updateProgress(RoutineLoadProgress progress) { - this.progress.update(progress); + protected void replayUpdateProgress(RLTaskTxnCommitAttachment attachment) { + super.replayUpdateProgress(attachment); + this.progress.update(attachment.getProgress()); } @Override - protected RoutineLoadTaskInfo reNewTask(RoutineLoadTaskInfo routineLoadTaskInfo) throws AnalysisException, - LabelAlreadyUsedException, BeginTransactionException { + protected RoutineLoadTaskInfo unprotectRenewTask(RoutineLoadTaskInfo routineLoadTaskInfo) { + KafkaTaskInfo oldKafkaTaskInfo = (KafkaTaskInfo) routineLoadTaskInfo; + // add new task + KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(oldKafkaTaskInfo, + ((KafkaProgress) progress).getPartitionIdToOffset(oldKafkaTaskInfo.getPartitions())); // remove old task routineLoadTaskInfoList.remove(routineLoadTaskInfo); // add new task - KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo((KafkaTaskInfo) routineLoadTaskInfo); routineLoadTaskInfoList.add(kafkaTaskInfo); - needScheduleTaskInfoList.add(kafkaTaskInfo); return kafkaTaskInfo; } + @Override + protected void unprotectUpdateProgress() { + updateNewPartitionProgress(); + } + // if customKafkaPartition is not null, then return false immediately // else if kafka partitions of topic has been changed, return true. // else return false + // update current kafka partition at the same time + // current kafka partitions = customKafkaPartitions == 0 ? all of partition of kafka topic : customKafkaPartitions @Override - protected boolean needReschedule() { - if (customKafkaPartitions != null && customKafkaPartitions.size() != 0) { - return false; - } else { - List newCurrentKafkaPartition = getAllKafkaPartitions(); - if (currentKafkaPartitions.containsAll(newCurrentKafkaPartition)) { - if (currentKafkaPartitions.size() > newCurrentKafkaPartition.size()) { - return true; - } else { + protected boolean unprotectNeedReschedule() throws UserException { + // only running and need_schedule job need to be changed current kafka partitions + if (this.state == JobState.RUNNING || this.state == JobState.NEED_SCHEDULE) { + if (customKafkaPartitions != null && customKafkaPartitions.size() != 0) { + currentKafkaPartitions = customKafkaPartitions; + return false; + } else { + List newCurrentKafkaPartition; + try { + newCurrentKafkaPartition = getAllKafkaPartitions(); + } catch (Exception e) { + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("error_msg", "Job failed to fetch all current partition with error " + e.getMessage()) + .build(), e); + if (this.state == JobState.NEED_SCHEDULE) { + unprotectUpdateState(JobState.PAUSED, + "Job failed to fetch all current partition with error " + e.getMessage(), + false /* not replay */); + } return false; } - } else { - return true; + if (currentKafkaPartitions.containsAll(newCurrentKafkaPartition)) { + if (currentKafkaPartitions.size() > newCurrentKafkaPartition.size()) { + currentKafkaPartitions = newCurrentKafkaPartition; + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("current_kafka_partitions", Joiner.on(",").join(currentKafkaPartitions)) + .add("msg", "current kafka partitions has been change") + .build()); + } + return true; + } else { + return false; + } + } else { + currentKafkaPartitions = newCurrentKafkaPartition; + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("current_kafka_partitions", Joiner.on(",").join(currentKafkaPartitions)) + .add("msg", "current kafka partitions has been change") + .build()); + } + return true; + } } - + } else { + return false; } } - private List getAllKafkaPartitions() { + @Override + protected String getStatistic() { + Map summary = Maps.newHashMap(); + summary.put("totalRows", Long.valueOf(totalRows)); + summary.put("loadedRows", Long.valueOf(totalRows - errorRows - unselectedRows)); + summary.put("errorRows", Long.valueOf(errorRows)); + summary.put("unselectedRows", Long.valueOf(unselectedRows)); + summary.put("receivedBytes", Long.valueOf(receivedBytes)); + summary.put("taskExecuteTimeMs", Long.valueOf(totalTaskExcutionTimeMs)); + summary.put("receivedBytesRate", Long.valueOf(receivedBytes / totalTaskExcutionTimeMs * 1000)); + summary.put("loadRowsRate", Long.valueOf((totalRows - errorRows - unselectedRows) / totalTaskExcutionTimeMs * 1000)); + summary.put("committedTaskNum", Long.valueOf(committedTaskNum)); + summary.put("abortedTaskNum", Long.valueOf(abortedTaskNum)); + Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + return gson.toJson(summary); + } + + private List getAllKafkaPartitions() throws LoadException { List result = new ArrayList<>(); - List partitionList = consumer.partitionsFor( - topic, Duration.ofSeconds(FETCH_PARTITIONS_TIMEOUT)); - for (PartitionInfo partitionInfo : partitionList) { - result.add(partitionInfo.partition()); + try { + List partitionList = consumer.partitionsFor(topic, + Duration.ofSeconds(FETCH_PARTITIONS_TIMEOUT_SECOND)); + for (PartitionInfo partitionInfo : partitionList) { + result.add(partitionInfo.partition()); + } + } catch (Exception e) { + throw new LoadException("failed to get partitions for topic: " + topic + ". " + e.getMessage()); } return result; } - public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) throws AnalysisException, - LoadException { - checkCreate(stmt); - // find dbId - Database database = Catalog.getCurrentCatalog().getDb(stmt.getDBTableName().getDb()); - Table table; - database.readLock(); + public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) throws UserException { + // check db and table + Database db = Catalog.getCurrentCatalog().getDb(stmt.getDBName()); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDBName()); + } + + long tableId = -1L; + db.readLock(); try { - table = database.getTable(stmt.getDBTableName().getTbl()); + unprotectedCheckMeta(db, stmt.getTableName(), stmt.getRoutineLoadDesc()); + tableId = db.getTable(stmt.getTableName()).getId(); } finally { - database.readUnlock(); + db.readUnlock(); } + // TODO(ml): check partition + // init kafka routine load job - KafkaRoutineLoadJob kafkaRoutineLoadJob = - new KafkaRoutineLoadJob(stmt.getName(), database.getId(), table.getId(), - stmt.getKafkaEndpoint(), - stmt.getKafkaTopic()); + long id = Catalog.getInstance().getNextId(); + KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(id, stmt.getName(), + db.getClusterName(), db.getId(), tableId, stmt.getKafkaBrokerList(), stmt.getKafkaTopic()); kafkaRoutineLoadJob.setOptional(stmt); return kafkaRoutineLoadJob; } - // current kafka partitions = customKafkaPartitions == 0 ? all of partition of kafka topic : customKafkaPartitions - private void updateCurrentKafkaPartitions() { - if (customKafkaPartitions == null || customKafkaPartitions.size() == 0) { - LOG.debug("All of partitions which belong to topic will be loaded for {} routine load job", name); - // fetch all of kafkaPartitions in topic - currentKafkaPartitions.addAll(getAllKafkaPartitions()); - } else { - currentKafkaPartitions = customKafkaPartitions; - } + private void updateNewPartitionProgress() { // update the progress of new partitions for (Integer kafkaPartition : currentKafkaPartitions) { - try { - ((KafkaProgress) progress).getPartitionIdToOffset().get(kafkaPartition); - } catch (NullPointerException e) { - ((KafkaProgress) progress).getPartitionIdToOffset().put(kafkaPartition, 0L); + if (!((KafkaProgress) progress).containsPartition(kafkaPartition)) { + // if offset is not assigned, start from OFFSET_END + ((KafkaProgress) progress).addPartitionOffset(Pair.create(kafkaPartition, KafkaProgress.OFFSET_END_VAL)); + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("kafka_partition_id", kafkaPartition) + .add("begin_offset", KafkaProgress.OFFSET_END) + .add("msg", "The new partition has been added in job")); + } } } } private void setConsumer() { Properties props = new Properties(); - props.put("bootstrap.servers", this.serverAddress); - props.put("group.id", FE_GROUP_ID); + props.put("bootstrap.servers", this.brokerList); + props.put("group.id", UUID.randomUUID().toString()); props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); consumer = new KafkaConsumer<>(props); } - private void setOptional(CreateRoutineLoadStmt stmt) throws LoadException { - if (stmt.getRoutineLoadDesc() != null) { - setRoutineLoadDesc(stmt.getRoutineLoadDesc()); + @Override + protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException { + super.setOptional(stmt); + + if (!stmt.getKafkaPartitionOffsets().isEmpty()) { + setCustomKafkaPartitions(stmt.getKafkaPartitionOffsets()); } - if (stmt.getDesiredConcurrentNum() != 0) { - setDesireTaskConcurrentNum(stmt.getDesiredConcurrentNum()); + } + + // this is a unprotected method which is called in the initialization function + private void setCustomKafkaPartitions(List> kafkaPartitionOffsets) throws LoadException { + // check if custom kafka partition is valid + List allKafkaPartitions = getAllKafkaPartitions(); + for (Pair partitionOffset : kafkaPartitionOffsets) { + if (!allKafkaPartitions.contains(partitionOffset.first)) { + throw new LoadException("there is a custom kafka partition " + partitionOffset.first + + " which is invalid for topic " + topic); + } + this.customKafkaPartitions.add(partitionOffset.first); + ((KafkaProgress) progress).addPartitionOffset(partitionOffset); } - if (stmt.getMaxErrorNum() != 0) { - setMaxErrorNum(stmt.getMaxErrorNum()); + } + + @Override + protected String dataSourcePropertiesJsonToString() { + Map dataSourceProperties = Maps.newHashMap(); + dataSourceProperties.put("brokerList", brokerList); + dataSourceProperties.put("topic", topic); + List sortedPartitions = Lists.newArrayList(currentKafkaPartitions); + Collections.sort(sortedPartitions); + dataSourceProperties.put("currentKafkaPartitions", Joiner.on(",").join(sortedPartitions)); + Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + return gson.toJson(dataSourceProperties); + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + Text.writeString(out, brokerList); + Text.writeString(out, topic); + + out.writeInt(customKafkaPartitions.size()); + for (Integer partitionId : customKafkaPartitions) { + out.writeInt(partitionId); } - if (stmt.getKafkaPartitions() != null) { - setCustomKafkaPartitions(stmt.getKafkaPartitions()); + } + + @Override + public void readFields(DataInput in) throws IOException { + super.readFields(in); + brokerList = Text.readString(in); + topic = Text.readString(in); + int size = in.readInt(); + for (int i = 0; i < size; i++) { + customKafkaPartitions.add(in.readInt()); } + + setConsumer(); } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java b/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java index 81aa67a4a69a30..20ac923fe2d3e4 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java @@ -18,14 +18,18 @@ package org.apache.doris.load.routineload; import org.apache.doris.catalog.Catalog; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.LabelAlreadyUsedException; -import org.apache.doris.common.LoadException; -import org.apache.doris.task.KafkaRoutineLoadTask; -import org.apache.doris.task.RoutineLoadTask; -import org.apache.doris.transaction.BeginTransactionException; +import org.apache.doris.catalog.Database; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.thrift.TExecPlanFragmentParams; +import org.apache.doris.thrift.TKafkaLoadInfo; +import org.apache.doris.thrift.TLoadSourceType; +import org.apache.doris.thrift.TPlanFragment; +import org.apache.doris.thrift.TRoutineLoadTask; +import org.apache.doris.thrift.TUniqueId; -import com.google.common.collect.Maps; +import com.google.common.base.Joiner; +import com.google.gson.Gson; import java.util.ArrayList; import java.util.List; @@ -36,46 +40,70 @@ public class KafkaTaskInfo extends RoutineLoadTaskInfo { private RoutineLoadManager routineLoadManager = Catalog.getCurrentCatalog().getRoutineLoadManager(); - private List partitions; + // + private Map partitionIdToOffset; - public KafkaTaskInfo(String id, String jobId) throws LabelAlreadyUsedException, - BeginTransactionException, AnalysisException { - super(id, jobId); - this.partitions = new ArrayList<>(); + public KafkaTaskInfo(UUID id, long jobId, String clusterName, Map partitionIdToOffset) { + super(id, jobId, clusterName); + this.partitionIdToOffset = partitionIdToOffset; } - public KafkaTaskInfo(KafkaTaskInfo kafkaTaskInfo) throws LabelAlreadyUsedException, - BeginTransactionException, AnalysisException { - super(UUID.randomUUID().toString(), kafkaTaskInfo.getJobId()); - this.partitions = kafkaTaskInfo.getPartitions(); + public KafkaTaskInfo(KafkaTaskInfo kafkaTaskInfo, Map partitionIdToOffset) { + super(UUID.randomUUID(), kafkaTaskInfo.getJobId(), kafkaTaskInfo.getClusterName(), kafkaTaskInfo.getBeId()); + this.partitionIdToOffset = partitionIdToOffset; } - public void addKafkaPartition(int partition) { - partitions.add(partition); + public List getPartitions() { + return new ArrayList<>(partitionIdToOffset.keySet()); } - public List getPartitions() { - return partitions; + @Override + public TRoutineLoadTask createRoutineLoadTask() throws UserException { + KafkaRoutineLoadJob routineLoadJob = (KafkaRoutineLoadJob) routineLoadManager.getJob(jobId); + + // init tRoutineLoadTask and create plan fragment + TRoutineLoadTask tRoutineLoadTask = new TRoutineLoadTask(); + TUniqueId queryId = new TUniqueId(id.getMostSignificantBits(), id.getLeastSignificantBits()); + tRoutineLoadTask.setId(queryId); + tRoutineLoadTask.setJob_id(jobId); + tRoutineLoadTask.setTxn_id(txnId); + Database database = Catalog.getCurrentCatalog().getDb(routineLoadJob.getDbId()); + tRoutineLoadTask.setDb(database.getFullName()); + tRoutineLoadTask.setTbl(database.getTable(routineLoadJob.getTableId()).getName()); + // label = job_name+job_id+task_id+txn_id + String label = Joiner.on("-").join(routineLoadJob.getName(), routineLoadJob.getId(), DebugUtil.printId(id), txnId); + tRoutineLoadTask.setLabel(label); + tRoutineLoadTask.setAuth_code(routineLoadJob.getAuthCode()); + TKafkaLoadInfo tKafkaLoadInfo = new TKafkaLoadInfo(); + tKafkaLoadInfo.setTopic((routineLoadJob).getTopic()); + tKafkaLoadInfo.setBrokers((routineLoadJob).getBrokerList()); + tKafkaLoadInfo.setPartition_begin_offset(partitionIdToOffset); + tRoutineLoadTask.setKafka_load_info(tKafkaLoadInfo); + tRoutineLoadTask.setType(TLoadSourceType.KAFKA); + tRoutineLoadTask.setParams(updateTExecPlanFragmentParams(routineLoadJob)); + tRoutineLoadTask.setMax_interval_s(routineLoadJob.getMaxBatchIntervalS()); + tRoutineLoadTask.setMax_batch_rows(routineLoadJob.getMaxBatchRows()); + tRoutineLoadTask.setMax_batch_size(routineLoadJob.getMaxBatchSizeBytes()); + return tRoutineLoadTask; } @Override - public RoutineLoadTask createStreamLoadTask(long beId) throws LoadException { - RoutineLoadJob routineLoadJob = routineLoadManager.getJob(jobId); - Map partitionIdToOffset = Maps.newHashMap(); - for (Integer partitionId : partitions) { - KafkaProgress kafkaProgress = (KafkaProgress) routineLoadJob.getProgress(); - if (!kafkaProgress.getPartitionIdToOffset().containsKey(partitionId)) { - kafkaProgress.getPartitionIdToOffset().put(partitionId, 0L); - } - partitionIdToOffset.put(partitionId, kafkaProgress.getPartitionIdToOffset().get(partitionId)); - } - RoutineLoadTask routineLoadTask = new KafkaRoutineLoadTask(routineLoadJob.getResourceInfo(), - beId, routineLoadJob.getDbId(), - routineLoadJob.getTableId(), - id, txnId, partitionIdToOffset); - if (routineLoadJob.getRoutineLoadDesc() != null) { - routineLoadTask.setRoutineLoadDesc(routineLoadJob.getRoutineLoadDesc()); - } - return routineLoadTask; + protected String getTaskDataSourceProperties() { + Gson gson = new Gson(); + return gson.toJson(partitionIdToOffset); + } + + private TExecPlanFragmentParams updateTExecPlanFragmentParams(RoutineLoadJob routineLoadJob) throws UserException { + // plan for each task, in case table has change(rollup or schema change) + TExecPlanFragmentParams tExecPlanFragmentParams = routineLoadJob.plan(); + TPlanFragment tPlanFragment = tExecPlanFragmentParams.getFragment(); + // we use task id as both query id(TPlanFragmentExecParams) and load id(olap table sink/scan range desc) + TUniqueId queryId = new TUniqueId(id.getMostSignificantBits(), id.getLeastSignificantBits()); + tPlanFragment.getOutput_sink().getOlap_table_sink().setLoad_id(queryId); + tPlanFragment.getOutput_sink().getOlap_table_sink().setTxn_id(this.txnId); + tExecPlanFragmentParams.getParams().setQuery_id(queryId); + tExecPlanFragmentParams.getParams().getPer_node_scan_ranges().values().stream() + .forEach(entity -> entity.get(0).getScan_range().getBroker_scan_range().getRanges().get(0).setLoad_id(queryId)); + return tExecPlanFragmentParams; } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java b/fe/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java index 71f58a0a3c1213..ef6a7f38d398c3 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java @@ -17,8 +17,9 @@ package org.apache.doris.load.routineload; -import org.apache.doris.common.io.Text; import org.apache.doris.thrift.TRLTaskTxnCommitAttachment; +import org.apache.doris.thrift.TUniqueId; +import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnCommitAttachment; import java.io.DataInput; @@ -29,116 +30,86 @@ // "numOfTotalData": "", "taskId": "", "jobId": ""} public class RLTaskTxnCommitAttachment extends TxnCommitAttachment { - public enum RoutineLoadType { - KAFKA(1); - - private final int flag; - - private RoutineLoadType(int flag) { - this.flag = flag; - } - - public int value() { - return flag; - } - - public static RoutineLoadType valueOf(int flag) { - switch (flag) { - case 1: - return KAFKA; - default: - return null; - } - } - } - + private long jobId; + private TUniqueId taskId; + private long filteredRows; + private long loadedRows; + private long unselectedRows; + private long receivedBytes; + private long taskExecutionTimeMs; private RoutineLoadProgress progress; - private long backendId; - private long taskSignature; - private int numOfErrorData; - private int numOfTotalData; - private String taskId; - private String jobId; - private RoutineLoadType routineLoadType; + private String errorLogUrl; public RLTaskTxnCommitAttachment() { + super(TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK); } public RLTaskTxnCommitAttachment(TRLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { - this.backendId = rlTaskTxnCommitAttachment.getBackendId(); - this.taskSignature = rlTaskTxnCommitAttachment.getTaskSignature(); - this.numOfErrorData = rlTaskTxnCommitAttachment.getNumOfErrorData(); - this.numOfTotalData = rlTaskTxnCommitAttachment.getNumOfTotalData(); - this.taskId = rlTaskTxnCommitAttachment.getTaskId(); + super(TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK); this.jobId = rlTaskTxnCommitAttachment.getJobId(); - switch (rlTaskTxnCommitAttachment.getRoutineLoadType()) { + this.taskId = rlTaskTxnCommitAttachment.getId(); + this.filteredRows = rlTaskTxnCommitAttachment.getFilteredRows(); + this.loadedRows = rlTaskTxnCommitAttachment.getLoadedRows(); + this.unselectedRows = rlTaskTxnCommitAttachment.getUnselectedRows(); + this.receivedBytes = rlTaskTxnCommitAttachment.getReceivedBytes(); + this.taskExecutionTimeMs = rlTaskTxnCommitAttachment.getLoadCostMs(); + + switch (rlTaskTxnCommitAttachment.getLoadSourceType()) { case KAFKA: this.progress = new KafkaProgress(rlTaskTxnCommitAttachment.getKafkaRLTaskProgress()); + break; + default: + break; } - } - - public RoutineLoadProgress getProgress() { - return progress; - } - public void setProgress(RoutineLoadProgress progress) { - this.progress = progress; - } - - public long getBackendId() { - return backendId; - } - - public void setBackendId(long backendId) { - this.backendId = backendId; - } - - public long getTaskSignature() { - return taskSignature; + if (rlTaskTxnCommitAttachment.isSetErrorLogUrl()) { + this.errorLogUrl = rlTaskTxnCommitAttachment.getErrorLogUrl(); + } } - public void setTaskSignature(long taskSignature) { - this.taskSignature = taskSignature; + public TUniqueId getTaskId() { + return taskId; } - public int getNumOfErrorData() { - return numOfErrorData; + public long getFilteredRows() { + return filteredRows; } - public void setNumOfErrorData(int numOfErrorData) { - this.numOfErrorData = numOfErrorData; + public long getLoadedRows() { + return loadedRows; } - public int getNumOfTotalData() { - return numOfTotalData; + public long getUnselectedRows() { + return unselectedRows; } - public void setNumOfTotalData(int numOfTotalData) { - this.numOfTotalData = numOfTotalData; + public long getTotalRows() { + return filteredRows + loadedRows + unselectedRows; } - public String getTaskId() { - return taskId; + public long getReceivedBytes() { + return receivedBytes; } - public void setTaskId(String taskId) { - this.taskId = taskId; + public long getTaskExecutionTimeMs() { + return taskExecutionTimeMs; } - public String getJobId() { - return jobId; + public RoutineLoadProgress getProgress() { + return progress; } - public void setJobId(String jobId) { - this.jobId = jobId; + public String getErrorLogUrl() { + return errorLogUrl; } @Override public String toString() { - return "RoutineLoadTaskTxnExtra [backendId=" + backendId - + ", taskSignature=" + taskSignature - + ", numOfErrorData=" + numOfErrorData - + ", numOfTotalData=" + numOfTotalData + return "RLTaskTxnCommitAttachment [filteredRows=" + filteredRows + + ", loadedRows=" + loadedRows + + ", unselectedRows=" + unselectedRows + + ", receivedBytes=" + receivedBytes + + ", taskExecutionTimeMs=" + taskExecutionTimeMs + ", taskId=" + taskId + ", jobId=" + jobId + ", progress=" + progress.toString() + "]"; @@ -146,30 +117,23 @@ public String toString() { @Override public void write(DataOutput out) throws IOException { - out.writeLong(backendId); - out.writeLong(taskSignature); - out.writeInt(numOfErrorData); - out.writeInt(numOfTotalData); - Text.writeString(out, taskId); - Text.writeString(out, jobId); - out.writeInt(routineLoadType.value()); + super.write(out); + out.writeLong(filteredRows); + out.writeLong(loadedRows); + out.writeLong(unselectedRows); + out.writeLong(receivedBytes); + out.writeLong(taskExecutionTimeMs); progress.write(out); } @Override public void readFields(DataInput in) throws IOException { - backendId = in.readLong(); - taskSignature = in.readLong(); - numOfErrorData = in.readInt(); - numOfTotalData = in.readInt(); - taskId = Text.readString(in); - jobId = Text.readString(in); - routineLoadType = RoutineLoadType.valueOf(in.readInt()); - switch (routineLoadType) { - case KAFKA: - KafkaProgress kafkaProgress = new KafkaProgress(); - kafkaProgress.readFields(in); - progress = kafkaProgress; - } + super.readFields(in); + filteredRows = in.readLong(); + loadedRows = in.readLong(); + unselectedRows = in.readLong(); + receivedBytes = in.readLong(); + taskExecutionTimeMs = in.readLong(); + progress = RoutineLoadProgress.read(in); } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index fe0453ca792fc4..04a7850603cee1 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -17,46 +17,63 @@ package org.apache.doris.load.routineload; +import org.apache.doris.analysis.ColumnSeparator; import org.apache.doris.analysis.CreateRoutineLoadStmt; -import org.apache.doris.analysis.TableName; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.ImportColumnDesc; +import org.apache.doris.analysis.ImportColumnsStmt; +import org.apache.doris.analysis.SqlParser; +import org.apache.doris.analysis.SqlScanner; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.LabelAlreadyUsedException; -import org.apache.doris.common.LoadException; +import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.UserException; +import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; +import org.apache.doris.common.util.TimeUtils; import org.apache.doris.load.RoutineLoadDesc; -import org.apache.doris.load.TxnStateChangeListener; -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.service.ExecuteEnv; -import org.apache.doris.service.FrontendServiceImpl; -import org.apache.doris.task.AgentTaskQueue; -import org.apache.doris.thrift.TLoadTxnCommitRequest; -import org.apache.doris.thrift.TResourceInfo; -import org.apache.doris.thrift.TTaskType; -import org.apache.doris.transaction.AbortTransactionException; -import org.apache.doris.transaction.BeginTransactionException; +import org.apache.doris.metric.MetricRepo; +import org.apache.doris.persist.RoutineLoadOperation; +import org.apache.doris.planner.StreamLoadPlanner; +import org.apache.doris.task.StreamLoadTask; +import org.apache.doris.thrift.TExecPlanFragmentParams; +import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; +import org.apache.doris.transaction.TransactionStatus; +import org.apache.doris.transaction.TxnStateChangeCallback; -import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.EvictingQueue; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.thrift.TException; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.io.StringReader; import java.util.ArrayList; import java.util.List; -import java.util.NoSuchElementException; +import java.util.Map; import java.util.Optional; +import java.util.Queue; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; /** @@ -66,132 +83,216 @@ * The desireTaskConcurrentNum means that user expect the number of concurrent stream load * The routine load job support different streaming medium such as KAFKA */ -public abstract class RoutineLoadJob implements Writable, TxnStateChangeListener { - +public abstract class RoutineLoadJob implements TxnStateChangeCallback, Writable { private static final Logger LOG = LogManager.getLogger(RoutineLoadJob.class); - private static final int DEFAULT_TASK_TIMEOUT_SECONDS = 10; - private static final int BASE_OF_ERROR_RATE = 10000; - private static final String STAR_STRING = "*"; - - /** - * +-----------------+ - * fe schedule job | NEED_SCHEDULE | user resume job - * +----------- + | <---------+ - * | | | | - * v +-----------------+ ^ - * | - * +------------+ user pause job +-------+----+ - * | RUNNING | | PAUSED | - * | +-----------------------> | | - * +----+-------+ +-------+----+ - * | | - * | +---------------+ | - * | | STOPPED | | - * +--------------> | | <-----------+ - * user stop job +---------------+ user stop| job - * | | - * | | - * | +---------------+ | - * | | CANCELLED | | - * +---------------> | | <----------+ - * system error +---------------+ system error + public static final long DEFAULT_MAX_ERROR_NUM = 0; + + public static final long DEFAULT_MAX_INTERVAL_SECOND = 10; + public static final long DEFAULT_MAX_BATCH_ROWS = 200000; + public static final long DEFAULT_MAX_BATCH_SIZE = 100 * 1024 * 1024; // 100MB + + protected static final String STAR_STRING = "*"; + /* + +-----------------+ + fe schedule job | NEED_SCHEDULE | user resume job + +-----------+ | <---------+ + | | | | + v +-----------------+ ^ + | | + +------------+ user(system)pause job +-------+----+ + | RUNNING | | PAUSED | + | +-----------------------> | | + +----+-------+ +-------+----+ + | | | + | | +---------------+ | + | | | STOPPED | | + | +---------> | | <-----------+ + | user stop job+---------------+ user stop job + | + | + | +---------------+ + | | CANCELLED | + +-------------> | | + system error +---------------+ */ public enum JobState { NEED_SCHEDULE, RUNNING, PAUSED, - STOPPED, - CANCELLED; + STOPPED, CANCELLED; public boolean isFinalState() { return this == STOPPED || this == CANCELLED; } - } - protected String id; + protected long id; protected String name; + protected String clusterName; protected long dbId; protected long tableId; - protected RoutineLoadDesc routineLoadDesc; // optional + // this code is used to verify be task request + protected long authCode; + // protected RoutineLoadDesc routineLoadDesc; // optional + protected List partitions; // optional + protected List columnDescs; // optional + protected Expr whereExpr; // optional + protected ColumnSeparator columnSeparator; // optional protected int desireTaskConcurrentNum; // optional - protected JobState state; + protected JobState state = JobState.NEED_SCHEDULE; protected LoadDataSourceType dataSourceType; - // max number of error data in ten thousand data - // maxErrorNum / BASE_OF_ERROR_RATE = max error rate of routine load job + // max number of error data in max batch rows * 10 + // maxErrorNum / (maxBatchRows * 10) = max error rate of routine load job // if current error rate is more then max error rate, the job will be paused - protected int maxErrorNum; // optional - // thrift object - protected TResourceInfo resourceInfo; + protected long maxErrorNum = DEFAULT_MAX_ERROR_NUM; // optional + /* + * The following 3 variables control the max execute time of a single task. + * The default max batch interval time is 10 secs. + * If a task can consume data from source at rate of 10MB/s, and 500B a row, + * then we can process 100MB for 10 secs, which is 200000 rows + */ + protected long maxBatchIntervalS = DEFAULT_MAX_INTERVAL_SECOND; + protected long maxBatchRows = DEFAULT_MAX_BATCH_ROWS; + protected long maxBatchSizeBytes = DEFAULT_MAX_BATCH_SIZE; + + protected int currentTaskConcurrentNum; protected RoutineLoadProgress progress; - protected String pausedReason; - // currentErrorNum and currentTotalNum will be update - // when currentTotalNum is more then ten thousand or currentErrorNum is more then maxErrorNum - protected int currentErrorNum; - protected int currentTotalNum; + protected String pauseReason; + protected String cancelReason; + + protected long createTimestamp = System.currentTimeMillis(); + protected long pauseTimestamp = -1; + protected long endTimestamp = -1; + + /* + * The following variables are for statistics + * currentErrorRows/currentTotalRows: the row statistics of current sampling period + * errorRows/totalRows/receivedBytes: cumulative measurement + * totalTaskExcutorTimeMs: cumulative execution time of tasks + */ + /* + * Rows will be updated after txn state changed when txn state has been successfully changed. + */ + protected long currentErrorRows = 0; + protected long currentTotalRows = 0; + protected long errorRows = 0; + protected long totalRows = 0; + protected long unselectedRows = 0; + protected long receivedBytes = 0; + protected long totalTaskExcutionTimeMs = 1; // init as 1 to avoid division by zero + protected long committedTaskNum = 0; + protected long abortedTaskNum = 0; // The tasks belong to this job - protected List routineLoadTaskInfoList; - protected List needScheduleTaskInfoList; + protected List routineLoadTaskInfoList = Lists.newArrayList(); + + // stream load planer will be initialized during job schedule + StreamLoadPlanner planner; + + // this is the origin stmt of CreateRoutineLoadStmt, we use it to persist the RoutineLoadJob, + // because we can not serialize the Expressions contained in job. + protected String origStmt; - protected ReentrantReadWriteLock lock; + protected ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); // TODO(ml): error sample - public RoutineLoadJob(String name, long dbId, long tableId, LoadDataSourceType dataSourceType) { - this.id = UUID.randomUUID().toString(); - this.name = name; - this.dbId = dbId; - this.tableId = tableId; - this.state = JobState.NEED_SCHEDULE; - this.dataSourceType = dataSourceType; - this.resourceInfo = ConnectContext.get().toResourceCtx(); - this.routineLoadTaskInfoList = new ArrayList<>(); - this.needScheduleTaskInfoList = new ArrayList<>(); - lock = new ReentrantReadWriteLock(true); - } - - // TODO(ml): I will change it after ut. - @VisibleForTesting - public RoutineLoadJob(String id, String name, long dbId, long tableId, - RoutineLoadDesc routineLoadDesc, - int desireTaskConcurrentNum, LoadDataSourceType dataSourceType, - int maxErrorNum) { + // save the latest 3 error log urls + private Queue errorLogUrls = EvictingQueue.create(3); + + protected boolean isTypeRead = false; + + public void setTypeRead(boolean isTypeRead) { + this.isTypeRead = isTypeRead; + } + + public RoutineLoadJob(long id, LoadDataSourceType type) { this.id = id; + this.dataSourceType = type; + } + + public RoutineLoadJob(Long id, String name, String clusterName, long dbId, long tableId, + LoadDataSourceType dataSourceType) { + this(id, dataSourceType); this.name = name; + this.clusterName = clusterName; this.dbId = dbId; this.tableId = tableId; - this.routineLoadDesc = routineLoadDesc; - this.desireTaskConcurrentNum = desireTaskConcurrentNum; - this.state = JobState.NEED_SCHEDULE; - this.dataSourceType = dataSourceType; - this.maxErrorNum = maxErrorNum; - this.resourceInfo = ConnectContext.get().toResourceCtx(); - this.routineLoadTaskInfoList = new ArrayList<>(); - this.needScheduleTaskInfoList = new ArrayList<>(); - lock = new ReentrantReadWriteLock(true); - } - - public void readLock() { + this.authCode = 0; + } + + protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException { + setRoutineLoadDesc(stmt.getRoutineLoadDesc()); + if (stmt.getDesiredConcurrentNum() != -1) { + this.desireTaskConcurrentNum = stmt.getDesiredConcurrentNum(); + } + if (stmt.getMaxErrorNum() != -1) { + this.maxErrorNum = stmt.getMaxErrorNum(); + } + if (stmt.getMaxBatchIntervalS() != -1) { + this.maxBatchIntervalS = stmt.getMaxBatchIntervalS(); + } + if (stmt.getMaxBatchRows() != -1) { + this.maxBatchRows = stmt.getMaxBatchRows(); + } + if (stmt.getMaxBatchSize() != -1) { + this.maxBatchSizeBytes = stmt.getMaxBatchSize(); + } + } + + private void setRoutineLoadDesc(RoutineLoadDesc routineLoadDesc) { + if (routineLoadDesc != null) { + if (routineLoadDesc.getColumnsInfo() != null) { + ImportColumnsStmt columnsStmt = routineLoadDesc.getColumnsInfo(); + if (columnsStmt.getColumns() != null || columnsStmt.getColumns().size() != 0) { + columnDescs = Lists.newArrayList(); + for (ImportColumnDesc columnDesc : columnsStmt.getColumns()) { + columnDescs.add(columnDesc); + } + } + } + if (routineLoadDesc.getWherePredicate() != null) { + whereExpr = routineLoadDesc.getWherePredicate().getExpr(); + } + if (routineLoadDesc.getColumnSeparator() != null) { + columnSeparator = routineLoadDesc.getColumnSeparator(); + } + if (routineLoadDesc.getPartitionNames() != null && routineLoadDesc.getPartitionNames().size() != 0) { + partitions = routineLoadDesc.getPartitionNames(); + } + } + } + + @Override + public long getId() { + return id; + } + + protected void readLock() { lock.readLock().lock(); } - public void readUnlock() { + protected void readUnlock() { lock.readLock().unlock(); } - public void writeLock() { + protected void writeLock() { lock.writeLock().lock(); } - public void writeUnlock() { + protected void writeUnlock() { lock.writeLock().unlock(); } - public String getId() { - return id; + protected boolean tryWriteLock(long timeout, TimeUnit unit) { + try { + return this.lock.writeLock().tryLock(timeout, unit); + } catch (InterruptedException e) { + LOG.warn("failed to try write lock at db[" + id + "]", e); + return false; + } } public String getName() { @@ -202,20 +303,34 @@ public long getDbId() { return dbId; } - public String getDbFullName() { + public String getDbFullName() throws MetaNotFoundException { Database database = Catalog.getCurrentCatalog().getDb(dbId); - return database.getFullName(); + if (database == null) { + throw new MetaNotFoundException("Database " + dbId + "has been deleted"); + } + database.readLock(); + try { + return database.getFullName(); + } finally { + database.readUnlock(); + } } public long getTableId() { return tableId; } - public String getTableName() { + public String getTableName() throws MetaNotFoundException { Database database = Catalog.getCurrentCatalog().getDb(dbId); + if (database == null) { + throw new MetaNotFoundException("Database " + dbId + "has been deleted"); + } database.readLock(); try { Table table = database.getTable(tableId); + if (table == null) { + throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId); + } return table.getName(); } finally { database.readUnlock(); @@ -226,72 +341,44 @@ public JobState getState() { return state; } - public void setState(JobState state) { - this.state = state; + public long getAuthCode() { + return authCode; } - protected void setRoutineLoadDesc(RoutineLoadDesc routineLoadDesc) throws LoadException { - writeLock(); - try { - if (this.routineLoadDesc != null) { - throw new LoadException("Routine load desc has been initialized"); - } - this.routineLoadDesc = routineLoadDesc; - } finally { - writeUnlock(); - } + public long getEndTimestamp() { + return endTimestamp; } - public RoutineLoadDesc getRoutineLoadDesc() { - return routineLoadDesc; + public List getPartitions() { + return partitions; } - public TResourceInfo getResourceInfo() { - return resourceInfo; + public List getColumnDescs() { + return columnDescs; } - public RoutineLoadProgress getProgress() { - return progress; + public Expr getWhereExpr() { + return whereExpr; } - public String getPartitions() { - if (routineLoadDesc.getPartitionNames() == null || routineLoadDesc.getPartitionNames().size() == 0) { - return STAR_STRING; - } else { - return String.join(",", routineLoadDesc.getPartitionNames()); - } + public ColumnSeparator getColumnSeparator() { + return columnSeparator; } - protected void setDesireTaskConcurrentNum(int desireTaskConcurrentNum) throws LoadException { - writeLock(); - try { - if (this.desireTaskConcurrentNum != 0) { - throw new LoadException("Desired task concurrent num has been initialized"); - } - this.desireTaskConcurrentNum = desireTaskConcurrentNum; - } finally { - writeUnlock(); - } + public RoutineLoadProgress getProgress() { + return progress; } - public String getDesiredConcurrentNumber() { - if (desireTaskConcurrentNum == 0) { - return ""; - } else { - return String.valueOf(desireTaskConcurrentNum); - } + public long getMaxBatchIntervalS() { + return maxBatchIntervalS; } - protected void setMaxErrorNum(int maxErrorNum) throws LoadException { - writeLock(); - try { - if (this.maxErrorNum != 0) { - throw new LoadException("Max error num has been initialized"); - } - this.maxErrorNum = maxErrorNum; - } finally { - writeUnlock(); - } + public long getMaxBatchRows() { + return maxBatchRows; + } + + public long getMaxBatchSizeBytes() { + return maxBatchSizeBytes; } public int getSizeOfRoutineLoadTaskInfoList() { @@ -301,93 +388,56 @@ public int getSizeOfRoutineLoadTaskInfoList() { } finally { readUnlock(); } - - } - - public List getNeedScheduleTaskInfoList() { - return needScheduleTaskInfoList; - } - - public void updateState(JobState jobState) { - writeLock(); - try { - state = jobState; - } finally { - writeUnlock(); - } } - public List processTimeoutTasks() { - List result = new ArrayList<>(); + // only check loading task + public void processTimeoutTasks() { writeLock(); try { List runningTasks = new ArrayList<>(routineLoadTaskInfoList); - runningTasks.removeAll(needScheduleTaskInfoList); - for (RoutineLoadTaskInfo routineLoadTaskInfo : runningTasks) { - if ((System.currentTimeMillis() - routineLoadTaskInfo.getLoadStartTimeMs()) - > DEFAULT_TASK_TIMEOUT_SECONDS * 1000) { - String oldSignature = routineLoadTaskInfo.getId(); - // abort txn if not committed - try { - Catalog.getCurrentGlobalTransactionMgr() - .abortTransaction(routineLoadTaskInfo.getTxnId(), "routine load task of txn was timeout"); - } catch (UserException e) { - if (e.getMessage().contains("committed")) { - LOG.debug("txn of task {} has been committed, timeout task has been ignored", oldSignature); - continue; - } - } - - try { - result.add(reNewTask(routineLoadTaskInfo)); - LOG.debug("Task {} was ran more then {} minutes. It was removed and rescheduled", - oldSignature, DEFAULT_TASK_TIMEOUT_SECONDS); - } catch (UserException e) { - state = JobState.CANCELLED; - // TODO(ml): edit log - LOG.warn("failed to renew a routine load task in job {} with error message {}", id, e.getMessage()); - } + if (routineLoadTaskInfo.isRunning() + && ((System.currentTimeMillis() - routineLoadTaskInfo.getExecuteStartTimeMs()) + > maxBatchIntervalS * 2 * 1000)) { + RoutineLoadTaskInfo newTask = unprotectRenewTask(routineLoadTaskInfo); + Catalog.getCurrentCatalog().getRoutineLoadTaskScheduler().addTaskInQueue(newTask); } } } finally { writeUnlock(); } - return result; } - abstract List divideRoutineLoadJob(int currentConcurrentTaskNum); + abstract void divideRoutineLoadJob(int currentConcurrentTaskNum) throws UserException; public int calculateCurrentConcurrentTaskNum() throws MetaNotFoundException { return 0; } - @Override - public void write(DataOutput out) throws IOException { - // TODO(ml) - } - - @Override - public void readFields(DataInput in) throws IOException { - // TODO(ml) - } - - - public void removeNeedScheduleTask(RoutineLoadTaskInfo routineLoadTaskInfo) { - writeLock(); + public Map getBeIdToConcurrentTaskNum() { + Map beIdConcurrentTasksNum = Maps.newHashMap(); + readLock(); try { - needScheduleTaskInfoList.remove(routineLoadTaskInfo); + for (RoutineLoadTaskInfo routineLoadTaskInfo : routineLoadTaskInfoList) { + if (routineLoadTaskInfo.getBeId() != -1L) { + long beId = routineLoadTaskInfo.getBeId(); + if (beIdConcurrentTasksNum.containsKey(beId)) { + beIdConcurrentTasksNum.put(beId, beIdConcurrentTasksNum.get(beId) + 1); + } else { + beIdConcurrentTasksNum.put(beId, 1); + } + } + } + return beIdConcurrentTasksNum; } finally { - writeUnlock(); + readUnlock(); } } - abstract void updateProgress(RoutineLoadProgress progress); - - public boolean containsTask(String taskId) { + public boolean containsTask(UUID taskId) { readLock(); try { - return routineLoadTaskInfoList.parallelStream() + return routineLoadTaskInfoList.stream() .anyMatch(entity -> entity.getId().equals(taskId)); } finally { readUnlock(); @@ -395,238 +445,723 @@ public boolean containsTask(String taskId) { } // All of private method could not be call without lock - private void checkStateTransform(RoutineLoadJob.JobState desireState) - throws UnsupportedOperationException { + private void checkStateTransform(RoutineLoadJob.JobState desireState) throws UserException { switch (state) { case RUNNING: if (desireState == JobState.NEED_SCHEDULE) { - throw new UnsupportedOperationException("Could not transform " + state + " to " + desireState); + throw new DdlException("Could not transform " + state + " to " + desireState); } break; case PAUSED: if (desireState == JobState.PAUSED) { - throw new UnsupportedOperationException("Could not transform " + state + " to " + desireState); + throw new DdlException("Could not transform " + state + " to " + desireState); } break; case STOPPED: case CANCELLED: - throw new UnsupportedOperationException("Could not transfrom " + state + " to " + desireState); + throw new DdlException("Could not transform " + state + " to " + desireState); default: break; } } - private void loadTxnCommit(TLoadTxnCommitRequest request) throws TException { - FrontendServiceImpl frontendService = new FrontendServiceImpl(ExecuteEnv.getInstance()); - frontendService.loadTxnCommit(request); + // if rate of error data is more then max_filter_ratio, pause job + protected void updateProgress(RLTaskTxnCommitAttachment attachment) throws UserException { + updateNumOfData(attachment.getTotalRows(), attachment.getFilteredRows(), attachment.getUnselectedRows(), + attachment.getReceivedBytes(), attachment.getTaskExecutionTimeMs(), + false /* not replay */); } - private void updateNumOfData(int numOfErrorData, int numOfTotalData) { - currentErrorNum += numOfErrorData; - currentTotalNum += numOfTotalData; - if (currentTotalNum > BASE_OF_ERROR_RATE) { - if (currentErrorNum > maxErrorNum) { - LOG.info("current error num {} of job {} is more then max error num {}. begin to pause job", - currentErrorNum, id, maxErrorNum); - // remove all of task in jobs and change job state to paused - executePause("current error num of job is more then max error num"); + private void updateNumOfData(long numOfTotalRows, long numOfErrorRows, long unselectedRows, long receivedBytes, + long taskExecutionTime, boolean isReplay) throws UserException { + this.totalRows += numOfTotalRows; + this.errorRows += numOfErrorRows; + this.unselectedRows += unselectedRows; + this.receivedBytes += receivedBytes; + this.totalTaskExcutionTimeMs += taskExecutionTime; + + if (MetricRepo.isInit.get() && !isReplay) { + MetricRepo.COUNTER_ROUTINE_LOAD_ROWS.increase(numOfTotalRows); + MetricRepo.COUNTER_ROUTINE_LOAD_ERROR_ROWS.increase(numOfErrorRows); + MetricRepo.COUNTER_ROUTINE_LOAD_RECEIVED_BYTES.increase(receivedBytes); + } + + // check error rate + currentErrorRows += numOfErrorRows; + currentTotalRows += numOfTotalRows; + if (currentTotalRows > maxBatchRows * 10) { + if (currentErrorRows > maxErrorNum) { + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("current_total_rows", currentTotalRows) + .add("current_error_rows", currentErrorRows) + .add("max_error_num", maxErrorNum) + .add("msg", "current error rows is more then max error num, begin to pause job") + .build()); + // if this is a replay thread, the update state should already be replayed by OP_CHANGE_ROUTINE_LOAD_JOB + if (!isReplay) { + // remove all of task in jobs and change job state to paused + updateState(JobState.PAUSED, "current error rows of job is more then max error num", isReplay); + } } + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("current_total_rows", currentTotalRows) + .add("current_error_rows", currentErrorRows) + .add("max_error_num", maxErrorNum) + .add("msg", "reset current total rows and current error rows " + + "when current total rows is more then base") + .build()); + } // reset currentTotalNum and currentErrorNum - currentErrorNum = 0; - currentTotalNum = 0; - } else if (currentErrorNum > maxErrorNum) { - LOG.info("current error num {} of job {} is more then max error num {}. begin to pause job", - currentErrorNum, id, maxErrorNum); - // remove all of task in jobs and change job state to paused - executePause("current error num is more then max error num"); + currentErrorRows = 0; + currentTotalRows = 0; + } else if (currentErrorRows > maxErrorNum) { + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("current_total_rows", currentTotalRows) + .add("current_error_rows", currentErrorRows) + .add("max_error_num", maxErrorNum) + .add("msg", "current error rows is more then max error rows, begin to pause job") + .build()); + if (!isReplay) { + // remove all of task in jobs and change job state to paused + updateState(JobState.PAUSED, "current error rows is more then max error num", isReplay); + } // reset currentTotalNum and currentErrorNum - currentErrorNum = 0; - currentTotalNum = 0; + currentErrorRows = 0; + currentTotalRows = 0; } } - abstract RoutineLoadTaskInfo reNewTask(RoutineLoadTaskInfo routineLoadTaskInfo) throws AnalysisException, - LabelAlreadyUsedException, BeginTransactionException; + protected void replayUpdateProgress(RLTaskTxnCommitAttachment attachment) { + try { + updateNumOfData(attachment.getTotalRows(), attachment.getFilteredRows(), attachment.getUnselectedRows(), + attachment.getReceivedBytes(), attachment.getTaskExecutionTimeMs(), true /* is replay */); + } catch (UserException e) { + LOG.error("should not happen", e); + } + } - @Override - public void beforeAborted(TransactionState txnState, TransactionState.TxnStatusChangeReason txnStatusChangeReason) - throws AbortTransactionException { - readLock(); + abstract RoutineLoadTaskInfo unprotectRenewTask(RoutineLoadTaskInfo routineLoadTaskInfo); + + public void initPlanner() throws UserException { + StreamLoadTask streamLoadTask = StreamLoadTask.fromRoutineLoadJob(this); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + throw new MetaNotFoundException("db " + dbId + " does not exist"); + } + planner = new StreamLoadPlanner(db, (OlapTable) db.getTable(this.tableId), streamLoadTask); + } + + public TExecPlanFragmentParams plan() throws UserException { + Preconditions.checkNotNull(planner); + Database db = Catalog.getCurrentCatalog().getDb(dbId); + if (db == null) { + throw new MetaNotFoundException("db " + dbId + " does not exist"); + } + db.readLock(); try { - if (txnStatusChangeReason != null) { - switch (txnStatusChangeReason) { - case TIMEOUT: - String taskId = txnState.getLabel(); - if (routineLoadTaskInfoList.parallelStream().anyMatch(entity -> entity.getId().equals(taskId))) { - throw new AbortTransactionException( - "there are task " + taskId + " related to this txn, " - + "txn could not be abort", txnState.getTransactionId()); - } - break; - } - } + return planner.plan(); } finally { - readUnlock(); + db.readUnlock(); } } + // if task not exists, before aborted will reset the txn attachment to null, task will not be updated + // if task pass the checker, task will be updated by attachment + // *** Please do not call before individually. It must be combined use with after *** @Override - public void onCommitted(TransactionState txnState) { - // step0: get progress from transaction state - RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment = (RLTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(); + public void beforeAborted(TransactionState txnState) throws TransactionException { + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) + .add("txn_state", txnState) + .add("msg", "task before aborted") + .build()); + } + executeBeforeCheck(txnState, TransactionStatus.ABORTED); + } + // if task not exists, before committed will throw exception, commit txn will failed + // if task pass the checker, lock job will be locked + // *** Please do not call before individually. It must be combined use with after *** + @Override + public void beforeCommitted(TransactionState txnState) throws TransactionException { + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) + .add("txn_state", txnState) + .add("msg", "task before committed") + .build()); + } + executeBeforeCheck(txnState, TransactionStatus.COMMITTED); + } + + /* + * try lock the write lock. + * Make sure lock is released if any exception being thrown + */ + private void executeBeforeCheck(TransactionState txnState, TransactionStatus transactionStatus) + throws TransactionException { writeLock(); + + // task already pass the checker + boolean passCheck = false; try { - // step1: find task in job + // check if task has been aborted Optional routineLoadTaskInfoOptional = - routineLoadTaskInfoList.parallelStream() - .filter(entity -> entity.getId().equals(txnState.getLabel())).findFirst(); - RoutineLoadTaskInfo routineLoadTaskInfo = routineLoadTaskInfoOptional.get(); - - // step2: update job progress - updateProgress(rlTaskTxnCommitAttachment.getProgress()); - - // step3: remove task in agentTaskQueue - AgentTaskQueue.removeTask(rlTaskTxnCommitAttachment.getBackendId(), TTaskType.STREAM_LOAD, - rlTaskTxnCommitAttachment.getTaskSignature()); - - // step4: if rate of error data is more then max_filter_ratio, pause job - updateNumOfData(rlTaskTxnCommitAttachment.getNumOfErrorData(), rlTaskTxnCommitAttachment.getNumOfTotalData()); + routineLoadTaskInfoList.stream() + .filter(entity -> entity.getTxnId() == txnState.getTransactionId()).findFirst(); + if (!routineLoadTaskInfoOptional.isPresent()) { + switch (transactionStatus) { + case COMMITTED: + throw new TransactionException("txn " + txnState.getTransactionId() + + " could not be " + transactionStatus + + " while task " + txnState.getLabel() + " has been aborted."); + case ABORTED: + // reset attachment in txn state + // txn will be aborted normal but without attachment + // task will not be update when attachment is null + txnState.setTxnCommitAttachment(null); + break; + } + } + passCheck = true; + } finally { + if (!passCheck) { + writeUnlock(); + } + } + } - if (state == JobState.RUNNING) { - // step5: create a new task for partitions - RoutineLoadTaskInfo newRoutineLoadTaskInfo = reNewTask(routineLoadTaskInfo); - Catalog.getCurrentCatalog().getRoutineLoadManager() - .getNeedScheduleTasksQueue().add(newRoutineLoadTaskInfo); + // txn already committed before calling afterCommitted + // the task will be committed + // check currentErrorRows > maxErrorRows + // paused job or renew task + // *** Please do not call after individually. It must be combined use with before *** + @Override + public void afterCommitted(TransactionState txnState, boolean txnOperated) throws UserException { + long taskBeId = -1L; + try { + if (txnOperated) { + // find task in job + Optional routineLoadTaskInfoOptional = routineLoadTaskInfoList.stream().filter( + entity -> entity.getTxnId() == txnState.getTransactionId()).findFirst(); + RoutineLoadTaskInfo routineLoadTaskInfo = routineLoadTaskInfoOptional.get(); + taskBeId = routineLoadTaskInfo.getBeId(); + executeCommitTask(routineLoadTaskInfo, txnState); + ++committedTaskNum; } - } catch (NoSuchElementException e) { - LOG.debug("There is no {} task in task info list. Maybe task has been renew or job state has changed. " - + " Transaction {} will not be committed", - txnState.getLabel(), txnState.getTransactionId()); } catch (Throwable e) { - LOG.error("failed to update offset in routine load task {} when transaction {} has been committed. " - + "change job to paused", - rlTaskTxnCommitAttachment.getTaskId(), txnState.getTransactionId(), e); - executePause("failed to update offset when transaction " - + txnState.getTransactionId() + " has been committed"); + LOG.warn("after committed failed", e); + updateState(JobState.PAUSED, "be " + taskBeId + " commit task failed " + txnState.getLabel() + + " with error " + e.getMessage() + + " while transaction " + txnState.getTransactionId() + " has been committed", false /* not replay */); } finally { writeUnlock(); } } @Override - public void onAborted(TransactionState txnState, TransactionState.TxnStatusChangeReason txnStatusChangeReason) { - pause(txnStatusChangeReason.name()); - LOG.debug("job {} need to be pause while txn {} abort with reason {}", - id, txnState.getTransactionId(), txnStatusChangeReason.name()); + public void replayOnCommitted(TransactionState txnState) { + Preconditions.checkNotNull(txnState.getTxnCommitAttachment(), txnState); + replayUpdateProgress((RLTaskTxnCommitAttachment) txnState.getTxnCommitAttachment()); + this.committedTaskNum++; + LOG.debug("replay on committed: {}", txnState); } - protected static void checkCreate(CreateRoutineLoadStmt stmt) throws AnalysisException { - // check table belong to db, partitions belong to table - if (stmt.getRoutineLoadDesc() == null) { - checkDBSemantics(stmt.getDBTableName(), null); - } else { - checkDBSemantics(stmt.getDBTableName(), stmt.getRoutineLoadDesc().getPartitionNames()); + // the task is aborted when the correct number of rows is more then 0 + // be will abort txn when all of kafka data is wrong or total consume data is 0 + // txn will be aborted but progress will be update + // progress will be update otherwise the progress will be hung + // *** Please do not call after individually. It must be combined use with before *** + @Override + public void afterAborted(TransactionState txnState, boolean txnOperated, String txnStatusChangeReasonString) + throws UserException { + long taskBeId = -1L; + try { + if (txnOperated) { + if (txnState.getTxnCommitAttachment() == null) { + // this is a task which already has been aborted by fe + return; + } + // step0: find task in job + Optional routineLoadTaskInfoOptional = routineLoadTaskInfoList.stream().filter( + entity -> entity.getTxnId() == txnState.getTransactionId()).findFirst(); + RoutineLoadTaskInfo routineLoadTaskInfo = routineLoadTaskInfoOptional.get(); + taskBeId = routineLoadTaskInfo.getBeId(); + // step1: job state will be changed depending on txnStatusChangeReasonString + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) + .add("txn_id", txnState.getTransactionId()) + .add("msg", "txn abort with reason " + txnStatusChangeReasonString) + .build()); + } + ++abortedTaskNum; + if (txnStatusChangeReasonString != null) { + TransactionState.TxnStatusChangeReason txnStatusChangeReason = + TransactionState.TxnStatusChangeReason.fromString(txnStatusChangeReasonString); + if (txnStatusChangeReason != null) { + switch (txnStatusChangeReason) { + case OFFSET_OUT_OF_RANGE: + updateState(JobState.PAUSED, "be " + taskBeId + " abort task " + + "with reason " + txnStatusChangeReason.toString(), false /* not replay */); + return; + default: + break; + } + } + // TODO(ml): use previous be id depend on change reason + } + // step2: commit task , update progress, maybe create a new task + executeCommitTask(routineLoadTaskInfo, txnState); + } + } catch (Exception e) { + updateState(JobState.PAUSED, "be " + taskBeId + " abort task " + txnState.getLabel() + " failed with error " + e.getMessage(), + false /* not replay */); + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("task_id", txnState.getLabel()) + .add("error_msg", "change job state to paused when task has been aborted with error " + e.getMessage()) + .build(), e); + } finally { + writeUnlock(); } } - private static void checkDBSemantics(TableName dbTableName, List partitionNames) - throws AnalysisException { - String tableName = dbTableName.getTbl(); - String dbName = dbTableName.getDb(); - // check database - Database database = Catalog.getCurrentCatalog().getDb(dbName); - if (database == null) { - throw new AnalysisException("There is no database named " + dbName); + @Override + public void replayOnAborted(TransactionState txnState) { + // attachment may be null if this task is aborted by FE + if (txnState.getTxnCommitAttachment() != null) { + replayUpdateProgress((RLTaskTxnCommitAttachment) txnState.getTxnCommitAttachment()); } + this.abortedTaskNum++; + LOG.debug("replay on aborted: {}, has attachment: {}", txnState, txnState.getTxnCommitAttachment() == null); + } - database.readLock(); - try { - Table table = database.getTable(tableName); - // check table belong to database - if (table == null) { - throw new AnalysisException("There is no table named " + tableName + " in " + dbName); - } - // check table type - if (table.getType() != Table.TableType.OLAP) { - throw new AnalysisException("Only doris table support routine load"); + // check task exists or not before call method + private void executeCommitTask(RoutineLoadTaskInfo routineLoadTaskInfo, TransactionState txnState) + throws UserException { + // step0: get progress from transaction state + RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment = (RLTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(); + if (rlTaskTxnCommitAttachment == null) { + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, routineLoadTaskInfo.getId()) + .add("job_id", routineLoadTaskInfo.getJobId()) + .add("txn_id", routineLoadTaskInfo.getTxnId()) + .add("msg", "commit task will be ignore when attachment txn of task is null," + + " maybe task was aborted by master when timeout") + .build()); } + } else if (checkCommitInfo(rlTaskTxnCommitAttachment)) { + // step2: update job progress + updateProgress(rlTaskTxnCommitAttachment); + } - if (partitionNames == null || partitionNames.size() == 0) { - return; - } - // check partitions belong to table - Optional partitionNotInTable = partitionNames.parallelStream() - .filter(entity -> ((OlapTable) table).getPartition(entity) == null).findFirst(); - if (partitionNotInTable != null && partitionNotInTable.isPresent()) { - throw new AnalysisException("Partition " + partitionNotInTable.get() - + " does not belong to table " + tableName); + if (rlTaskTxnCommitAttachment != null && !Strings.isNullOrEmpty(rlTaskTxnCommitAttachment.getErrorLogUrl())) { + errorLogUrls.add(rlTaskTxnCommitAttachment.getErrorLogUrl()); + } + + if (state == JobState.RUNNING) { + // step2: create a new task for partitions + RoutineLoadTaskInfo newRoutineLoadTaskInfo = unprotectRenewTask(routineLoadTaskInfo); + Catalog.getCurrentCatalog().getRoutineLoadTaskScheduler().addTaskInQueue(newRoutineLoadTaskInfo); + } + } + + protected static void unprotectedCheckMeta(Database db, String tblName, RoutineLoadDesc routineLoadDesc) + throws UserException { + Table table = db.getTable(tblName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName); + } + + if (table.getType() != Table.TableType.OLAP) { + throw new AnalysisException("Only olap table support routine load"); + } + + if (routineLoadDesc == null) { + return; + } + + List partitionNames = routineLoadDesc.getPartitionNames(); + if (partitionNames == null || partitionNames.isEmpty()) { + return; + } + + // check partitions + OlapTable olapTable = (OlapTable) table; + for (String partName : partitionNames) { + if (olapTable.getPartition(partName) == null) { + throw new DdlException("Partition " + partName + " does not exist"); } - } finally { - database.readUnlock(); } + + // columns will be checked when planing } - public void pause(String reason) { + public void updateState(JobState jobState, String reason, boolean isReplay) throws UserException { writeLock(); try { - checkStateTransform(JobState.PAUSED); - executePause(reason); + unprotectUpdateState(jobState, reason, isReplay); } finally { writeUnlock(); } } + protected void unprotectUpdateState(JobState jobState, String reason, boolean isReplay) throws UserException { + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("current_job_state", getState()) + .add("desire_job_state", jobState) + .add("msg", "job will be change to desire state") + .build()); + } + checkStateTransform(jobState); + switch (jobState) { + case RUNNING: + executeRunning(); + break; + case PAUSED: + executePause(reason); + break; + case NEED_SCHEDULE: + executeNeedSchedule(); + break; + case STOPPED: + executeStop(); + break; + case CANCELLED: + executeCancel(reason); + break; + default: + break; + } + + if (state.isFinalState()) { + Catalog.getCurrentGlobalTransactionMgr().getCallbackFactory().removeCallback(id); + } + + if (!isReplay && jobState != JobState.RUNNING) { + Catalog.getInstance().getEditLog().logOpRoutineLoadJob(new RoutineLoadOperation(id, jobState)); + } + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("current_job_state", getState()) + .add("msg", "job state has been changed") + .add("is replay", String.valueOf(isReplay)) + .build()); + } + + private void executeRunning() { + state = JobState.RUNNING; + } + private void executePause(String reason) { - // TODO(ml): edit log // remove all of task in jobs and change job state to paused - pausedReason = reason; + pauseReason = reason; state = JobState.PAUSED; + pauseTimestamp = System.currentTimeMillis(); routineLoadTaskInfoList.clear(); - needScheduleTaskInfoList.clear(); } - public void resume() { - // TODO(ml): edit log - writeLock(); + private void executeNeedSchedule() { + state = JobState.NEED_SCHEDULE; + pauseTimestamp = -1; + routineLoadTaskInfoList.clear(); + } + + private void executeStop() { + state = JobState.STOPPED; + routineLoadTaskInfoList.clear(); + endTimestamp = System.currentTimeMillis(); + } + + private void executeCancel(String reason) { + cancelReason = reason; + state = JobState.CANCELLED; + routineLoadTaskInfoList.clear(); + endTimestamp = System.currentTimeMillis(); + } + + public void update() throws UserException { + // check if db and table exist + Database database = Catalog.getCurrentCatalog().getDb(dbId); + if (database == null) { + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("db_id", dbId) + .add("msg", "The database has been deleted. Change job state to cancelled").build()); + writeLock(); + try { + if (!state.isFinalState()) { + unprotectUpdateState(JobState.CANCELLED, "db " + dbId + "not exist", false /* not replay */); + } + return; + } finally { + writeUnlock(); + } + } + + // check table belong to database + database.readLock(); + Table table; try { - checkStateTransform(JobState.NEED_SCHEDULE); - state = JobState.NEED_SCHEDULE; + table = database.getTable(tableId); } finally { - writeUnlock(); + database.readUnlock(); + } + if (table == null) { + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id).add("db_id", dbId) + .add("table_id", tableId) + .add("msg", "The table has been deleted change job state to cancelled").build()); + writeLock(); + try { + if (!state.isFinalState()) { + unprotectUpdateState(JobState.CANCELLED, "table not exist", false /* not replay */); + } + return; + } finally { + writeUnlock(); + } } - } - public void stop() { - // TODO(ml): edit log + // check if partition has been changed writeLock(); try { - checkStateTransform(JobState.STOPPED); - state = JobState.STOPPED; - routineLoadTaskInfoList.clear(); - needScheduleTaskInfoList.clear(); + if (unprotectNeedReschedule()) { + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("msg", "Job need to be rescheduled") + .build()); + unprotectUpdateProgress(); + executeNeedSchedule(); + } } finally { writeUnlock(); } } - public void reschedule() { - if (needReschedule()) { - writeLock(); + protected void unprotectUpdateProgress() { + } + + protected boolean unprotectNeedReschedule() throws UserException { + return false; + } + + public void setOrigStmt(String origStmt) { + this.origStmt = origStmt; + } + + // check the correctness of commit info + protected abstract boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment); + + protected abstract String getStatistic(); + + public List getShowInfo() { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + Table tbl = null; + if (db != null) { + db.readLock(); try { - if (state == JobState.RUNNING) { - state = JobState.NEED_SCHEDULE; - routineLoadTaskInfoList.clear(); - needScheduleTaskInfoList.clear(); - } + tbl = db.getTable(tableId); } finally { - writeUnlock(); + db.readUnlock(); } } + + readLock(); + try { + List row = Lists.newArrayList(); + row.add(String.valueOf(id)); + row.add(name); + row.add(TimeUtils.longToTimeString(createTimestamp)); + row.add(TimeUtils.longToTimeString(pauseTimestamp)); + row.add(TimeUtils.longToTimeString(endTimestamp)); + row.add(db == null ? String.valueOf(dbId) : db.getFullName()); + row.add(tbl == null ? String.valueOf(tableId) : tbl.getName()); + row.add(getState().name()); + row.add(dataSourceType.name()); + row.add(String.valueOf(getSizeOfRoutineLoadTaskInfoList())); + row.add(jobPropertiesToJsonString()); + row.add(dataSourcePropertiesJsonToString()); + row.add(getStatistic()); + row.add(getProgress().toJsonString()); + switch (state) { + case PAUSED: + row.add(pauseReason); + break; + case CANCELLED: + row.add(cancelReason); + break; + default: + row.add(""); + } + row.add(Joiner.on(", ").join(errorLogUrls)); + return row; + } finally { + readUnlock(); + } + } + + public List> getTasksShowInfo() { + List> rows = Lists.newArrayList(); + routineLoadTaskInfoList.stream().forEach(entity -> rows.add(entity.getTaskShowInfo())); + return rows; + } + + public List getShowStatistic() { + Database db = Catalog.getCurrentCatalog().getDb(dbId); + + List row = Lists.newArrayList(); + row.add(name); + row.add(String.valueOf(id)); + row.add(db == null ? String.valueOf(dbId) : db.getFullName()); + row.add(getStatistic()); + row.add(getTaskStatistic()); + return row; + } + + private String getTaskStatistic() { + Map result = Maps.newHashMap(); + result.put("running_task", + String.valueOf(routineLoadTaskInfoList.stream().filter(entity -> entity.isRunning()).count())); + result.put("waiting_task", + String.valueOf(routineLoadTaskInfoList.stream().filter(entity -> !entity.isRunning()).count())); + Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + return gson.toJson(result); + } + + private String jobPropertiesToJsonString() { + Map jobProperties = Maps.newHashMap(); + jobProperties.put("partitions", partitions == null ? STAR_STRING : Joiner.on(",").join(partitions)); + jobProperties.put("columnToColumnExpr", columnDescs == null ? STAR_STRING : Joiner.on(",").join(columnDescs)); + jobProperties.put("whereExpr", whereExpr == null ? STAR_STRING : whereExpr.toSql()); + jobProperties.put("columnSeparator", columnSeparator == null ? "\t" : columnSeparator.toString()); + jobProperties.put("maxErrorNum", String.valueOf(maxErrorNum)); + jobProperties.put("maxBatchIntervalS", String.valueOf(maxBatchIntervalS)); + jobProperties.put("maxBatchRows", String.valueOf(maxBatchRows)); + jobProperties.put("maxBatchSizeBytes", String.valueOf(maxBatchSizeBytes)); + jobProperties.put("currentTaskConcurrentNum", String.valueOf(currentTaskConcurrentNum)); + Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + return gson.toJson(jobProperties); } - protected boolean needReschedule() { + abstract String dataSourcePropertiesJsonToString(); + + + public boolean needRemove() { + if (!isFinal()) { + return false; + } + Preconditions.checkState(endTimestamp != -1, endTimestamp); + if ((System.currentTimeMillis() - endTimestamp) > Config.label_clean_interval_second * 1000) { + return true; + } return false; } + + public boolean isFinal() { + return state.isFinalState(); + } + + public static RoutineLoadJob read(DataInput in) throws IOException { + RoutineLoadJob job = null; + LoadDataSourceType type = LoadDataSourceType.valueOf(Text.readString(in)); + if (type == LoadDataSourceType.KAFKA) { + job = new KafkaRoutineLoadJob(); + } else { + throw new IOException("Unknown load data source type: " + type.name()); + } + + job.setTypeRead(true); + job.readFields(in); + return job; + } + + @Override + public void write(DataOutput out) throws IOException { + // ATTN: must write type first + Text.writeString(out, dataSourceType.name()); + + out.writeLong(id); + Text.writeString(out, name); + Text.writeString(out, clusterName); + out.writeLong(dbId); + out.writeLong(tableId); + out.writeInt(desireTaskConcurrentNum); + Text.writeString(out, state.name()); + out.writeLong(maxErrorNum); + out.writeLong(maxBatchIntervalS); + out.writeLong(maxBatchRows); + out.writeLong(maxBatchSizeBytes); + progress.write(out); + + out.writeLong(createTimestamp); + out.writeLong(pauseTimestamp); + out.writeLong(endTimestamp); + + out.writeLong(currentErrorRows); + out.writeLong(currentTotalRows); + out.writeLong(errorRows); + out.writeLong(totalRows); + out.writeLong(unselectedRows); + out.writeLong(receivedBytes); + out.writeLong(totalTaskExcutionTimeMs); + out.writeLong(committedTaskNum); + out.writeLong(abortedTaskNum); + + Text.writeString(out, origStmt); + } + + @Override + public void readFields(DataInput in) throws IOException { + if (!isTypeRead) { + dataSourceType = LoadDataSourceType.valueOf(Text.readString(in)); + isTypeRead = true; + } + + id = in.readLong(); + name = Text.readString(in); + clusterName = Text.readString(in); + dbId = in.readLong(); + tableId = in.readLong(); + desireTaskConcurrentNum = in.readInt(); + state = JobState.valueOf(Text.readString(in)); + maxErrorNum = in.readLong(); + maxBatchIntervalS = in.readLong(); + maxBatchRows = in.readLong(); + maxBatchSizeBytes = in.readLong(); + + switch (dataSourceType) { + case KAFKA: { + progress = new KafkaProgress(); + progress.readFields(in); + break; + } + default: + throw new IOException("unknown data source type: " + dataSourceType); + } + + createTimestamp = in.readLong(); + pauseTimestamp = in.readLong(); + endTimestamp = in.readLong(); + + currentErrorRows = in.readLong(); + currentTotalRows = in.readLong(); + errorRows = in.readLong(); + totalRows = in.readLong(); + unselectedRows = in.readLong(); + receivedBytes = in.readLong(); + totalTaskExcutionTimeMs = in.readLong(); + committedTaskNum = in.readLong(); + abortedTaskNum = in.readLong(); + + origStmt = Text.readString(in); + + // parse the origin stmt to get routine load desc + SqlParser parser = new SqlParser(new SqlScanner(new StringReader(origStmt))); + CreateRoutineLoadStmt stmt = null; + try { + stmt = (CreateRoutineLoadStmt) parser.parse().value; + stmt.checkLoadProperties(null); + setRoutineLoadDesc(stmt.getRoutineLoadDesc()); + } catch (Exception e) { + throw new IOException("error happens when parsing create routine load stmt: " + origStmt, e); + } + } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java index 4d0400fd2104c5..cd2c490a1b7719 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java @@ -17,50 +17,55 @@ package org.apache.doris.load.routineload; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Queues; import org.apache.doris.analysis.CreateRoutineLoadStmt; import org.apache.doris.analysis.PauseRoutineLoadStmt; import org.apache.doris.analysis.ResumeRoutineLoadStmt; import org.apache.doris.analysis.StopRoutineLoadStmt; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; -import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; +import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.persist.RoutineLoadOperation; import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Queue; +import java.util.UUID; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; -public class RoutineLoadManager { +public class RoutineLoadManager implements Writable { private static final Logger LOG = LogManager.getLogger(RoutineLoadManager.class); - private static final int DEFAULT_BE_CONCURRENT_TASK_NUM = 100; + private static final int DEFAULT_BE_CONCURRENT_TASK_NUM = 10; // Long is beId, integer is the size of tasks in be - private Map beIdToMaxConcurrentTasks; - private Map beIdToConcurrentTasks; + private Map beIdToMaxConcurrentTasks = Maps.newHashMap(); // stream load job meta - private Map idToRoutineLoadJob; - private Map>> dbToNameToRoutineLoadJob; - - private Queue needScheduleTasksQueue; + private Map idToRoutineLoadJob = Maps.newConcurrentMap(); + private Map>> dbToNameToRoutineLoadJob = Maps.newConcurrentMap(); - private ReentrantReadWriteLock lock; + private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); private void readLock() { lock.readLock().lock(); @@ -79,21 +84,11 @@ private void writeUnlock() { } public RoutineLoadManager() { - idToRoutineLoadJob = Maps.newConcurrentMap(); - dbToNameToRoutineLoadJob = Maps.newConcurrentMap(); - beIdToConcurrentTasks = Maps.newHashMap(); - beIdToMaxConcurrentTasks = Maps.newHashMap(); - needScheduleTasksQueue = Queues.newLinkedBlockingQueue(); - lock = new ReentrantReadWriteLock(true); - } - - public Queue getNeedScheduleTasksQueue() { - return needScheduleTasksQueue; } - private void updateBeIdToMaxConcurrentTasks() { + public void updateBeIdToMaxConcurrentTasks() { beIdToMaxConcurrentTasks = Catalog.getCurrentSystemInfo().getBackendIds(true) - .parallelStream().collect(Collectors.toMap(beId -> beId, beId -> DEFAULT_BE_CONCURRENT_TASK_NUM)); + .stream().collect(Collectors.toMap(beId -> beId, beId -> DEFAULT_BE_CONCURRENT_TASK_NUM)); } // this is not real-time number @@ -101,58 +96,36 @@ public int getTotalMaxConcurrentTaskNum() { return beIdToMaxConcurrentTasks.values().stream().mapToInt(i -> i).sum(); } - public void updateBeIdTaskMaps() { - writeLock(); - try { - updateBeIdToMaxConcurrentTasks(); - List beIds = Catalog.getCurrentSystemInfo().getBackendIds(true); - - // diff beIds and beIdToMaxConcurrentTasks.keys() - List newBeIds = beIds.parallelStream().filter(entity -> beIdToMaxConcurrentTasks.get(entity) == null) - .collect(Collectors.toList()); - List unavailableBeIds = beIdToMaxConcurrentTasks.keySet().parallelStream() - .filter(entity -> !beIds.contains(entity)) - .collect(Collectors.toList()); - newBeIds.parallelStream().forEach(entity -> beIdToMaxConcurrentTasks.put(entity, DEFAULT_BE_CONCURRENT_TASK_NUM)); - for (long beId : unavailableBeIds) { - beIdToMaxConcurrentTasks.remove(beId); - beIdToConcurrentTasks.remove(beId); + private Map getBeIdConcurrentTaskMaps() { + Map beIdToConcurrentTasks = Maps.newHashMap(); + for (RoutineLoadJob routineLoadJob : getRoutineLoadJobByState(RoutineLoadJob.JobState.RUNNING)) { + Map jobBeIdToConcurrentTaskNum = routineLoadJob.getBeIdToConcurrentTaskNum(); + for (Map.Entry entry : jobBeIdToConcurrentTaskNum.entrySet()) { + if (beIdToConcurrentTasks.containsKey(entry.getKey())) { + beIdToConcurrentTasks.put(entry.getKey(), beIdToConcurrentTasks.get(entry.getKey()) + entry.getValue()); + } else { + beIdToConcurrentTasks.put(entry.getKey(), entry.getValue()); + } } - LOG.info("There are {} backends which participate in routine load scheduler. " - + "There are {} new backends and {} unavailable backends for routine load", - beIdToMaxConcurrentTasks.size(), newBeIds.size(), unavailableBeIds.size()); - } finally { - writeUnlock(); } - } + return beIdToConcurrentTasks; - public void addNumOfConcurrentTasksByBeId(long beId) { - writeLock(); - try { - if (beIdToConcurrentTasks.get(beId) == null) { - beIdToConcurrentTasks.put(beId, 1); - } else { - int concurrentTaskNum = (int) beIdToConcurrentTasks.get(beId); - concurrentTaskNum++; - beIdToConcurrentTasks.put(beId, concurrentTaskNum); - } - } finally { - writeUnlock(); - } } - public void addRoutineLoadJob(CreateRoutineLoadStmt createRoutineLoadStmt) - throws AnalysisException, DdlException, LoadException { + public void createRoutineLoadJob(CreateRoutineLoadStmt createRoutineLoadStmt, String origStmt) + throws UserException { // check load auth if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), - createRoutineLoadStmt.getDBTableName().getDb(), - createRoutineLoadStmt.getDBTableName().getTbl(), + createRoutineLoadStmt.getDBName(), + createRoutineLoadStmt.getTableName(), PrivPredicate.LOAD)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - createRoutineLoadStmt.getDBTableName()); + createRoutineLoadStmt.getDBName(), + createRoutineLoadStmt.getTableName()); } + RoutineLoadJob routineLoadJob = null; LoadDataSourceType type = LoadDataSourceType.valueOf(createRoutineLoadStmt.getTypeName()); switch (type) { @@ -160,28 +133,47 @@ public void addRoutineLoadJob(CreateRoutineLoadStmt createRoutineLoadStmt) routineLoadJob = KafkaRoutineLoadJob.fromCreateStmt(createRoutineLoadStmt); break; default: - break; - } - if (routineLoadJob != null) { - addRoutineLoadJob(routineLoadJob); + throw new UserException("Unknown data source type: " + type); } + + routineLoadJob.setOrigStmt(origStmt); + addRoutineLoadJob(routineLoadJob, createRoutineLoadStmt.getDBName()); } - public void addRoutineLoadJob(RoutineLoadJob routineLoadJob) throws DdlException { + public void addRoutineLoadJob(RoutineLoadJob routineLoadJob, String dbName) throws DdlException { writeLock(); try { // check if db.routineLoadName has been used - if (isNameUsed(routineLoadJob.dbId, routineLoadJob.getName())) { + if (isNameUsed(routineLoadJob.getDbId(), routineLoadJob.getName())) { throw new DdlException("Name " + routineLoadJob.getName() + " already used in db " - + routineLoadJob.getDbId()); + + dbName); } - idToRoutineLoadJob.put(routineLoadJob.getId(), routineLoadJob); - addJobToDbToNameToRoutineLoadJob(routineLoadJob); - // TODO(ml): edit log + + unprotectedAddJob(routineLoadJob); + + Catalog.getInstance().getEditLog().logCreateRoutineLoadJob(routineLoadJob); + LOG.info("create routine load job: id: {}, name: {}", routineLoadJob.getId(), routineLoadJob.getName()); } finally { writeUnlock(); } + } + private void unprotectedAddJob(RoutineLoadJob routineLoadJob) { + idToRoutineLoadJob.put(routineLoadJob.getId(), routineLoadJob); + + Map> nameToRoutineLoadJob = dbToNameToRoutineLoadJob.get(routineLoadJob.getDbId()); + if (nameToRoutineLoadJob == null) { + nameToRoutineLoadJob = Maps.newConcurrentMap(); + dbToNameToRoutineLoadJob.put(routineLoadJob.getDbId(), nameToRoutineLoadJob); + } + List routineLoadJobList = nameToRoutineLoadJob.get(routineLoadJob.getName()); + if (routineLoadJobList == null) { + routineLoadJobList = Lists.newArrayList(); + nameToRoutineLoadJob.put(routineLoadJob.getName(), routineLoadJobList); + } + routineLoadJobList.add(routineLoadJob); + // add txn state callback in factory + Catalog.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(routineLoadJob); } // TODO(ml): Idempotency @@ -201,80 +193,104 @@ private boolean isNameUsed(Long dbId, String name) { return false; } - private void addJobToDbToNameToRoutineLoadJob(RoutineLoadJob routineLoadJob) { - if (dbToNameToRoutineLoadJob.containsKey(routineLoadJob.getDbId())) { - Map> nameToRoutineLoadJob = - dbToNameToRoutineLoadJob.get(routineLoadJob.getDbId()); - if (nameToRoutineLoadJob.containsKey(routineLoadJob.getName())) { - nameToRoutineLoadJob.get(routineLoadJob.getName()).add(routineLoadJob); - } else { - List routineLoadJobList = Lists.newArrayList(); - routineLoadJobList.add(routineLoadJob); - nameToRoutineLoadJob.put(routineLoadJob.getName(), routineLoadJobList); - } - } else { - List routineLoadJobList = Lists.newArrayList(); - routineLoadJobList.add(routineLoadJob); - Map> nameToRoutineLoadJob = Maps.newConcurrentMap(); - nameToRoutineLoadJob.put(routineLoadJob.getName(), routineLoadJobList); - dbToNameToRoutineLoadJob.put(routineLoadJob.getDbId(), nameToRoutineLoadJob); - } - } - - public void pauseRoutineLoadJob(PauseRoutineLoadStmt pauseRoutineLoadStmt) throws DdlException, AnalysisException { - RoutineLoadJob routineLoadJob = getJobByName(pauseRoutineLoadStmt.getName()); + public void pauseRoutineLoadJob(PauseRoutineLoadStmt pauseRoutineLoadStmt) + throws UserException { + RoutineLoadJob routineLoadJob = getJob(pauseRoutineLoadStmt.getDbFullName(), pauseRoutineLoadStmt.getName()); if (routineLoadJob == null) { - throw new DdlException("There is not routine load job with name " + pauseRoutineLoadStmt.getName()); + throw new DdlException("There is not operable routine load job with name " + pauseRoutineLoadStmt.getName()); } // check auth + String dbFullName; + String tableName; + try { + dbFullName = routineLoadJob.getDbFullName(); + tableName = routineLoadJob.getTableName(); + } catch (MetaNotFoundException e) { + throw new DdlException("The metadata of job has been changed. The job will be cancelled automatically", e); + } if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), - routineLoadJob.getDbFullName(), - routineLoadJob.getTableName(), + dbFullName, + tableName, PrivPredicate.LOAD)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - routineLoadJob.getTableName()); + tableName); } - routineLoadJob.pause("User " + ConnectContext.get().getQualifiedUser() + "pauses routine load job"); + routineLoadJob.updateState(RoutineLoadJob.JobState.PAUSED, + "User " + ConnectContext.get().getQualifiedUser() + " pauses routine load job", + false /* not replay */); + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("current_state", routineLoadJob.getState()) + .add("user", ConnectContext.get().getQualifiedUser()) + .add("msg", "routine load job has been paused by user") + .build()); } - public void resumeRoutineLoadJob(ResumeRoutineLoadStmt resumeRoutineLoadStmt) throws DdlException, - AnalysisException { - RoutineLoadJob routineLoadJob = getJobByName(resumeRoutineLoadStmt.getName()); + public void resumeRoutineLoadJob(ResumeRoutineLoadStmt resumeRoutineLoadStmt) throws UserException { + RoutineLoadJob routineLoadJob = getJob(resumeRoutineLoadStmt.getDbFullName(), resumeRoutineLoadStmt.getName()); if (routineLoadJob == null) { - throw new DdlException("There is not routine load job with name " + resumeRoutineLoadStmt.getName()); + throw new DdlException("There is not operable routine load job with name " + resumeRoutineLoadStmt.getName() + "."); } // check auth + String dbFullName; + String tableName; + try { + dbFullName = routineLoadJob.getDbFullName(); + tableName = routineLoadJob.getTableName(); + } catch (MetaNotFoundException e) { + throw new DdlException("The metadata of job has been changed. The job will be cancelled automatically", e); + } if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), - routineLoadJob.getDbFullName(), - routineLoadJob.getTableName(), + dbFullName, + tableName, PrivPredicate.LOAD)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - routineLoadJob.getTableName()); + tableName); } - routineLoadJob.resume(); + routineLoadJob.updateState(RoutineLoadJob.JobState.NEED_SCHEDULE, null, false /* not replay */); + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("current_state", routineLoadJob.getState()) + .add("user", ConnectContext.get().getQualifiedUser()) + .add("msg", "routine load job has been resumed by user") + .build()); } - public void stopRoutineLoadJob(StopRoutineLoadStmt stopRoutineLoadStmt) throws DdlException, AnalysisException { - RoutineLoadJob routineLoadJob = getJobByName(stopRoutineLoadStmt.getName()); + public void stopRoutineLoadJob(StopRoutineLoadStmt stopRoutineLoadStmt) + throws UserException { + RoutineLoadJob routineLoadJob = getJob(stopRoutineLoadStmt.getDbFullName(), stopRoutineLoadStmt.getName()); if (routineLoadJob == null) { - throw new DdlException("There is not routine load job with name " + stopRoutineLoadStmt.getName()); + throw new DdlException("There is not operable routine load job with name " + stopRoutineLoadStmt.getName()); } // check auth + String dbFullName; + String tableName; + try { + dbFullName = routineLoadJob.getDbFullName(); + tableName = routineLoadJob.getTableName(); + } catch (MetaNotFoundException e) { + throw new DdlException("The metadata of job has been changed. The job will be cancelled automatically", e); + } if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), - routineLoadJob.getDbFullName(), - routineLoadJob.getTableName(), + dbFullName, + tableName, PrivPredicate.LOAD)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - routineLoadJob.getTableName()); + tableName); } - routineLoadJob.stop(); + routineLoadJob.updateState(RoutineLoadJob.JobState.STOPPED, + "User " + ConnectContext.get().getQualifiedUser() + " stop routine load job", + false /* not replay */); + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("current_state", routineLoadJob.getState()) + .add("user", ConnectContext.get().getQualifiedUser()) + .add("msg", "routine load job has been stopped by user") + .build()); } public int getSizeOfIdToRoutineLoadTask() { @@ -290,11 +306,12 @@ public int getClusterIdleSlotNum() { try { int result = 0; updateBeIdToMaxConcurrentTasks(); + Map beIdToConcurrentTasks = getBeIdConcurrentTaskMaps(); for (Map.Entry entry : beIdToMaxConcurrentTasks.entrySet()) { - if (beIdToConcurrentTasks.get(entry.getKey()) == null) { - result += entry.getValue(); - } else { + if (beIdToConcurrentTasks.containsKey(entry.getKey())) { result += entry.getValue() - beIdToConcurrentTasks.get(entry.getKey()); + } else { + result += entry.getValue(); } } return result; @@ -303,20 +320,32 @@ public int getClusterIdleSlotNum() { } } - public long getMinTaskBeId() throws LoadException { + public long getMinTaskBeId(String clusterName) throws LoadException { + List beIdsInCluster = Catalog.getCurrentSystemInfo().getClusterBackendIds(clusterName, true); + if (beIdsInCluster == null) { + throw new LoadException("The " + clusterName + " has been deleted"); + } + readLock(); try { long result = -1L; int maxIdleSlotNum = 0; updateBeIdToMaxConcurrentTasks(); - for (Map.Entry entry : beIdToMaxConcurrentTasks.entrySet()) { - if (beIdToConcurrentTasks.get(entry.getKey()) == null) { - result = maxIdleSlotNum < entry.getValue() ? entry.getKey() : result; - maxIdleSlotNum = Math.max(maxIdleSlotNum, entry.getValue()); - } else { - int idelTaskNum = entry.getValue() - beIdToConcurrentTasks.get(entry.getKey()); - result = maxIdleSlotNum < idelTaskNum ? entry.getKey() : result; - maxIdleSlotNum = Math.max(maxIdleSlotNum, idelTaskNum); + Map beIdToConcurrentTasks = getBeIdConcurrentTaskMaps(); + for (Long beId : beIdsInCluster) { + if (beIdToMaxConcurrentTasks.containsKey(beId)) { + int idleTaskNum = 0; + if (beIdToConcurrentTasks.containsKey(beId)) { + idleTaskNum = beIdToMaxConcurrentTasks.get(beId) - beIdToConcurrentTasks.get(beId); + } else { + idleTaskNum = DEFAULT_BE_CONCURRENT_TASK_NUM; + } + if (LOG.isDebugEnabled()) { + LOG.debug("be {} has idle {}, concurrent task {}, max concurrent task {}", beId, idleTaskNum, + beIdToConcurrentTasks.get(beId), beIdToMaxConcurrentTasks.get(beId)); + } + result = maxIdleSlotNum < idleTaskNum ? beId : result; + maxIdleSlotNum = Math.max(maxIdleSlotNum, idleTaskNum); } } if (result < 0) { @@ -328,78 +357,264 @@ public long getMinTaskBeId() throws LoadException { } } - public RoutineLoadJob getJob(String jobId) { - return idToRoutineLoadJob.get(jobId); - } + public boolean checkBeToTask(long beId, String clusterName) throws LoadException { + List beIdsInCluster = Catalog.getCurrentSystemInfo().getClusterBackendIds(clusterName, true); + if (beIdsInCluster == null) { + throw new LoadException("The " + clusterName + " has been deleted"); + } - public RoutineLoadJob getJobByName(String jobName) { - String dbfullName = ConnectContext.get().getDatabase(); - Database database = Catalog.getCurrentCatalog().getDb(dbfullName); - if (database == null) { - return null; + if (!beIdsInCluster.contains(beId)) { + return false; } + + // check if be has idle slot readLock(); try { - Map> nameToRoutineLoadJob = dbToNameToRoutineLoadJob.get(database.getId()); - if (nameToRoutineLoadJob == null) { - return null; - } - List routineLoadJobList = nameToRoutineLoadJob.get(jobName); - if (routineLoadJobList == null) { - return null; + int idleTaskNum = 0; + Map beIdToConcurrentTasks = getBeIdConcurrentTaskMaps(); + if (beIdToConcurrentTasks.containsKey(beId)) { + idleTaskNum = beIdToMaxConcurrentTasks.get(beId) - beIdToConcurrentTasks.get(beId); + } else { + idleTaskNum = DEFAULT_BE_CONCURRENT_TASK_NUM; } - Optional optional = routineLoadJobList.parallelStream() - .filter(entity -> !entity.getState().isFinalState()).findFirst(); - if (optional.isPresent()) { - return null; + if (idleTaskNum > 0) { + return true; } - return optional.get(); + return false; } finally { readUnlock(); } } - public RoutineLoadJob getJobByTaskId(String taskId) throws MetaNotFoundException { + public RoutineLoadJob getJob(long jobId) { + return idToRoutineLoadJob.get(jobId); + } + + public RoutineLoadJob getJob(String dbFullName, String jobName) throws MetaNotFoundException { + List routineLoadJobList = getJob(dbFullName, jobName, false); + if (routineLoadJobList == null || routineLoadJobList.size() == 0) { + return null; + } else { + return routineLoadJobList.get(0); + } + } + + /* + if dbFullName is null, result = all of routine load job in all of db + else if jobName is null, result = all of routine load job in dbFullName + + if includeHistory is false, filter not running job in result + else return all of result + */ + public List getJob(String dbFullName, String jobName, boolean includeHistory) + throws MetaNotFoundException { + // return all of routine load job + List result; + RESULT: + { + if (dbFullName == null) { + result = new ArrayList<>(idToRoutineLoadJob.values()); + sortRoutineLoadJob(result); + break RESULT; + } + + long dbId = 0L; + Database database = Catalog.getCurrentCatalog().getDb(dbFullName); + if (database == null) { + throw new MetaNotFoundException("failed to find database by dbFullName " + dbFullName); + } + dbId = database.getId(); + if (!dbToNameToRoutineLoadJob.containsKey(dbId)) { + result = new ArrayList<>(); + break RESULT; + } + if (jobName == null) { + result = Lists.newArrayList(); + for (List nameToRoutineLoadJob : dbToNameToRoutineLoadJob.get(dbId).values()) { + List routineLoadJobList = new ArrayList<>(nameToRoutineLoadJob); + sortRoutineLoadJob(routineLoadJobList); + result.addAll(routineLoadJobList); + } + break RESULT; + } + if (dbToNameToRoutineLoadJob.get(dbId).containsKey(jobName)) { + result = new ArrayList<>(dbToNameToRoutineLoadJob.get(dbId).get(jobName)); + sortRoutineLoadJob(result); + break RESULT; + } + return null; + } + + if (!includeHistory) { + result = result.stream().filter(entity -> !entity.getState().isFinalState()).collect(Collectors.toList()); + } + return result; + } + + // return all of routine load job named jobName in all of db + public List getJobByName(String jobName) { + List result = Lists.newArrayList(); + for (Map> nameToRoutineLoadJob : dbToNameToRoutineLoadJob.values()) { + if (nameToRoutineLoadJob.containsKey(jobName)) { + List routineLoadJobList = new ArrayList<>(nameToRoutineLoadJob.get(jobName)); + sortRoutineLoadJob(routineLoadJobList); + result.addAll(routineLoadJobList); + } + } + return result; + } + + // put history job in the end + private void sortRoutineLoadJob(List routineLoadJobList) { + if (routineLoadJobList == null) { + return; + } + int i = 0; + int j = routineLoadJobList.size() - 1; + while (i < j) { + while (!routineLoadJobList.get(i).isFinal() && (i < j)) + i++; + while (routineLoadJobList.get(j).isFinal() && (i < j)) + j--; + if (i < j) { + RoutineLoadJob routineLoadJob = routineLoadJobList.get(i); + routineLoadJobList.set(i, routineLoadJobList.get(j)); + routineLoadJobList.set(j, routineLoadJob); + } + } + } + + public boolean checkTaskInJob(UUID taskId) { for (RoutineLoadJob routineLoadJob : idToRoutineLoadJob.values()) { if (routineLoadJob.containsTask(taskId)) { - return routineLoadJob; + return true; } } - throw new MetaNotFoundException("could not found task by id " + taskId); + return false; } - public List getRoutineLoadJobByState(RoutineLoadJob.JobState jobState) throws LoadException { - List jobs = new ArrayList<>(); - Collection stateJobs = null; - LOG.debug("begin to get routine load job by state {}", jobState.name()); - stateJobs = idToRoutineLoadJob.values().stream() + public List getRoutineLoadJobByState(RoutineLoadJob.JobState jobState) { + List stateJobs = idToRoutineLoadJob.values().stream() .filter(entity -> entity.getState() == jobState).collect(Collectors.toList()); - if (stateJobs != null) { - jobs.addAll(stateJobs); - LOG.info("got {} routine load jobs by state {}", jobs.size(), jobState.name()); - } - return jobs; + return stateJobs; } - public List processTimeoutTasks() { - List routineLoadTaskInfoList = new ArrayList<>(); + public void processTimeoutTasks() { for (RoutineLoadJob routineLoadJob : idToRoutineLoadJob.values()) { - routineLoadTaskInfoList.addAll(routineLoadJob.processTimeoutTasks()); + routineLoadJob.processTimeoutTasks(); } - return routineLoadTaskInfoList; } // Remove old routine load jobs from idToRoutineLoadJob // This function is called periodically. // Cancelled and stopped job will be remove after Configure.label_keep_max_second seconds - public void removeOldRoutineLoadJobs() { - // TODO(ml): remove old routine load job + public void cleanOldRoutineLoadJobs() { + writeLock(); + try { + Iterator> iterator = idToRoutineLoadJob.entrySet().iterator(); + long currentTimestamp = System.currentTimeMillis(); + while (iterator.hasNext()) { + RoutineLoadJob routineLoadJob = iterator.next().getValue(); + if (routineLoadJob.needRemove()) { + unprotectedRemoveJobFromDb(routineLoadJob); + iterator.remove(); + + RoutineLoadOperation operation = new RoutineLoadOperation(routineLoadJob.getId(), + routineLoadJob.getState()); + Catalog.getInstance().getEditLog().logRemoveRoutineLoadJob(operation); + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("end_timestamp", routineLoadJob.getEndTimestamp()) + .add("current_timestamp", currentTimestamp) + .add("job_state", routineLoadJob.getState()) + .add("msg", "old job has been cleaned") + ); + } + } + } finally { + writeUnlock(); + } } - public void rescheduleRoutineLoadJob() { + public void replayRemoveOldRoutineLoad(RoutineLoadOperation operation) { + writeLock(); + try { + RoutineLoadJob job = idToRoutineLoadJob.remove(operation.getId()); + if (job != null) { + unprotectedRemoveJobFromDb(job); + } + LOG.info("replay remove routine load job: {}", operation.getId()); + } finally { + writeUnlock(); + } + } + + private void unprotectedRemoveJobFromDb(RoutineLoadJob routineLoadJob) { + dbToNameToRoutineLoadJob.get(routineLoadJob.getDbId()).get(routineLoadJob.getName()).remove(routineLoadJob); + if (dbToNameToRoutineLoadJob.get(routineLoadJob.getDbId()).get(routineLoadJob.getName()).isEmpty()) { + dbToNameToRoutineLoadJob.get(routineLoadJob.getDbId()).remove(routineLoadJob.getName()); + } + if (dbToNameToRoutineLoadJob.get(routineLoadJob.getDbId()).isEmpty()) { + dbToNameToRoutineLoadJob.remove(routineLoadJob.getDbId()); + } + } + + public void updateRoutineLoadJob() throws UserException { for (RoutineLoadJob routineLoadJob : idToRoutineLoadJob.values()) { - routineLoadJob.reschedule(); + if (!routineLoadJob.state.isFinalState()) { + routineLoadJob.update(); + } } } + public void replayCreateRoutineLoadJob(RoutineLoadJob routineLoadJob) { + unprotectedAddJob(routineLoadJob); + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("msg", "replay create routine load job") + .build()); + } + + public void replayChangeRoutineLoadJob(RoutineLoadOperation operation) { + RoutineLoadJob job = getJob(operation.getId()); + try { + job.updateState(operation.getJobState(), null, true /* is replay */); + } catch (UserException e) { + LOG.error("should not happend", e); + } + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, operation.getId()) + .add("current_state", operation.getJobState()) + .add("msg", "replay change routine load job") + .build()); + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(idToRoutineLoadJob.size()); + for (RoutineLoadJob routineLoadJob : idToRoutineLoadJob.values()) { + routineLoadJob.write(out); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + int size = in.readInt(); + for (int i = 0; i < size; i++) { + RoutineLoadJob routineLoadJob = RoutineLoadJob.read(in); + idToRoutineLoadJob.put(routineLoadJob.getId(), routineLoadJob); + Map> map = dbToNameToRoutineLoadJob.get(routineLoadJob.getDbId()); + if (map == null) { + map = Maps.newConcurrentMap(); + dbToNameToRoutineLoadJob.put(routineLoadJob.getDbId(), map); + } + + List jobs = map.get(routineLoadJob.getName()); + if (jobs == null) { + jobs = Lists.newArrayList(); + map.put(routineLoadJob.getName(), jobs); + } + jobs.add(routineLoadJob); + if (!routineLoadJob.getState().isFinalState()) { + Catalog.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(routineLoadJob); + } + } + } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java index eb3b593c258d47..344fdc1569be28 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java @@ -17,9 +17,55 @@ package org.apache.doris.load.routineload; +import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + public abstract class RoutineLoadProgress implements Writable { + protected LoadDataSourceType loadDataSourceType; + protected boolean isTypeRead = false; + + public void setTypeRead(boolean isTypeRead) { + this.isTypeRead = isTypeRead; + } + + public RoutineLoadProgress(LoadDataSourceType loadDataSourceType) { + this.loadDataSourceType = loadDataSourceType; + } + abstract void update(RoutineLoadProgress progress); + + abstract String toJsonString(); + + public static RoutineLoadProgress read(DataInput in) throws IOException { + RoutineLoadProgress progress = null; + LoadDataSourceType type = LoadDataSourceType.valueOf(Text.readString(in)); + if (type == LoadDataSourceType.KAFKA) { + progress = new KafkaProgress(); + } else { + throw new IOException("Unknown load data source type: " + type.name()); + } + + progress.setTypeRead(true); + progress.readFields(in); + return progress; + } + + @Override + public void write(DataOutput out) throws IOException { + // ATTN: must write type first + Text.writeString(out, loadDataSourceType.name()); + } + + @Override + public void readFields(DataInput in) throws IOException { + if (!isTypeRead) { + loadDataSourceType = LoadDataSourceType.valueOf(Text.readString(in)); + isTypeRead = true; + } + } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java index 0757376f58cb78..06798be4039294 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java @@ -20,7 +20,12 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.Daemon; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; + +import com.google.common.annotations.VisibleForTesting; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -30,58 +35,104 @@ public class RoutineLoadScheduler extends Daemon { private static final Logger LOG = LogManager.getLogger(RoutineLoadScheduler.class); + private static final int DEFAULT_INTERVAL_SECONDS = 10; + + private RoutineLoadManager routineLoadManager; - private RoutineLoadManager routineLoadManager = Catalog.getInstance().getRoutineLoadManager(); + @VisibleForTesting + public RoutineLoadScheduler() { + super(); + routineLoadManager = Catalog.getInstance().getRoutineLoadManager(); + } + + public RoutineLoadScheduler(RoutineLoadManager routineLoadManager) { + super("Routine load", DEFAULT_INTERVAL_SECONDS * 1000); + this.routineLoadManager = routineLoadManager; + } @Override protected void runOneCycle() { try { process(); } catch (Throwable e) { - LOG.error("failed to schedule jobs with error massage {}", e.getMessage(), e); + LOG.warn("Failed to process one round of RoutineLoadScheduler", e); } } - private void process() { + private void process() throws UserException { // update - routineLoadManager.rescheduleRoutineLoadJob(); + routineLoadManager.updateRoutineLoadJob(); // get need schedule routine jobs List routineLoadJobList = null; try { routineLoadJobList = getNeedScheduleRoutineJobs(); } catch (LoadException e) { - LOG.error("failed to get need schedule routine jobs"); + LOG.warn("failed to get need schedule routine jobs", e); } - LOG.debug("there are {} job need schedule", routineLoadJobList.size()); + LOG.info("there are {} job need schedule", routineLoadJobList.size()); for (RoutineLoadJob routineLoadJob : routineLoadJobList) { + RoutineLoadJob.JobState errorJobState = null; + UserException userException = null; try { + // init the stream load planner + routineLoadJob.initPlanner(); // judge nums of tasks more then max concurrent tasks of cluster - int currentConcurrentTaskNum = routineLoadJob.calculateCurrentConcurrentTaskNum(); - int totalTaskNum = currentConcurrentTaskNum + routineLoadManager.getSizeOfIdToRoutineLoadTask(); - if (totalTaskNum > routineLoadManager.getTotalMaxConcurrentTaskNum()) { - LOG.info("job {} concurrent task num {}, current total task num {}. " - + "desired total task num {} more then total max task num {}, " - + "skip this turn of job scheduler", - routineLoadJob.getId(), currentConcurrentTaskNum, - routineLoadManager.getSizeOfIdToRoutineLoadTask(), - totalTaskNum, routineLoadManager.getTotalMaxConcurrentTaskNum()); + int desiredConcurrentTaskNum = routineLoadJob.calculateCurrentConcurrentTaskNum(); + if (desiredConcurrentTaskNum <= 0) { + // the job will be rescheduled later. + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("msg", "the current concurrent num is less then or equal to zero, " + + "job will be rescheduled later") + .build()); + continue; + } + int currentTotalTaskNum = routineLoadManager.getSizeOfIdToRoutineLoadTask(); + int desiredTotalTaskNum = desiredConcurrentTaskNum + currentTotalTaskNum; + if (desiredTotalTaskNum > routineLoadManager.getTotalMaxConcurrentTaskNum()) { + LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("desired_concurrent_task_num", desiredConcurrentTaskNum) + .add("current_total_task_num", currentTotalTaskNum) + .add("desired_total_task_num", desiredTotalTaskNum) + .add("total_max_task_num", routineLoadManager.getTotalMaxConcurrentTaskNum()) + .add("msg", "skip this turn of job scheduler while there are not enough slot in backends") + .build()); break; } - // divide job into tasks - List needScheduleTasksList = - routineLoadJob.divideRoutineLoadJob(currentConcurrentTaskNum); - // save task into queue of needScheduleTasks - routineLoadManager.getNeedScheduleTasksQueue().addAll(needScheduleTasksList); + // check state and divide job into tasks + routineLoadJob.divideRoutineLoadJob(desiredConcurrentTaskNum); } catch (MetaNotFoundException e) { - routineLoadJob.updateState(RoutineLoadJob.JobState.CANCELLED); + errorJobState = RoutineLoadJob.JobState.CANCELLED; + userException = e; + } catch (UserException e) { + errorJobState = RoutineLoadJob.JobState.PAUSED; + userException = e; + } + + if (errorJobState != null) { + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("current_state", routineLoadJob.getState()) + .add("desired_state", errorJobState) + .add("warn_msg", "failed to scheduler job, change job state to desired_state with error reason " + userException.getMessage()) + .build(), userException); + try { + routineLoadJob.updateState(errorJobState, userException.getMessage(), false); + } catch (UserException e) { + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("current_state", routineLoadJob.getState()) + .add("desired_state", errorJobState) + .add("warn_msg", "failed to change state to desired state") + .build(), e); + } } } LOG.debug("begin to check timeout tasks"); // check timeout tasks - List rescheduleTasksList = routineLoadManager.processTimeoutTasks(); - routineLoadManager.getNeedScheduleTasksQueue().addAll(rescheduleTasksList); + routineLoadManager.processTimeoutTasks(); + + LOG.debug("begin to clean old jobs "); + routineLoadManager.cleanOldRoutineLoadJobs(); } private List getNeedScheduleRoutineJobs() throws LoadException { diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java index d7259382b891fb..3fa493557ceeed 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java @@ -22,10 +22,19 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.LoadException; -import org.apache.doris.task.RoutineLoadTask; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.service.FrontendOptions; +import org.apache.doris.thrift.TRoutineLoadTask; import org.apache.doris.transaction.BeginTransactionException; import org.apache.doris.transaction.TransactionState; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.UUID; + /** * Routine load task info is the task info include the only id (signature). * For the kafka type of task info, it also include partitions which will be obtained data in this task. @@ -36,51 +45,104 @@ public abstract class RoutineLoadTaskInfo { private RoutineLoadManager routineLoadManager = Catalog.getCurrentCatalog().getRoutineLoadManager(); - protected String id; - protected long txnId; - protected String jobId; + protected UUID id; + protected long txnId = -1L; + protected long jobId; + protected String clusterName; + private long createTimeMs; - private long loadStartTimeMs; - - public RoutineLoadTaskInfo(String id, String jobId) throws BeginTransactionException, - LabelAlreadyUsedException, AnalysisException { + private long executeStartTimeMs = -1L; + // the be id of previous task + protected long previousBeId = -1L; + // the be id of this task + protected long beId = -1L; + + public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName) { this.id = id; this.jobId = jobId; + this.clusterName = clusterName; this.createTimeMs = System.currentTimeMillis(); - // begin a txn for task - RoutineLoadJob routineLoadJob = routineLoadManager.getJob(jobId); - txnId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction( - routineLoadJob.getDbId(), id, -1, "streamLoad", - TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, routineLoadJob); + } + + public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long previousBeId) { + this(id, jobId, clusterName); + this.previousBeId = previousBeId; } - public String getId() { + public UUID getId() { return id; } - public String getJobId() { + public long getJobId() { return jobId; } - public void setLoadStartTimeMs(long loadStartTimeMs) { - this.loadStartTimeMs = loadStartTimeMs; + public String getClusterName() { + return clusterName; } - - public long getLoadStartTimeMs() { - return loadStartTimeMs; + + public void setExecuteStartTimeMs(long executeStartTimeMs) { + this.executeStartTimeMs = executeStartTimeMs; + } + + public long getPreviousBeId() { + return previousBeId; + } + + public void setBeId(long beId) { + this.beId = beId; + } + + public long getBeId() { + return beId; + } + + public long getCreateTimeMs() { + return createTimeMs; + } + + public long getExecuteStartTimeMs() { + return executeStartTimeMs; } public long getTxnId() { return txnId; } - - abstract RoutineLoadTask createStreamLoadTask(long beId) throws LoadException; + + public boolean isRunning() { + return executeStartTimeMs > 0; + } + + abstract TRoutineLoadTask createRoutineLoadTask() throws LoadException, UserException; + + public void beginTxn() throws LabelAlreadyUsedException, BeginTransactionException, AnalysisException { + // begin a txn for task + RoutineLoadJob routineLoadJob = routineLoadManager.getJob(jobId); + txnId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction( + routineLoadJob.getDbId(), DebugUtil.printId(id), -1, "FE: " + FrontendOptions.getLocalHostAddress(), + TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, routineLoadJob.getId(), + routineLoadJob.getMaxBatchIntervalS() * 2); + } + + public List getTaskShowInfo() { + List row = Lists.newArrayList(); + row.add(DebugUtil.printId(id)); + row.add(String.valueOf(txnId)); + row.add(String.valueOf(jobId)); + row.add(String.valueOf(TimeUtils.longToTimeString(createTimeMs))); + row.add(String.valueOf(TimeUtils.longToTimeString(executeStartTimeMs))); + row.add(String.valueOf(beId)); + row.add(getTaskDataSourceProperties()); + return row; + } + + abstract String getTaskDataSourceProperties(); @Override public boolean equals(Object obj) { if (obj instanceof RoutineLoadTaskInfo) { RoutineLoadTaskInfo routineLoadTaskInfo = (RoutineLoadTaskInfo) obj; - return this.id.equals(routineLoadTaskInfo.getId()); + return this.id.toString().equals(routineLoadTaskInfo.getId().toString()); } else { return false; } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java index 76da250fbb190a..ba84b1ddfa453c 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java @@ -18,16 +18,29 @@ package org.apache.doris.load.routineload; import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.ClientPool; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.Daemon; -import org.apache.doris.task.AgentBatchTask; -import org.apache.doris.task.AgentTaskExecutor; -import org.apache.doris.task.AgentTaskQueue; -import org.apache.doris.task.RoutineLoadTask; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; +import org.apache.doris.load.routineload.RoutineLoadJob.JobState; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.BackendService; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TRoutineLoadTask; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Queues; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.List; +import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; /** @@ -43,13 +56,22 @@ public class RoutineLoadTaskScheduler extends Daemon { private static final Logger LOG = LogManager.getLogger(RoutineLoadTaskScheduler.class); + private static final long BACKEND_SLOT_UPDATE_INTERVAL_MS = 10000; // 10s + private RoutineLoadManager routineLoadManager; - private LinkedBlockingQueue needScheduleTasksQueue; + private LinkedBlockingQueue needScheduleTasksQueue = Queues.newLinkedBlockingQueue(); + + private long lastBackendSlotUpdateTime = -1; + @VisibleForTesting public RoutineLoadTaskScheduler() { super("routine load task", 0); - routineLoadManager = Catalog.getInstance().getRoutineLoadManager(); - needScheduleTasksQueue = (LinkedBlockingQueue) routineLoadManager.getNeedScheduleTasksQueue(); + this.routineLoadManager = Catalog.getInstance().getRoutineLoadManager(); + } + + public RoutineLoadTaskScheduler(RoutineLoadManager routineLoadManager) { + super("routine load task", 0); + this.routineLoadManager = routineLoadManager; } @Override @@ -57,58 +79,151 @@ protected void runOneCycle() { try { process(); } catch (Throwable e) { - LOG.warn("Failed to process one round of RoutineLoadTaskScheduler with error message {}", - e.getMessage(), e); + LOG.warn("Failed to process one round of RoutineLoadTaskScheduler", e); } } - private void process() throws LoadException { - // update current beIdMaps for tasks - routineLoadManager.updateBeIdTaskMaps(); + private void process() throws LoadException, UserException, InterruptedException { + updateBackendSlotIfNecessary(); - LOG.info("There are {} need schedule task in queue when {}", - needScheduleTasksQueue.size(), System.currentTimeMillis()); - AgentBatchTask batchTask = new AgentBatchTask(); int sizeOfTasksQueue = needScheduleTasksQueue.size(); int clusterIdleSlotNum = routineLoadManager.getClusterIdleSlotNum(); int needScheduleTaskNum = sizeOfTasksQueue < clusterIdleSlotNum ? sizeOfTasksQueue : clusterIdleSlotNum; + + if (needScheduleTaskNum == 0) { + return; + } + + LOG.info("There are {} tasks need to be scheduled in queue", needScheduleTasksQueue.size()); + int scheduledTaskNum = 0; - // get idle be task num - // allocate task to be - while (needScheduleTaskNum > 0) { + Map> beIdToBatchTask = Maps.newHashMap(); + while (needScheduleTaskNum-- > 0) { + // allocate be to task and begin transaction for task RoutineLoadTaskInfo routineLoadTaskInfo = null; try { routineLoadTaskInfo = needScheduleTasksQueue.take(); } catch (InterruptedException e) { - LOG.warn("Taking routine load task from queue has been interrupted with error msg {}", - e.getMessage()); + LOG.warn("Taking routine load task from queue has been interrupted", e); return; } + try { + if (!routineLoadManager.checkTaskInJob(routineLoadTaskInfo.getId())) { + // task has been abandoned while renew task has been added in queue + // or database has been deleted + LOG.warn(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, routineLoadTaskInfo.getId()) + .add("error_msg", "task has been abandoned") + .build()); + continue; + } + routineLoadTaskInfo.beginTxn(); + allocateTaskToBe(routineLoadTaskInfo); + } catch (LoadException e) { + // todo(ml): if cluster has been deleted, the job will be cancelled. + needScheduleTasksQueue.put(routineLoadTaskInfo); + LOG.warn(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, routineLoadTaskInfo.getId()) + .add("error_msg", "put task to the rear of queue with error " + e.getMessage()) + .build(), e); + continue; + } - long beId = routineLoadManager.getMinTaskBeId(); - RoutineLoadJob routineLoadJob = null; + // task to thrift + TRoutineLoadTask tRoutineLoadTask = null; try { - routineLoadJob = routineLoadManager.getJobByTaskId(routineLoadTaskInfo.getId()); - } catch (MetaNotFoundException e) { - LOG.warn("task {} has been abandoned", routineLoadTaskInfo.getId()); - return; + tRoutineLoadTask = routineLoadTaskInfo.createRoutineLoadTask(); + } catch (UserException e) { + routineLoadManager.getJob(routineLoadTaskInfo.getJobId()).updateState(JobState.PAUSED, + "failed to create task: " + e.getMessage(), false); + throw e; + } + + // set the executeStartTimeMs of task + routineLoadTaskInfo.setExecuteStartTimeMs(System.currentTimeMillis()); + // add to batch task map + if (beIdToBatchTask.containsKey(routineLoadTaskInfo.getBeId())) { + beIdToBatchTask.get(routineLoadTaskInfo.getBeId()).add(tRoutineLoadTask); + } else { + List tRoutineLoadTaskList = Lists.newArrayList(); + tRoutineLoadTaskList.add(tRoutineLoadTask); + beIdToBatchTask.put(routineLoadTaskInfo.getBeId(), tRoutineLoadTaskList); } - RoutineLoadTask routineLoadTask = routineLoadTaskInfo.createStreamLoadTask(beId); - // remove task for needScheduleTasksList in job - routineLoadJob.removeNeedScheduleTask(routineLoadTaskInfo); - routineLoadTaskInfo.setLoadStartTimeMs(System.currentTimeMillis()); - AgentTaskQueue.addTask(routineLoadTask); - batchTask.addTask(routineLoadTask); - clusterIdleSlotNum--; + // count scheduledTaskNum++; - routineLoadManager.addNumOfConcurrentTasksByBeId(beId); + } + submitBatchTask(beIdToBatchTask); + LOG.info("{} tasks have been allocated to be.", scheduledTaskNum); + } + + private void updateBackendSlotIfNecessary() { + long currentTime = System.currentTimeMillis(); + if (lastBackendSlotUpdateTime != -1 + && currentTime - lastBackendSlotUpdateTime > BACKEND_SLOT_UPDATE_INTERVAL_MS) { + routineLoadManager.updateBeIdToMaxConcurrentTasks(); + lastBackendSlotUpdateTime = currentTime; + if (LOG.isDebugEnabled()) { + LOG.debug("update backend max slot for routine load task scheduling"); + } + } + } + + public void addTaskInQueue(RoutineLoadTaskInfo routineLoadTaskInfo) { + needScheduleTasksQueue.add(routineLoadTaskInfo); + } - needScheduleTaskNum--; + public void addTasksInQueue(List routineLoadTaskInfoList) { + needScheduleTasksQueue.addAll(routineLoadTaskInfoList); + } + + private void submitBatchTask(Map> beIdToRoutineLoadTask) { + for (Map.Entry> entry : beIdToRoutineLoadTask.entrySet()) { + Backend backend = Catalog.getCurrentSystemInfo().getBackend(entry.getKey()); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBePort()); + BackendService.Client client = null; + boolean ok = false; + try { + client = ClientPool.backendPool.borrowObject(address); + client.submit_routine_load_task(entry.getValue()); + if (LOG.isDebugEnabled()) { + LOG.debug("{} tasks sent to be {}", entry.getValue().size(), entry.getKey()); + } + ok = true; + } catch (Exception e) { + LOG.warn("task send error. backend[{}]", backend.getId(), e); + } finally { + if (ok) { + ClientPool.backendPool.returnObject(address, client); + } else { + ClientPool.backendPool.invalidateObject(address, client); + } + } } - LOG.info("{} tasks have bean allocated to be.", scheduledTaskNum); + } - if (batchTask.getTaskNum() > 0) { - AgentTaskExecutor.submit(batchTask); + // check if previous be has idle slot + // true: allocate previous be to task + // false: allocate the most idle be to task + private void allocateTaskToBe(RoutineLoadTaskInfo routineLoadTaskInfo) + throws MetaNotFoundException, LoadException { + if (routineLoadTaskInfo.getPreviousBeId() != -1L) { + if (routineLoadManager.checkBeToTask(routineLoadTaskInfo.getPreviousBeId(), routineLoadTaskInfo.getClusterName())) { + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, routineLoadTaskInfo.getId()) + .add("job_id", routineLoadTaskInfo.getJobId()) + .add("previous_be_id", routineLoadTaskInfo.getPreviousBeId()) + .add("msg", "task use the previous be id") + .build()); + } + routineLoadTaskInfo.setBeId(routineLoadTaskInfo.getPreviousBeId()); + return; + } + } + routineLoadTaskInfo.setBeId(routineLoadManager.getMinTaskBeId(routineLoadTaskInfo.getClusterName())); + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, routineLoadTaskInfo.getId()) + .add("job_id", routineLoadTaskInfo.getJobId()) + .add("be_id", routineLoadTaskInfo.getBeId()) + .add("msg", "task has been allocated to be") + .build()); } } } diff --git a/fe/src/main/java/org/apache/doris/metric/MetricRepo.java b/fe/src/main/java/org/apache/doris/metric/MetricRepo.java index 44e8e3301c15a4..7a0ee7b748172c 100644 --- a/fe/src/main/java/org/apache/doris/metric/MetricRepo.java +++ b/fe/src/main/java/org/apache/doris/metric/MetricRepo.java @@ -61,8 +61,13 @@ public final class MetricRepo { public static LongCounterMetric COUNTER_EDIT_LOG_SIZE_BYTES; public static LongCounterMetric COUNTER_IMAGE_WRITE; public static LongCounterMetric COUNTER_IMAGE_PUSH; + public static LongCounterMetric COUNTER_TXN_BEGIN; public static LongCounterMetric COUNTER_TXN_FAILED; public static LongCounterMetric COUNTER_TXN_SUCCESS; + public static LongCounterMetric COUNTER_ROUTINE_LOAD_ROWS; + public static LongCounterMetric COUNTER_ROUTINE_LOAD_RECEIVED_BYTES; + public static LongCounterMetric COUNTER_ROUTINE_LOAD_ERROR_ROWS; + public static Histogram HISTO_QUERY_LATENCY; public static Histogram HISTO_EDIT_LOG_WRITE_LATENCY; @@ -202,13 +207,24 @@ public Long getValue() { COUNTER_IMAGE_PUSH = new LongCounterMetric("image_push", "counter of image succeeded in pushing to other frontends"); PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_IMAGE_PUSH); - COUNTER_TXN_SUCCESS = new LongCounterMetric("txn_success", - "counter of success transactions"); + + COUNTER_TXN_BEGIN = new LongCounterMetric("txn_begin", "counter of begining transactions"); + PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_TXN_BEGIN); + COUNTER_TXN_FAILED = new LongCounterMetric("txn_failed", "counter of failed transactions"); + COUNTER_TXN_SUCCESS = new LongCounterMetric("txn_success", "counter of success transactions"); PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_TXN_SUCCESS); - COUNTER_TXN_FAILED = new LongCounterMetric("txn_failed", - "counter of failed transactions"); + COUNTER_TXN_FAILED = new LongCounterMetric("txn_failed", "counter of failed transactions"); PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_TXN_FAILED); + COUNTER_ROUTINE_LOAD_ROWS = new LongCounterMetric("routine_load_rows", "total rows of routine load"); + PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_ROUTINE_LOAD_ROWS); + COUNTER_ROUTINE_LOAD_RECEIVED_BYTES = new LongCounterMetric("routine_load_receive_bytes", + "total received bytes of routine load"); + PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_ROUTINE_LOAD_RECEIVED_BYTES); + COUNTER_ROUTINE_LOAD_ERROR_ROWS = new LongCounterMetric("routine_load_error_rows", + "total error rows of routine load"); + PALO_METRIC_REGISTER.addPaloMetrics(COUNTER_ROUTINE_LOAD_ERROR_ROWS); + // 3. histogram HISTO_QUERY_LATENCY = METRIC_REGISTER.histogram(MetricRegistry.name("query", "latency", "ms")); HISTO_EDIT_LOG_WRITE_LATENCY = METRIC_REGISTER.histogram(MetricRegistry.name("editlog", "write", "latency", diff --git a/fe/src/main/java/org/apache/doris/persist/EditLog.java b/fe/src/main/java/org/apache/doris/persist/EditLog.java index 8223c20959f1ba..14a37479b96a34 100644 --- a/fe/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/src/main/java/org/apache/doris/persist/EditLog.java @@ -664,6 +664,21 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { Catalog.getCurrentCatalog().replayBackendTabletsInfo(backendTabletsInfo); break; } + case OperationType.OP_CREATE_ROUTINE_LOAD_JOB: { + RoutineLoadJob routineLoadJob = (RoutineLoadJob) journal.getData(); + Catalog.getCurrentCatalog().getRoutineLoadManager().replayCreateRoutineLoadJob(routineLoadJob); + break; + } + case OperationType.OP_CHANGE_ROUTINE_LOAD_JOB: { + RoutineLoadOperation operation = (RoutineLoadOperation) journal.getData(); + Catalog.getCurrentCatalog().getRoutineLoadManager().replayChangeRoutineLoadJob(operation); + break; + } + case OperationType.OP_REMOVE_ROUTINE_LOAD_JOB: { + RoutineLoadOperation operation = (RoutineLoadOperation) journal.getData(); + Catalog.getCurrentCatalog().getRoutineLoadManager().replayRemoveOldRoutineLoad(operation); + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); @@ -704,7 +719,7 @@ public void rollEditLog() { */ private synchronized void logEdit(short op, Writable writable) { if (this.getNumEditStreams() == 0) { - LOG.error("Fatal Error : no editLog stream"); + LOG.error("Fatal Error : no editLog stream", new Exception()); throw new Error("Fatal Error : no editLog stream"); } @@ -1166,4 +1181,16 @@ public void logDropFunction(FunctionSearchDesc function) { public void logBackendTabletsInfo(BackendTabletsInfo backendTabletsInfo) { logEdit(OperationType.OP_BACKEND_TABLETS_INFO, backendTabletsInfo); } + + public void logCreateRoutineLoadJob(RoutineLoadJob routineLoadJob) { + logEdit(OperationType.OP_CREATE_ROUTINE_LOAD_JOB, routineLoadJob); + } + + public void logOpRoutineLoadJob(RoutineLoadOperation routineLoadOperation) { + logEdit(OperationType.OP_CHANGE_ROUTINE_LOAD_JOB, routineLoadOperation); + } + + public void logRemoveRoutineLoadJob(RoutineLoadOperation operation) { + logEdit(OperationType.OP_REMOVE_ROUTINE_LOAD_JOB, operation); + } } diff --git a/fe/src/main/java/org/apache/doris/persist/OperationType.java b/fe/src/main/java/org/apache/doris/persist/OperationType.java index 2af084dd89ba86..05b788ec238c4a 100644 --- a/fe/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/src/main/java/org/apache/doris/persist/OperationType.java @@ -156,4 +156,8 @@ public class OperationType { public static final short OP_ADD_FUNCTION = 130; public static final short OP_DROP_FUNCTION = 131; + // routine load 200 + public static final short OP_CREATE_ROUTINE_LOAD_JOB = 200; + public static final short OP_CHANGE_ROUTINE_LOAD_JOB = 201; + public static final short OP_REMOVE_ROUTINE_LOAD_JOB = 202; } diff --git a/fe/src/main/java/org/apache/doris/persist/RoutineLoadOperation.java b/fe/src/main/java/org/apache/doris/persist/RoutineLoadOperation.java new file mode 100644 index 00000000000000..208096877019e2 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/persist/RoutineLoadOperation.java @@ -0,0 +1,70 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.persist; + +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.load.routineload.RoutineLoadJob.JobState; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +/* + * Author: Chenmingyu + * Date: Mar 14, 2019 + */ + +public class RoutineLoadOperation implements Writable { + private long id; + private JobState jobState; + + private RoutineLoadOperation() { + } + + public RoutineLoadOperation(long id, JobState jobState) { + this.id = id; + this.jobState = jobState; + } + + public long getId() { + return id; + } + + public JobState getJobState() { + return jobState; + } + + public static RoutineLoadOperation read(DataInput in) throws IOException { + RoutineLoadOperation operation = new RoutineLoadOperation(); + operation.readFields(in); + return operation; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(id); + Text.writeString(out, jobState.name()); + } + + @Override + public void readFields(DataInput in) throws IOException { + id = in.readLong(); + jobState = JobState.valueOf(Text.readString(in)); + } +} diff --git a/fe/src/main/java/org/apache/doris/planner/EsScanNode.java b/fe/src/main/java/org/apache/doris/planner/EsScanNode.java index 790b976ebb0d44..f54c9e0846fd30 100644 --- a/fe/src/main/java/org/apache/doris/planner/EsScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/EsScanNode.java @@ -128,7 +128,11 @@ public void finalize(Analyzer analyzer) throws UserException { @Override protected void toThrift(TPlanNode msg) { - msg.node_type = TPlanNodeType.ES_SCAN_NODE; + if (EsTable.TRANSPORT_HTTP.equals(table.getTransport())) { + msg.node_type = TPlanNodeType.ES_HTTP_SCAN_NODE; + } else { + msg.node_type = TPlanNodeType.ES_SCAN_NODE; + } Map properties = Maps.newHashMap(); properties.put(EsTable.USER, table.getUserName()); properties.put(EsTable.PASSWORD, table.getPasswd()); @@ -187,8 +191,11 @@ private List getShardLocations() throws UserException { // get backends Set colocatedBes = Sets.newHashSet(); int numBe = Math.min(3, backendMap.size()); - List shardAllocations = shardRouting.stream().map(e -> e.getAddress()) - .collect(Collectors.toList()); + List shardAllocations = new ArrayList<>(); + for (EsShardRouting item : shardRouting) { + shardAllocations.add(EsTable.TRANSPORT_HTTP.equals(table.getTransport()) ? item.getHttpAddress() : item.getAddress()); + } + Collections.shuffle(shardAllocations, random); for (TNetworkAddress address : shardAllocations) { colocatedBes.addAll(backendMap.get(address.getHostname())); diff --git a/fe/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/src/main/java/org/apache/doris/planner/OlapScanNode.java index 2efb62ef15e9af..05f07b4acf03fd 100644 --- a/fe/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -74,6 +74,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; /** * Full scan of an Olap table. @@ -87,8 +88,6 @@ public class OlapScanNode extends ScanNode { private boolean canTurnOnPreAggr = true; private ArrayList tupleColumns = new ArrayList(); private HashSet predicateColumns = new HashSet(); - private HashSet inPredicateColumns = new HashSet(); - private HashSet eqJoinColumns = new HashSet(); private OlapTable olapTable = null; private long selectedTabletsNum = 0; private long totalTabletsNum = 0; @@ -169,193 +168,6 @@ public void computeStats(Analyzer analyzer) { } } - - // private void analyzeVectorizedConjuncts(Analyzer analyzer) throws InternalException { - // for (SlotDescriptor slot : desc.getSlots()) { - // for (Expr conjunct : conjuncts) { - // if (expr.isConstant()) { - // continue; - // } - // if (analyzer.isWhereClauseConjunct(conjunct) - // && expr.isBound(slot.getId()) - // && conjunct.isVectorized() - // && conjunct instanceof Predicate) { - // conjunct.computeOutputColumn(analyzer); - // } else { - // Preconditions.checkState(false); - // } - // } - // } - // } - - private List selectRollupIndex(Partition partition) throws UserException { - if (olapTable.getKeysType() == KeysType.DUP_KEYS) { - isPreAggregation = true; - } - - List allIndices = Lists.newArrayList(); - allIndices.add(partition.getBaseIndex()); - allIndices.addAll(partition.getRollupIndices()); - LOG.debug("num of rollup(base included): {}, pre aggr: {}", allIndices.size(), isPreAggregation); - - // 1. find all rollup indexes which contains all tuple columns - List containTupleIndexes = Lists.newArrayList(); - List baseIndexKeyColumns = olapTable.getKeyColumnsByIndexId(partition.getBaseIndex().getId()); - for (MaterializedIndex index : allIndices) { - Set indexColNames = Sets.newHashSet(); - for (Column col : olapTable.getSchemaByIndexId(index.getId())) { - indexColNames.add(col.getName()); - } - - if (indexColNames.containsAll(tupleColumns)) { - // If preAggregation is off, so that we only can use base table - // or those rollup tables which key columns is the same with base table - // (often in different order) - if (isPreAggregation) { - LOG.debug("preAggregation is on. add index {} which contains all tuple columns", index.getId()); - containTupleIndexes.add(index); - } else if (olapTable.getKeyColumnsByIndexId(index.getId()).size() == baseIndexKeyColumns.size()) { - LOG.debug("preAggregation is off, but index {} have same key columns with base index.", - index.getId()); - containTupleIndexes.add(index); - } - } else { - LOG.debug("exclude index {} because it does not contain all tuple columns", index.getId()); - } - } - - if (containTupleIndexes.isEmpty()) { - throw new UserException("Failed to select index, no match index"); - } - - // 2. find all indexes which match the prefix most based on predicate/sort/in predicate columns - // from containTupleIndices. - List prefixMatchedIndexes = Lists.newArrayList(); - int maxPrefixMatchCount = 0; - int prefixMatchCount = 0; - for (MaterializedIndex index : containTupleIndexes) { - prefixMatchCount = 0; - for (Column col : olapTable.getSchemaByIndexId(index.getId())) { - if (sortColumn != null) { - if (inPredicateColumns.contains(col.getName())) { - prefixMatchCount++; - } else if (sortColumn.equals(col.getName())) { - prefixMatchCount++; - break; - } else { - break; - } - } else { - if (predicateColumns.contains(col.getName())) { - break; - } - } - } - if (prefixMatchCount == maxPrefixMatchCount) { - LOG.debug("s2: find a equal prefix match index {}. match count: {}", index.getId(), prefixMatchCount); - prefixMatchedIndexes.add(index); - } else if (prefixMatchCount > maxPrefixMatchCount) { - LOG.debug("s2: find a better prefix match index {}. match count: {}", index.getId(), prefixMatchCount); - maxPrefixMatchCount = prefixMatchCount; - prefixMatchedIndexes.clear(); - prefixMatchedIndexes.add(index); - } - } - - // 3. find all indexes which match the prefix most based on equal join columns - // from containTupleIndices. - List eqJoinPrefixMatchedIndexes = Lists.newArrayList(); - maxPrefixMatchCount = 0; - for (MaterializedIndex index : containTupleIndexes) { - prefixMatchCount = 0; - for (Column col : olapTable.getSchemaByIndexId(index.getId())) { - if (eqJoinColumns.contains(col.getName()) || predicateColumns.contains(col.getName())) { - prefixMatchCount++; - } else { - break; - } - } - if (prefixMatchCount == maxPrefixMatchCount) { - LOG.debug("s3: find a equal prefix match index {}. match count: {}", index.getId(), prefixMatchCount); - eqJoinPrefixMatchedIndexes.add(index); - } else if (prefixMatchCount > maxPrefixMatchCount) { - LOG.debug("s3: find a better prefix match index {}. match count: {}", index.getId(), prefixMatchCount); - maxPrefixMatchCount = prefixMatchCount; - eqJoinPrefixMatchedIndexes.clear(); - eqJoinPrefixMatchedIndexes.add(index); - } - } - - // 4. find the intersection of prefixMatchIndices and eqJoinPrefixMatchIndices as candidate indexes - List finalCandidateIndexes = Lists.newArrayList(); - for (MaterializedIndex index : prefixMatchedIndexes) { - for (MaterializedIndex oneIndex : eqJoinPrefixMatchedIndexes) { - if (oneIndex.getId() == index.getId()) { - finalCandidateIndexes.add(index); - LOG.debug("find a matched index {} in intersection of " - + "prefixMatchIndices and eqJoinPrefixMatchIndices", - index.getId()); - } - } - } - // maybe there is no intersection between prefixMatchIndices and eqJoinPrefixMatchIndices. - // in this case, use prefixMatchIndices; - if (finalCandidateIndexes.isEmpty()) { - finalCandidateIndexes = prefixMatchedIndexes; - } - - // 5. sorted the final candidate indexes by index id - // this is to make sure that candidate indexes find in all partitions will be returned in same order - Collections.sort(finalCandidateIndexes, new Comparator() { - @Override - public int compare(MaterializedIndex index1, MaterializedIndex index2) - { - return (int) (index1.getId() - index2.getId()); - } - }); - return finalCandidateIndexes; - } - - private void normalizePredicate(Analyzer analyzer) throws UserException { - // 1. Get Columns which has eqJoin on it - List eqJoinPredicate = analyzer.getEqJoinConjuncts(desc.getId()); - for (Expr expr : eqJoinPredicate) { - for (SlotDescriptor slot : desc.getSlots()) { - for (int i = 0; i < 2; i++) { - if (expr.getChild(i).isBound(slot.getId())) { - eqJoinColumns.add(slot.getColumn().getName()); - LOG.debug("Add eqJoinColumn: ColName=" + slot.getColumn().getName()); - break; - } - } - } - } - - // 2. Get Columns which has predicate on it - for (SlotDescriptor slot : desc.getSlots()) { - for (Expr expr : conjuncts) { - if (expr.isConstant()) { - continue; - } - if (expr.isBound(slot.getId())) { - predicateColumns.add(slot.getColumn().getName()); - LOG.debug("Add predicateColumn: ColName=" + slot.getColumn().getName()); - if (expr instanceof InPredicate) { - inPredicateColumns.add(slot.getColumn().getName()); - LOG.debug("Add inPredicateColumn: ColName=" + slot.getColumn().getName()); - } - } - } - } - - // 3. Get Columns of this tuple - for (SlotDescriptor slot : desc.getSlots()) { - Column col = slot.getColumn(); - tupleColumns.add(col.getName()); - LOG.debug("Add tupleColumn: ColName=" + col.getName()); - } - } - private Collection partitionPrune(PartitionInfo partitionInfo) throws AnalysisException { PartitionPruner partitionPruner = null; switch(partitionInfo.getType()) { @@ -500,86 +312,48 @@ private void addScanRangeLocations(Partition partition, } private void getScanRangeLocations(Analyzer analyzer) throws UserException, AnalysisException { - normalizePredicate(analyzer); - long start = System.currentTimeMillis(); Collection partitionIds = partitionPrune(olapTable.getPartitionInfo()); - + if (partitionIds == null) { partitionIds = new ArrayList(); for (Partition partition : olapTable.getPartitions()) { + if (!partition.hasData()) { + continue; + } partitionIds.add(partition.getId()); } + } else { + partitionIds = partitionIds.stream().filter(id -> olapTable.getPartition(id).hasData()).collect( + Collectors.toList()); } + selectedPartitionNum = partitionIds.size(); LOG.debug("partition prune cost: {} ms, partitions: {}", (System.currentTimeMillis() - start), partitionIds); start = System.currentTimeMillis(); - // find all candidate rollups - int candidateTableSize = 0; - List> tables = Lists.newArrayList(); - for (Long partitionId : partitionIds) { - Partition partition = olapTable.getPartition(partitionId); - List candidateTables = selectRollupIndex(partition); - if (candidateTableSize == 0) { - candidateTableSize = candidateTables.size(); - } else { - if (candidateTableSize != candidateTables.size()) { - String errMsg = "two partition's candidate_table_size not equal, one is " + candidateTableSize - + ", the other is" + candidateTables.size(); - throw new AnalysisException(errMsg); - } - } - tables.add(candidateTables); + if (olapTable.getKeysType() == KeysType.DUP_KEYS) { + isPreAggregation = true; } - // chose one rollup from candidate rollups - long minRowCount = Long.MAX_VALUE; - int partitionPos = -1; - for (int i = 0; i < candidateTableSize; i++) { - MaterializedIndex candidateIndex = null; - long rowCount = 0; - for (List candidateTables : tables) { - if (candidateIndex == null) { - candidateIndex = candidateTables.get(i); - } else { - if (candidateIndex.getId() != candidateTables.get(i).getId()) { - String errMsg = "two partition's candidate_table not equal, one is " - + candidateIndex.getId() + ", the other is " + candidateTables.get(i).getId(); - throw new AnalysisException(errMsg); - } - } - rowCount += candidateTables.get(i).getRowCount(); - } - LOG.debug("rowCount={} for table={}", rowCount, candidateIndex.getId()); - if (rowCount < minRowCount) { - minRowCount = rowCount; - selectedIndexId = tables.get(0).get(i).getId(); - partitionPos = i; - } else if (rowCount == minRowCount) { - // check column number, select one minimum column number - int selectedColumnSize = olapTable.getIndexIdToSchema().get(selectedIndexId).size(); - int currColumnSize = olapTable.getIndexIdToSchema().get(tables.get(0).get(i).getId()).size(); - if (currColumnSize < selectedColumnSize) { - selectedIndexId = tables.get(0).get(i).getId(); - partitionPos = i; - } - } + if (partitionIds.size() == 0) { + return; } + final RollupSelector rollupSelector = new RollupSelector(analyzer, desc, olapTable); + selectedIndexId = rollupSelector.selectBestRollup(partitionIds, conjuncts, isPreAggregation); + long localBeId = -1; if (Config.enable_local_replica_selection) { localBeId = Catalog.getCurrentSystemInfo().getBackendIdByHost(FrontendOptions.getLocalHostAddress()); } - MaterializedIndex selectedTable = null; - int j = 0; + for (Long partitionId : partitionIds) { - Partition partition = olapTable.getPartition(partitionId); - LOG.debug("selected partition: " + partition.getName()); - selectedTable = tables.get(j++).get(partitionPos); - List tablets = new ArrayList(); - Collection tabletIds = distributionPrune(selectedTable, partition.getDistributionInfo()); + final Partition partition = olapTable.getPartition(partitionId); + final MaterializedIndex selectedTable = partition.getIndex(selectedIndexId); + final List tablets = Lists.newArrayList(); + final Collection tabletIds = distributionPrune(selectedTable, partition.getDistributionInfo()); LOG.debug("distribution prune tablets: {}", tabletIds); List allTabletIds = selectedTable.getTabletIdsInOrder(); diff --git a/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java b/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java index 14a91586f22e8f..4719fe7196d1f2 100644 --- a/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java +++ b/fe/src/main/java/org/apache/doris/planner/OlapTableSink.java @@ -52,6 +52,7 @@ import org.apache.doris.thrift.TUniqueId; import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Range; import com.google.common.collect.Sets; @@ -84,7 +85,7 @@ public OlapTableSink(OlapTable dstTable, TupleDescriptor tupleDescriptor) { public OlapTableSink(OlapTable dstTable, TupleDescriptor tupleDescriptor, String partitions) { this.dstTable = dstTable; this.tupleDescriptor = tupleDescriptor; - this.partitions = partitions; + this.partitions = Strings.emptyToNull(partitions); } public void init(TUniqueId loadId, long txnId, long dbId) throws AnalysisException { @@ -284,13 +285,15 @@ private TOlapTableLocationParam createLocation(OlapTable table) throws UserExcep for (Partition partition : table.getPartitions()) { int quorum = table.getPartitionInfo().getReplicationNum(partition.getId()) / 2 + 1; for (MaterializedIndex index : partition.getMaterializedIndices()) { + //we should ensure the replica backend is available + //otherwise, there will be a 'unknown node id, id=xxx' error for stream load for (Tablet tablet : index.getTablets()) { - List beIds = tablet.getBackendIdsList(); + List beIds = tablet.getAvailableBackendIdsList(); if (beIds.size() < quorum) { throw new UserException("tablet " + tablet.getId() + " has few replicas: " + beIds.size()); } locationParam.addToTablets( - new TTabletLocation(tablet.getId(), Lists.newArrayList(tablet.getBackendIds()))); + new TTabletLocation(tablet.getId(), Lists.newArrayList(tablet.getAvailableBackendIds()))); } } } diff --git a/fe/src/main/java/org/apache/doris/planner/PartitionColumnFilter.java b/fe/src/main/java/org/apache/doris/planner/PartitionColumnFilter.java index 191354a5c68222..6350267fc7eac2 100644 --- a/fe/src/main/java/org/apache/doris/planner/PartitionColumnFilter.java +++ b/fe/src/main/java/org/apache/doris/planner/PartitionColumnFilter.java @@ -22,12 +22,13 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PartitionKey; import org.apache.doris.common.AnalysisException; + import com.google.common.collect.BoundType; import com.google.common.collect.Lists; import com.google.common.collect.Range; -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.List; @@ -68,7 +69,7 @@ public void setLowerBound(LiteralExpr newLowerBound, boolean newLowerBoundInclus } } - // selete the smaller bound + // select the smaller bound public void setUpperBound(LiteralExpr newUpperBound, boolean newUpperBoundInclusive) { if (null == upperBound) { upperBound = newUpperBound; diff --git a/fe/src/main/java/org/apache/doris/planner/RollupSelector.java b/fe/src/main/java/org/apache/doris/planner/RollupSelector.java new file mode 100644 index 00000000000000..b62e57a578bb8e --- /dev/null +++ b/fe/src/main/java/org/apache/doris/planner/RollupSelector.java @@ -0,0 +1,300 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.planner; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.CastExpr; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.BinaryPredicate; +import org.apache.doris.analysis.InPredicate; +import org.apache.doris.analysis.LiteralExpr; +import org.apache.doris.analysis.SlotDescriptor; +import org.apache.doris.analysis.SlotRef; +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.common.UserException; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.*; +import java.util.stream.Collectors; + +public final class RollupSelector { + private static final Logger LOG = LogManager.getLogger(RollupSelector.class); + + // Rollup's table info. + private final TupleDescriptor tupleDesc; + private final OlapTable table; + private final Analyzer analyzer; + + public RollupSelector(Analyzer analyzer, TupleDescriptor tupleDesc, OlapTable table) { + this.analyzer = analyzer; + this.tupleDesc = tupleDesc; + this.table = table; + } + + public long selectBestRollup( + Collection partitionIds, List conjuncts, boolean isPreAggregation) + throws UserException { + Preconditions.checkArgument(partitionIds != null && !partitionIds.isEmpty(), + "Paritition can't be null or empty."); + // Get first partition to select best prefix index rollups, because MaterializedIndex ids in one rollup's partitions are all same. + final List bestPrefixIndexRollups = + selectBestPrefixIndexRollup( + table.getPartition(partitionIds.iterator().next()), + conjuncts, + isPreAggregation); + return selectBestRowCountRollup(bestPrefixIndexRollups, partitionIds); + } + + private long selectBestRowCountRollup(List bestPrefixIndexRollups, Collection partitionIds) { + long minRowCount = Long.MAX_VALUE; + long selectedIndexId = 0; + for (Long indexId : bestPrefixIndexRollups) { + long rowCount = 0; + for (Long partitionId : partitionIds) { + rowCount += table.getPartition(partitionId).getIndex(indexId).getRowCount(); + } + LOG.debug("rowCount={} for table={}", rowCount, indexId); + if (rowCount < minRowCount) { + minRowCount = rowCount; + selectedIndexId = indexId; + } else if (rowCount == minRowCount) { + // check column number, select one minimum column number + int selectedColumnSize = table.getIndexIdToSchema().get(selectedIndexId).size(); + int currColumnSize = table.getIndexIdToSchema().get(indexId).size(); + if (currColumnSize < selectedColumnSize) { + selectedIndexId = indexId; + } + } + } + return selectedIndexId; + } + + private List selectBestPrefixIndexRollup( + Partition partition, List conjuncts, boolean isPreAggregation) throws UserException { + + final List outputColumns = Lists.newArrayList(); + for (SlotDescriptor slot : tupleDesc.getMaterializedSlots()) { + Column col = slot.getColumn(); + outputColumns.add(col.getName()); + } + + final List rollups = Lists.newArrayList(); + rollups.add(partition.getBaseIndex()); + rollups.addAll(partition.getRollupIndices()); + LOG.debug("num of rollup(base included): {}, pre aggr: {}", rollups.size(), isPreAggregation); + + // 1. find all rollup indexes which contains all tuple columns + final List rollupsContainsOutput = Lists.newArrayList(); + final List baseTableColumns = table.getKeyColumnsByIndexId(partition.getBaseIndex().getId()); + for (MaterializedIndex rollup : rollups) { + final Set rollupColumns = Sets.newHashSet(); + table.getSchemaByIndexId(rollup.getId()) + .stream().forEach(column -> rollupColumns.add(column.getName())); + + if (rollupColumns.containsAll(outputColumns)) { + // If preAggregation is off, so that we only can use base table + // or those rollup tables which key columns is the same with base table + // (often in different order) + if (isPreAggregation) { + LOG.debug("preAggregation is on. add index {} which contains all output columns", + rollup.getId()); + rollupsContainsOutput.add(rollup); + } else if (table.getKeyColumnsByIndexId(rollup.getId()).size() == baseTableColumns.size()) { + LOG.debug("preAggregation is off, but index {} have same key columns with base index.", + rollup.getId()); + rollupsContainsOutput.add(rollup); + } + } else { + LOG.debug("exclude index {} because it does not contain all output columns", rollup.getId()); + } + } + + Preconditions.checkArgument(rollupsContainsOutput.size() > 0, + "Can't find candicate rollup contains all output columns."); + + + // 2. find all rollups which match the prefix most based on predicates column from containTupleIndices. + final Set equivalenceColumns = Sets.newHashSet(); + final Set unequivalenceColumns = Sets.newHashSet(); + collectColumns(conjuncts, equivalenceColumns, unequivalenceColumns); + final List rollupsMatchingBestPrefixIndex = Lists.newArrayList(); + matchPrefixIndex(rollupsContainsOutput, rollupsMatchingBestPrefixIndex, + equivalenceColumns, unequivalenceColumns); + + if (rollupsMatchingBestPrefixIndex.isEmpty()) { + rollupsContainsOutput.stream().forEach(n -> rollupsMatchingBestPrefixIndex.add(n.getId())); + } + + // 3. sorted the final candidate indexes by index id + // this is to make sure that candidate indexes find in all partitions will be returned in same order + Collections.sort(rollupsMatchingBestPrefixIndex, new Comparator() { + @Override + public int compare(Long id1, Long id2) { + return (int) (id1 - id2); + } + }); + return rollupsMatchingBestPrefixIndex; + } + + private void matchPrefixIndex(List candidateRollups, + List rollupsMatchingBestPrefixIndex, + Set equivalenceColumns, + Set unequivalenceColumns) { + if (equivalenceColumns.size() == 0 && unequivalenceColumns.size() == 0) { + return; + } + int maxPrefixMatchCount = 0; + int prefixMatchCount; + for (MaterializedIndex index : candidateRollups) { + prefixMatchCount = 0; + for (Column col : table.getSchemaByIndexId(index.getId())) { + if (equivalenceColumns.contains(col.getName())) { + prefixMatchCount++; + } else if (unequivalenceColumns.contains(col.getName())) { + // Unequivalence predicate's columns can match only first column in rollup. + prefixMatchCount++; + break; + } else { + break; + } + } + + if (prefixMatchCount == maxPrefixMatchCount) { + LOG.debug("s3: find a equal prefix match index {}. match count: {}", index.getId(), prefixMatchCount); + rollupsMatchingBestPrefixIndex.add(index.getId()); + } else if (prefixMatchCount > maxPrefixMatchCount) { + LOG.debug("s3: find a better prefix match index {}. match count: {}", index.getId(), prefixMatchCount); + maxPrefixMatchCount = prefixMatchCount; + rollupsMatchingBestPrefixIndex.clear(); + rollupsMatchingBestPrefixIndex.add(index.getId()); + } + } + } + + private void collectColumns( + List conjuncts, Set equivalenceColumns, Set unequivalenceColumns) { + + // 1. Get columns which has predicate on it. + for (Expr expr : conjuncts) { + if (!isPredicateUsedForPrefixIndex(expr, false)) { + continue; + } + for (SlotDescriptor slot : tupleDesc.getMaterializedSlots()) { + if (expr.isBound(slot.getId())) { + if (!isEquivalenceExpr(expr)) { + unequivalenceColumns.add(slot.getColumn().getName()); + } else { + equivalenceColumns.add(slot.getColumn().getName()); + } + break; + } + } + } + + // 2. Equal join predicates when pushing inner child. + List eqJoinPredicate = analyzer.getEqJoinConjuncts(tupleDesc.getId()); + for (Expr expr : eqJoinPredicate) { + if (!isPredicateUsedForPrefixIndex(expr, true)) { + continue; + } + for (SlotDescriptor slot : tupleDesc.getMaterializedSlots()) { + for (int i = 0; i < 2; i++) { + if (expr.getChild(i).isBound(slot.getId())) { + equivalenceColumns.add(slot.getColumn().getName()); + break; + } + } + } + } + } + + private boolean isEquivalenceExpr(Expr expr) { + if (expr instanceof InPredicate) { + return true; + } + if (expr instanceof BinaryPredicate) { + final BinaryPredicate predicate = (BinaryPredicate) expr; + if (predicate.getOp().isEquivalence()) { + return true; + } + } + return false; + } + + private boolean isPredicateUsedForPrefixIndex(Expr expr, boolean isJoinConjunct) { + if (!(expr instanceof InPredicate) + && !(expr instanceof BinaryPredicate)) { + return false; + } + if (expr instanceof InPredicate) { + return isInPredicateUsedForPrefixIndex((InPredicate)expr); + } else if (expr instanceof BinaryPredicate) { + if (isJoinConjunct) { + return isEqualJoinConjunctUsedForPrefixIndex((BinaryPredicate)expr); + } else { + return isBinaryPredicateUsedForPrefixIndex((BinaryPredicate)expr); + } + } + return true; + } + + private boolean isEqualJoinConjunctUsedForPrefixIndex(BinaryPredicate expr) { + Preconditions.checkArgument(expr.getOp().isEquivalence()); + if (expr.isAuxExpr()) { + return false; + } + for (Expr child : expr.getChildren()) { + for (SlotDescriptor slot : tupleDesc.getMaterializedSlots()) { + if (child.isBound(slot.getId()) && isSlotRefNested(child)) { + return true; + } + } + } + return false; + } + + private boolean isBinaryPredicateUsedForPrefixIndex(BinaryPredicate expr) { + if (expr.isAuxExpr() || expr.getOp().isUnequivalence()) { + return false; + } + return (isSlotRefNested(expr.getChild(0)) && expr.getChild(1).isConstant()) + || (isSlotRefNested(expr.getChild(1)) && expr.getChild(0).isConstant()); + } + + private boolean isInPredicateUsedForPrefixIndex(InPredicate expr) { + if (expr.isNotIn()) { + return false; + } + return isSlotRefNested(expr.getChild(0)) && expr.isLiteralChildren(); + } + + private boolean isSlotRefNested(Expr expr) { + while (expr instanceof CastExpr) { + expr = expr.getChild(0); + } + return expr instanceof SlotRef; + } +} diff --git a/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java b/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java index 78ec6abf4b53b4..5bb2d4762d932a 100644 --- a/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java +++ b/fe/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.common.UserException; import org.apache.doris.load.LoadErrorHub; +import org.apache.doris.task.StreamLoadTask; import org.apache.doris.thrift.PaloInternalServiceVersion; import org.apache.doris.thrift.TExecPlanFragmentParams; import org.apache.doris.thrift.TLoadErrorHubInfo; @@ -36,7 +37,6 @@ import org.apache.doris.thrift.TQueryType; import org.apache.doris.thrift.TScanRangeLocations; import org.apache.doris.thrift.TScanRangeParams; -import org.apache.doris.thrift.TStreamLoadPutRequest; import org.apache.doris.thrift.TUniqueId; import com.google.common.collect.Lists; @@ -63,15 +63,15 @@ public class StreamLoadPlanner { // Data will load to this table private Database db; private OlapTable destTable; - private TStreamLoadPutRequest request; + private StreamLoadTask streamLoadTask; private Analyzer analyzer; private DescriptorTable descTable; - public StreamLoadPlanner(Database db, OlapTable destTable, TStreamLoadPutRequest request) { + public StreamLoadPlanner(Database db, OlapTable destTable, StreamLoadTask streamLoadTask) { this.db = db; this.destTable = destTable; - this.request = request; + this.streamLoadTask = streamLoadTask; analyzer = new Analyzer(Catalog.getInstance(), null); descTable = analyzer.getDescTbl(); @@ -84,22 +84,18 @@ public TExecPlanFragmentParams plan() throws UserException { SlotDescriptor slotDesc = descTable.addSlotDescriptor(tupleDesc); slotDesc.setIsMaterialized(true); slotDesc.setColumn(col); - if (col.isAllowNull()) { - slotDesc.setIsNullable(true); - } else { - slotDesc.setIsNullable(false); - } + slotDesc.setIsNullable(col.isAllowNull()); } // create scan node - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(0), tupleDesc, destTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(0), tupleDesc, destTable, streamLoadTask); scanNode.init(analyzer); descTable.computeMemLayout(); scanNode.finalize(analyzer); // create dest sink - OlapTableSink olapTableSink = new OlapTableSink(destTable, tupleDesc, request.getPartitions()); - olapTableSink.init(request.getLoadId(), request.getTxnId(), db.getId()); + OlapTableSink olapTableSink = new OlapTableSink(destTable, tupleDesc, streamLoadTask.getPartitions()); + olapTableSink.init(streamLoadTask.getId(), streamLoadTask.getTxnId(), db.getId()); olapTableSink.finalize(); // for stream load, we only need one fragment, ScanNode -> DataSink. @@ -150,7 +146,7 @@ public TExecPlanFragmentParams plan() throws UserException { } } - LOG.debug("stream load txn id: {}, plan: {}", request.txnId, params); + // LOG.debug("stream load txn id: {}, plan: {}", streamLoadTask.getTxnId(), params); return params; } } diff --git a/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java b/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java index 1d9f86e8e5ac87..a9eaac5013f800 100644 --- a/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/StreamLoadScanNode.java @@ -18,18 +18,13 @@ package org.apache.doris.planner; import org.apache.doris.analysis.Analyzer; -import org.apache.doris.analysis.ColumnSeparator; import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.ExprSubstitutionMap; import org.apache.doris.analysis.FunctionCallExpr; import org.apache.doris.analysis.ImportColumnDesc; -import org.apache.doris.analysis.ImportColumnsStmt; -import org.apache.doris.analysis.ImportWhereStmt; import org.apache.doris.analysis.NullLiteral; import org.apache.doris.analysis.SlotDescriptor; import org.apache.doris.analysis.SlotRef; -import org.apache.doris.analysis.SqlParser; -import org.apache.doris.analysis.SqlScanner; import org.apache.doris.analysis.StringLiteral; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Column; @@ -39,6 +34,7 @@ import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; +import org.apache.doris.task.StreamLoadTask; import org.apache.doris.thrift.TBrokerRangeDesc; import org.apache.doris.thrift.TBrokerScanNode; import org.apache.doris.thrift.TBrokerScanRange; @@ -48,7 +44,6 @@ import org.apache.doris.thrift.TPlanNodeType; import org.apache.doris.thrift.TScanRange; import org.apache.doris.thrift.TScanRangeLocations; -import org.apache.doris.thrift.TStreamLoadPutRequest; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -56,7 +51,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.io.StringReader; import java.nio.charset.Charset; import java.util.List; import java.util.Map; @@ -70,7 +64,7 @@ public class StreamLoadScanNode extends ScanNode { // TODO(zc): now we use scanRange // input parameter private Table dstTable; - private TStreamLoadPutRequest request; + private StreamLoadTask streamLoadTask; // helper private Analyzer analyzer; @@ -82,10 +76,10 @@ public class StreamLoadScanNode extends ScanNode { // used to construct for streaming loading public StreamLoadScanNode( - PlanNodeId id, TupleDescriptor tupleDesc, Table dstTable, TStreamLoadPutRequest request) { + PlanNodeId id, TupleDescriptor tupleDesc, Table dstTable, StreamLoadTask streamLoadTask) { super(id, tupleDesc, "StreamLoadScanNode"); this.dstTable = dstTable; - this.request = request; + this.streamLoadTask = streamLoadTask; } @Override @@ -97,19 +91,19 @@ public void init(Analyzer analyzer) throws UserException { brokerScanRange = new TBrokerScanRange(); TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc(); - rangeDesc.file_type = request.getFileType(); - rangeDesc.format_type = request.getFormatType(); + rangeDesc.file_type = streamLoadTask.getFileType(); + rangeDesc.format_type = streamLoadTask.getFormatType(); rangeDesc.splittable = false; - switch (request.getFileType()) { + switch (streamLoadTask.getFileType()) { case FILE_LOCAL: - rangeDesc.path = request.getPath(); + rangeDesc.path = streamLoadTask.getPath(); break; case FILE_STREAM: rangeDesc.path = "Invalid Path"; - rangeDesc.load_id = request.getLoadId(); + rangeDesc.load_id = streamLoadTask.getId(); break; default: - throw new UserException("unsupported file type, type=" + request.getFileType()); + throw new UserException("unsupported file type, type=" + streamLoadTask.getFileType()); } rangeDesc.start_offset = 0; rangeDesc.size = -1; @@ -123,40 +117,23 @@ public void init(Analyzer analyzer) throws UserException { // columns: k1, k2, v1, v2=k1 + k2 // this means that there are three columns(k1, k2, v1) in source file, // and v2 is derived from (k1 + k2) - if (request.isSetColumns()) { - String columnsSQL = new String("COLUMNS " + request.getColumns()); - SqlParser parser = new SqlParser(new SqlScanner(new StringReader(columnsSQL))); - ImportColumnsStmt columnsStmt; - try { - columnsStmt = (ImportColumnsStmt) parser.parse().value; - } catch (Error e) { - LOG.warn("error happens when parsing columns, sql={}", columnsSQL, e); - throw new AnalysisException("failed to parsing columns' header, maybe contain unsupported character"); - } catch (AnalysisException e) { - LOG.warn("analyze columns' statement failed, sql={}, error={}", - columnsSQL, parser.getErrorMsg(columnsSQL), e); - String errorMessage = parser.getErrorMsg(columnsSQL); - if (errorMessage == null) { - throw e; - } else { - throw new AnalysisException(errorMessage, e); - } - } catch (Exception e) { - LOG.warn("failed to parse columns header, sql={}", columnsSQL, e); - throw new UserException("parse columns header failed", e); - } - - for (ImportColumnDesc columnDesc : columnsStmt.getColumns()) { + if (streamLoadTask.getColumnExprDesc() != null && !streamLoadTask.getColumnExprDesc().isEmpty()) { + for (ImportColumnDesc importColumnDesc : streamLoadTask.getColumnExprDesc()) { // make column name case match with real column name - String realColName = dstTable.getColumn(columnDesc.getColumn()) == null ? columnDesc.getColumn() - : dstTable.getColumn(columnDesc.getColumn()).getName(); - if (columnDesc.getExpr() != null) { - exprsByName.put(realColName, columnDesc.getExpr()); + String columnName = importColumnDesc.getColumnName(); + String realColName = dstTable.getColumn(columnName) == null ? columnName + : dstTable.getColumn(columnName).getName(); + if (importColumnDesc.getExpr() != null) { + exprsByName.put(realColName, importColumnDesc.getExpr()); } else { SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); slotDesc.setIsMaterialized(true); - slotDesc.setIsNullable(false); + // ISSUE A: src slot should be nullable even if the column is not nullable. + // because src slot is what we read from file, not represent to real column value. + // If column is not nullable, error will be thrown when filling the dest slot, + // which is not nullable + slotDesc.setIsNullable(true); params.addToSrc_slot_ids(slotDesc.getId().asInt()); slotDescByName.put(realColName, slotDesc); } @@ -195,7 +172,8 @@ public void init(Analyzer analyzer) throws UserException { SlotDescriptor slotDesc = analyzer.getDescTbl().addSlotDescriptor(srcTupleDesc); slotDesc.setType(ScalarType.createType(PrimitiveType.VARCHAR)); slotDesc.setIsMaterialized(true); - slotDesc.setIsNullable(false); + // same as ISSUE A + slotDesc.setIsNullable(true); params.addToSrc_slot_ids(slotDesc.getId().asInt()); slotDescByName.put(column.getName(), slotDesc); @@ -203,36 +181,14 @@ public void init(Analyzer analyzer) throws UserException { } // analyze where statement - if (request.isSetWhere()) { + if (streamLoadTask.getWhereExpr() != null) { Map dstDescMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); for (SlotDescriptor slotDescriptor : desc.getSlots()) { dstDescMap.put(slotDescriptor.getColumn().getName(), slotDescriptor); } - String whereSQL = new String("WHERE " + request.getWhere()); - SqlParser parser = new SqlParser(new SqlScanner(new StringReader(whereSQL))); - ImportWhereStmt whereStmt; - try { - whereStmt = (ImportWhereStmt) parser.parse().value; - } catch (Error e) { - LOG.warn("error happens when parsing where header, sql={}", whereSQL, e); - throw new AnalysisException("failed to parsing where header, maybe contain unsupported character"); - } catch (AnalysisException e) { - LOG.warn("analyze where statement failed, sql={}, error={}", - whereSQL, parser.getErrorMsg(whereSQL), e); - String errorMessage = parser.getErrorMsg(whereSQL); - if (errorMessage == null) { - throw e; - } else { - throw new AnalysisException(errorMessage, e); - } - } catch (Exception e) { - LOG.warn("failed to parse where header, sql={}", whereSQL, e); - throw new UserException("parse columns header failed", e); - } - // substitute SlotRef in filter expression - Expr whereExpr = whereStmt.getExpr(); + Expr whereExpr = streamLoadTask.getWhereExpr(); List slots = Lists.newArrayList(); whereExpr.collect(SlotRef.class, slots); @@ -258,8 +214,8 @@ public void init(Analyzer analyzer) throws UserException { computeStats(analyzer); createDefaultSmap(analyzer); - if (request.isSetColumnSeparator()) { - String sep = ColumnSeparator.convertSeparator(request.getColumnSeparator()); + if (streamLoadTask.getColumnSeparator() != null) { + String sep = streamLoadTask.getColumnSeparator().getColumnSeparator(); params.setColumn_separator(sep.getBytes(Charset.forName("UTF-8"))[0]); } else { params.setColumn_separator((byte) '\t'); @@ -324,7 +280,7 @@ private void finalizeParams() throws UserException { brokerScanRange.params.putToExpr_of_dest_slot(dstSlotDesc.getId().asInt(), expr.treeToThrift()); } brokerScanRange.params.setDest_tuple_id(desc.getId().asInt()); - LOG.info("brokerScanRange is {}", brokerScanRange); + // LOG.info("brokerScanRange is {}", brokerScanRange); // Need re compute memory layout after set some slot descriptor to nullable srcTupleDesc.computeMemLayout(); diff --git a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java index 342a1e5f2086b0..dfc90a54e68152 100644 --- a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -72,7 +72,7 @@ * Created by zhaochun on 14/11/10. */ public class DdlExecutor { - public static void execute(Catalog catalog, DdlStmt ddlStmt) throws DdlException, Exception { + public static void execute(Catalog catalog, DdlStmt ddlStmt, String origStmt) throws DdlException, Exception { if (ddlStmt instanceof CreateClusterStmt) { CreateClusterStmt stmt = (CreateClusterStmt) ddlStmt; catalog.createCluster(stmt); @@ -116,7 +116,7 @@ public static void execute(Catalog catalog, DdlStmt ddlStmt) throws DdlException } else if (ddlStmt instanceof CancelLoadStmt) { catalog.getLoadInstance().cancelLoadJob((CancelLoadStmt) ddlStmt); } else if (ddlStmt instanceof CreateRoutineLoadStmt) { - catalog.getRoutineLoadManager().addRoutineLoadJob((CreateRoutineLoadStmt) ddlStmt); + catalog.getRoutineLoadManager().createRoutineLoadJob((CreateRoutineLoadStmt) ddlStmt, origStmt); } else if (ddlStmt instanceof PauseRoutineLoadStmt) { catalog.getRoutineLoadManager().pauseRoutineLoadJob((PauseRoutineLoadStmt) ddlStmt); } else if (ddlStmt instanceof ResumeRoutineLoadStmt) { diff --git a/fe/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/src/main/java/org/apache/doris/qe/SessionVariable.java index fd8658bf1ac2a2..cd57eb8f101ebe 100644 --- a/fe/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -68,8 +68,12 @@ public class SessionVariable implements Serializable, Writable { public static final String DISABLE_STREAMING_PREAGGREGATIONS = "disable_streaming_preaggregations"; public static final String DISABLE_COLOCATE_JOIN = "disable_colocate_join"; public static final String PARALLEL_FRAGMENT_EXEC_INSTANCE_NUM = "parallel_fragment_exec_instance_num"; + public static final String ENABLE_INSERT_STRICT = "enable_insert_strict"; public static final int MIN_EXEC_INSTANCE_NUM = 1; public static final int MAX_EXEC_INSTANCE_NUM = 32; + public static final String MT_DOP = "mt_dop"; + // if set to true, some of stmt will be forwarded to master FE to get result + public static final String FORWARD_TO_MASTER = "forward_to_master"; // max memory used on every backend. @VariableMgr.VarAttr(name = EXEC_MEM_LIMIT) @@ -183,6 +187,12 @@ public class SessionVariable implements Serializable, Writable { @VariableMgr.VarAttr(name = PARALLEL_FRAGMENT_EXEC_INSTANCE_NUM) private int parallelExecInstanceNum = 1; + @VariableMgr.VarAttr(name = ENABLE_INSERT_STRICT) + private boolean enableInsertStrict = false; + + @VariableMgr.VarAttr(name = FORWARD_TO_MASTER) + private boolean forwardToMaster = false; + public long getMaxExecMemByte() { return maxExecMemByte; } @@ -428,8 +438,21 @@ public void setParallelExecInstanceNum(int parallelExecInstanceNum) { this.parallelExecInstanceNum = parallelExecInstanceNum; } } + + public boolean getEnableInsertStrict() { return enableInsertStrict; } + public void setEnableInsertStrict(boolean enableInsertStrict) { this.enableInsertStrict = enableInsertStrict; } + // Serialize to thrift object + public boolean getForwardToMaster() { + return forwardToMaster; + } + + public void setForwardToMaster(boolean forwardToMaster) { + this.forwardToMaster = forwardToMaster; + } + + // Serialize to thrift object TQueryOptions toThrift() { TQueryOptions tResult = new TQueryOptions(); tResult.setMem_limit(maxExecMemByte); diff --git a/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java index 546dff1fd8e9ce..532d8847120663 100644 --- a/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -17,6 +17,7 @@ package org.apache.doris.qe; +import com.google.common.base.Strings; import org.apache.doris.analysis.AdminShowConfigStmt; import org.apache.doris.analysis.AdminShowReplicaDistributionStmt; import org.apache.doris.analysis.AdminShowReplicaStatusStmt; @@ -50,6 +51,7 @@ import org.apache.doris.analysis.ShowRolesStmt; import org.apache.doris.analysis.ShowRollupStmt; import org.apache.doris.analysis.ShowRoutineLoadStmt; +import org.apache.doris.analysis.ShowRoutineLoadTaskStmt; import org.apache.doris.analysis.ShowSnapshotStmt; import org.apache.doris.analysis.ShowStmt; import org.apache.doris.analysis.ShowTableStatusStmt; @@ -80,6 +82,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.PatternMatcher; import org.apache.doris.common.proc.BackendsProcDir; import org.apache.doris.common.proc.FrontendsProcNode; @@ -87,6 +90,8 @@ import org.apache.doris.common.proc.PartitionsProcDir; import org.apache.doris.common.proc.ProcNodeInterface; import org.apache.doris.common.proc.TabletsProcDir; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; import org.apache.doris.load.ExportJob; import org.apache.doris.load.ExportMgr; import org.apache.doris.load.Load; @@ -168,6 +173,8 @@ public ShowResultSet execute() throws AnalysisException { handleShowLoadWarnings(); } else if (stmt instanceof ShowRoutineLoadStmt) { handleShowRoutineLoad(); + } else if (stmt instanceof ShowRoutineLoadTaskStmt) { + handleShowRoutineLoadTask(); } else if (stmt instanceof ShowDeleteStmt) { handleShowDelete(); } else if (stmt instanceof ShowAlterStmt) { @@ -793,37 +800,98 @@ private void handleShowLoadWarningsFromURL(ShowLoadWarningsStmt showWarningsStmt private void handleShowRoutineLoad() throws AnalysisException { ShowRoutineLoadStmt showRoutineLoadStmt = (ShowRoutineLoadStmt) stmt; + List> rows = Lists.newArrayList(); + // if job exists + List routineLoadJobList; + try { + routineLoadJobList = Catalog.getCurrentCatalog().getRoutineLoadManager() + .getJob(showRoutineLoadStmt.getDbFullName(), + showRoutineLoadStmt.getName(), + showRoutineLoadStmt.isIncludeHistory()); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage(), e); + throw new AnalysisException(e.getMessage()); + } + + if (routineLoadJobList != null) { + String dbFullName = showRoutineLoadStmt.getDbFullName(); + String tableName = null; + for (RoutineLoadJob routineLoadJob : routineLoadJobList) { + // check auth + try { + tableName = routineLoadJob.getTableName(); + } catch (MetaNotFoundException e) { + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("error_msg", "The table metadata of job has been changed. " + + "The job will be cancelled automatically") + .build(), e); + } + if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), + dbFullName, + tableName, + PrivPredicate.LOAD)) { + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) + .add("operator", "show routine load job") + .add("user", ConnectContext.get().getQualifiedUser()) + .add("remote_ip", ConnectContext.get().getRemoteIP()) + .add("db_full_name", dbFullName) + .add("table_name", tableName) + .add("error_msg", "The table access denied")); + continue; + } + + // get routine load info + rows.add(routineLoadJob.getShowInfo()); + } + } + + if (!Strings.isNullOrEmpty(showRoutineLoadStmt.getName()) && rows.size() == 0) { + // if the jobName has been specified + throw new AnalysisException("There is no job named " + showRoutineLoadStmt.getName() + + " in db " + showRoutineLoadStmt.getDbFullName() + + " include history " + showRoutineLoadStmt.isIncludeHistory()); + } + resultSet = new ShowResultSet(showRoutineLoadStmt.getMetaData(), rows); + } + + private void handleShowRoutineLoadTask() throws AnalysisException { + ShowRoutineLoadTaskStmt showRoutineLoadTaskStmt = (ShowRoutineLoadTaskStmt) stmt; + List> rows = Lists.newArrayList(); // if job exists - RoutineLoadJob routineLoadJob = - Catalog.getCurrentCatalog().getRoutineLoadManager().getJobByName(showRoutineLoadStmt.getName()); + RoutineLoadJob routineLoadJob; + try { + routineLoadJob = Catalog.getCurrentCatalog().getRoutineLoadManager().getJob(showRoutineLoadTaskStmt.getDbFullName(), + showRoutineLoadTaskStmt.getJobName()); + } catch (MetaNotFoundException e) { + LOG.warn(e.getMessage(), e); + throw new AnalysisException(e.getMessage()); + } if (routineLoadJob == null) { - throw new AnalysisException("There is no routine load job with id " + showRoutineLoadStmt.getName()); + throw new AnalysisException("The job named " + showRoutineLoadTaskStmt.getJobName() + "does not exists " + + "or job state is stopped or cancelled"); } // check auth + String dbFullName = showRoutineLoadTaskStmt.getDbFullName(); + String tableName; + try { + tableName = routineLoadJob.getTableName(); + } catch (MetaNotFoundException e) { + throw new AnalysisException("The table metadata of job has been changed. The job will be cancelled automatically", e); + } if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), - routineLoadJob.getDbFullName(), - routineLoadJob.getTableName(), + dbFullName, + tableName, PrivPredicate.LOAD)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - routineLoadJob.getTableName()); + tableName); } - // get routine load info - List> rows = Lists.newArrayList(); - List row = Lists.newArrayList(); - row.add(routineLoadJob.getId()); - row.add(routineLoadJob.getName()); - row.add(String.valueOf(routineLoadJob.getDbId())); - row.add(String.valueOf(routineLoadJob.getTableId())); - row.add(routineLoadJob.getPartitions()); - row.add(routineLoadJob.getState().name()); - row.add(routineLoadJob.getDesiredConcurrentNumber()); - row.add(routineLoadJob.getProgress().toString()); - - resultSet = new ShowResultSet(showRoutineLoadStmt.getMetaData(), rows); + // get routine load task info + rows.addAll(routineLoadJob.getTasksShowInfo()); + resultSet = new ShowResultSet(showRoutineLoadTaskStmt.getMetaData(), rows); } // Show user property statement diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index 35913effc07b8b..2a31b042acd858 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -17,10 +17,6 @@ package org.apache.doris.qe; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.CreateTableAsSelectStmt; import org.apache.doris.analysis.DdlStmt; @@ -42,8 +38,8 @@ import org.apache.doris.analysis.UseStmt; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Table.TableType; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; @@ -53,11 +49,11 @@ import org.apache.doris.common.ErrorReport; import org.apache.doris.common.NotImplementedException; import org.apache.doris.common.UserException; +import org.apache.doris.common.Version; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.ProfileManager; import org.apache.doris.common.util.RuntimeProfile; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.common.Version; import org.apache.doris.mysql.MysqlChannel; import org.apache.doris.mysql.MysqlEofPacket; import org.apache.doris.mysql.MysqlSerializer; @@ -66,11 +62,18 @@ import org.apache.doris.rewrite.ExprRewriter; import org.apache.doris.rpc.PQueryStatistics; import org.apache.doris.rpc.RpcException; +import org.apache.doris.task.LoadEtlTask; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TQueryOptions; +import org.apache.doris.thrift.TQueryType; import org.apache.doris.thrift.TResultBatch; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TabletCommitInfo; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -586,6 +589,7 @@ private void handleInsertStmt() throws Exception { context.setQueryId(new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits())); coord = new Coordinator(context, analyzer, planner); + coord.setQueryType(TQueryType.LOAD); QeProcessorImpl.INSTANCE.registerQuery(context.queryId(), coord); @@ -611,6 +615,15 @@ private void handleInsertStmt() throws Exception { LOG.info("delta files is {}", coord.getDeltaUrls()); + if (context.getSessionVariable().getEnableInsertStrict()) { + Map counters = coord.getLoadCounters(); + String strValue = counters.get(LoadEtlTask.DPP_ABNORMAL_ALL); + if (strValue != null && Long.valueOf(strValue) > 0) { + throw new UserException("Insert has filtered data in strict mode, tracking_url=" + + coord.getTrackingUrl()); + } + } + if (insertStmt.getTargetTable().getType() != TableType.OLAP) { // no need to add load job. // mysql table is already being inserted. @@ -750,7 +763,7 @@ private void handleExplainStmt(String result) throws IOException { private void handleDdlStmt() { try { - DdlExecutor.execute(context.getCatalog(), (DdlStmt) parsedStmt); + DdlExecutor.execute(context.getCatalog(), (DdlStmt) parsedStmt, originStmt); context.getState().setOk(); } catch (UserException e) { // Return message to info client what happened. diff --git a/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 068d0de45916c3..959c05e776b32c 100644 --- a/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -47,6 +47,7 @@ import org.apache.doris.qe.VariableMgr; import org.apache.doris.system.Frontend; import org.apache.doris.system.SystemInfoService; +import org.apache.doris.task.StreamLoadTask; import org.apache.doris.thrift.FrontendService; import org.apache.doris.thrift.FrontendServiceVersion; import org.apache.doris.thrift.TColumnDef; @@ -330,7 +331,8 @@ public TFetchResourceResult fetchResource() throws TException { @Override public TFeResult miniLoad(TMiniLoadRequest request) throws TException { - LOG.info("mini load request is {}", request); + LOG.info("receive mini load request: label: {}, db: {}, tbl: {}, backend: {}", + request.getLabel(), request.getDb(), request.getTbl(), request.getBackend()); ConnectContext context = new ConnectContext(null); String cluster = SystemInfoService.DEFAULT_CLUSTER; @@ -507,7 +509,7 @@ private void checkPasswordAndPrivs(String cluster, String user, String passwd, S @Override public TFeResult loadCheck(TLoadCheckRequest request) throws TException { - LOG.info("load check request. label: {}, user: {}, ip: {}", + LOG.info("receive load check request. label: {}, user: {}, ip: {}", request.getLabel(), request.getUser(), request.getUser_ip()); TStatus status = new TStatus(TStatusCode.OK); @@ -536,22 +538,21 @@ public TFeResult loadCheck(TLoadCheckRequest request) throws TException { @Override public TLoadTxnBeginResult loadTxnBegin(TLoadTxnBeginRequest request) throws TException { - TNetworkAddress clientAddr = getClientAddr(); - - LOG.info("receive loadTxnBegin request, db: {}, tbl: {}, label: {}, backend: {}", - request.getDb(), request.getTbl(), request.getLabel(), - clientAddr == null ? "unknown" : clientAddr.getHostname()); + String clientAddr = getClientAddrAsString(); + LOG.info("receive txn begin request, db: {}, tbl: {}, label: {}, backend: {}", + request.getDb(), request.getTbl(), request.getLabel(), clientAddr); LOG.debug("txn begin request: {}", request); TLoadTxnBeginResult result = new TLoadTxnBeginResult(); TStatus status = new TStatus(TStatusCode.OK); result.setStatus(status); try { - result.setTxnId(loadTxnBeginImpl(request)); + result.setTxnId(loadTxnBeginImpl(request, clientAddr)); } catch (LabelAlreadyUsedException e) { status.setStatus_code(TStatusCode.LABEL_ALREADY_EXISTS); status.addToError_msgs(e.getMessage()); } catch (UserException e) { + LOG.warn("failed to begin: {}", e.getMessage()); status.setStatus_code(TStatusCode.ANALYSIS_ERROR); status.addToError_msgs(e.getMessage()); } catch (Throwable e) { @@ -560,11 +561,10 @@ public TLoadTxnBeginResult loadTxnBegin(TLoadTxnBeginRequest request) throws TEx status.addToError_msgs(Strings.nullToEmpty(e.getMessage())); return result; } - return result; } - private long loadTxnBeginImpl(TLoadTxnBeginRequest request) throws UserException { + private long loadTxnBeginImpl(TLoadTxnBeginRequest request, String clientIp) throws UserException { String cluster = request.getCluster(); if (Strings.isNullOrEmpty(cluster)) { cluster = SystemInfoService.DEFAULT_CLUSTER; @@ -588,16 +588,22 @@ private long loadTxnBeginImpl(TLoadTxnBeginRequest request) throws UserException } throw new UserException("unknown database, database=" + dbName); } + // begin long timestamp = request.isSetTimestamp() ? request.getTimestamp() : -1; + long timeoutSecond = request.isSetTimeout() ? request.getTimeout() : Config.stream_load_default_timeout_second; return Catalog.getCurrentGlobalTransactionMgr().beginTransaction( - db.getId(), request.getLabel(), timestamp, "streamLoad", - TransactionState.LoadJobSourceType.BACKEND_STREAMING, null); + db.getId(), request.getLabel(), timestamp, "BE: " + clientIp, + TransactionState.LoadJobSourceType.BACKEND_STREAMING, -1, timeoutSecond); } @Override public TLoadTxnCommitResult loadTxnCommit(TLoadTxnCommitRequest request) throws TException { - LOG.info("receive loadTxnCommit request, request={}", request); + String clientAddr = getClientAddrAsString(); + LOG.info("receive txn commit request. db: {}, tbl: {}, txn id: {}, backend: {}", + request.getDb(), request.getTbl(), request.getTxnId(), clientAddr); + LOG.debug("txn commit request: {}", request); + TLoadTxnCommitResult result = new TLoadTxnCommitResult(); TStatus status = new TStatus(TStatusCode.OK); result.setStatus(status); @@ -608,6 +614,7 @@ public TLoadTxnCommitResult loadTxnCommit(TLoadTxnCommitRequest request) throws status.addToError_msgs("transaction commit successfully, BUT data will be visible later"); } } catch (UserException e) { + LOG.warn("failed to commit: {}", e.getMessage()); status.setStatus_code(TStatusCode.ANALYSIS_ERROR); status.addToError_msgs(e.getMessage()); } catch (Throwable e) { @@ -626,8 +633,12 @@ private boolean loadTxnCommitImpl(TLoadTxnCommitRequest request) throws UserExce cluster = SystemInfoService.DEFAULT_CLUSTER; } - checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), - request.getTbl(), request.getUser_ip(), PrivPredicate.LOAD); + if (request.isSetAuth_code()) { + // TODO(cmy): find a way to check + } else { + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), + request.getTbl(), request.getUser_ip(), PrivPredicate.LOAD); + } // get database Catalog catalog = Catalog.getInstance(); @@ -649,7 +660,10 @@ private boolean loadTxnCommitImpl(TLoadTxnCommitRequest request) throws UserExce @Override public TLoadTxnRollbackResult loadTxnRollback(TLoadTxnRollbackRequest request) throws TException { - LOG.info("receive loadTxnRollback request, request={}", request); + String clientAddr = getClientAddrAsString(); + LOG.info("receive txn rollback request. db: {}, tbl: {}, txn id: {}, reason: {}, backend: {}", + request.getDb(), request.getTbl(), request.getTxnId(), request.getReason(), clientAddr); + LOG.debug("txn rollback request: {}", request); TLoadTxnRollbackResult result = new TLoadTxnRollbackResult(); TStatus status = new TStatus(TStatusCode.OK); @@ -657,6 +671,7 @@ public TLoadTxnRollbackResult loadTxnRollback(TLoadTxnRollbackRequest request) t try { loadTxnRollbackImpl(request); } catch (UserException e) { + LOG.warn("failed to rollback: {}", e.getMessage()); status.setStatus_code(TStatusCode.ANALYSIS_ERROR); status.addToError_msgs(e.getMessage()); } catch (Throwable e) { @@ -675,16 +690,24 @@ private void loadTxnRollbackImpl(TLoadTxnRollbackRequest request) throws UserExc cluster = SystemInfoService.DEFAULT_CLUSTER; } - checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), - request.getTbl(), request.getUser_ip(), PrivPredicate.LOAD); + if (request.isSetAuth_code()) { + // TODO(cmy): find a way to check + } else { + checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), + request.getTbl(), request.getUser_ip(), PrivPredicate.LOAD); + } Catalog.getCurrentGlobalTransactionMgr().abortTransaction(request.getTxnId(), - request.isSetReason() ? request.getReason() : "system cancel"); + request.isSetReason() ? request.getReason() : "system cancel", + TxnCommitAttachment.fromThrift(request.getTxnCommitAttachment())); } @Override public TStreamLoadPutResult streamLoadPut(TStreamLoadPutRequest request) throws TException { - LOG.info("receive streamLoadPut request, request={}", request); + String clientAddr = getClientAddrAsString(); + LOG.info("receive stream load put request. db:{}, tbl: {}, txn id: {}, backend: {}", + request.getDb(), request.getTbl(), request.getTxnId(), clientAddr); + LOG.debug("stream load put request: {}", request); TStreamLoadPutResult result = new TStreamLoadPutResult(); TStatus status = new TStatus(TStatusCode.OK); @@ -692,6 +715,7 @@ public TStreamLoadPutResult streamLoadPut(TStreamLoadPutRequest request) throws try { result.setParams(streamLoadPutImpl(request)); } catch (UserException e) { + LOG.warn("failed to get stream load plan: {}", e.getMessage()); status.setStatus_code(TStatusCode.ANALYSIS_ERROR); status.addToError_msgs(e.getMessage()); } catch (Throwable e) { @@ -719,6 +743,7 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) } throw new UserException("unknown database, database=" + dbName); } + db.readLock(); try { Table table = db.getTable(request.getTbl()); @@ -728,7 +753,7 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) if (!(table instanceof OlapTable)) { throw new UserException("load table type is not OlapTable, type=" + table.getClass()); } - StreamLoadPlanner planner = new StreamLoadPlanner(db, (OlapTable) table, request); + StreamLoadPlanner planner = new StreamLoadPlanner(db, (OlapTable) table, StreamLoadTask.fromTStreamLoadPutRequest(request)); return planner.plan(); } finally { db.readUnlock(); @@ -752,5 +777,11 @@ private TNetworkAddress getClientAddr() { } return null; } + + private String getClientAddrAsString() { + TNetworkAddress addr = getClientAddr(); + return addr == null ? "unknown" : addr.hostname + ":" + addr.port; + } } + diff --git a/fe/src/main/java/org/apache/doris/system/Backend.java b/fe/src/main/java/org/apache/doris/system/Backend.java index 2c50c22ea49bdc..63c46f7132c904 100644 --- a/fe/src/main/java/org/apache/doris/system/Backend.java +++ b/fe/src/main/java/org/apache/doris/system/Backend.java @@ -314,6 +314,20 @@ public long getDataUsedCapacityB() { return dataUsedCapacityB; } + public double getMaxDiskUsedPct() { + ImmutableMap disks = disksRef.get(); + double maxPct = 0.0; + for (DiskInfo diskInfo : disks.values()) { + if (diskInfo.getState() == DiskState.ONLINE) { + double percent = diskInfo.getUsedPct(); + if (percent > maxPct) { + maxPct = percent; + } + } + } + return maxPct; + } + public String getPathByPathHash(long pathHash) { for (DiskInfo diskInfo : disksRef.get().values()) { if (diskInfo.getPathHash() == pathHash) { diff --git a/fe/src/main/java/org/apache/doris/task/LoadPendingTask.java b/fe/src/main/java/org/apache/doris/task/LoadPendingTask.java index 711de8c3f967cf..c46eca8e4a0888 100644 --- a/fe/src/main/java/org/apache/doris/task/LoadPendingTask.java +++ b/fe/src/main/java/org/apache/doris/task/LoadPendingTask.java @@ -25,6 +25,7 @@ import org.apache.doris.load.LoadChecker; import org.apache.doris.load.LoadJob; import org.apache.doris.load.LoadJob.JobState; +import org.apache.doris.service.FrontendOptions; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.transaction.TransactionState.LoadJobSourceType; @@ -75,9 +76,8 @@ protected void exec() { // create etl request and make some guarantee for schema change and rollup if (job.getTransactionId() < 0) { long transactionId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction(dbId, - job.getLabel(), - "fe", - LoadJobSourceType.FRONTEND); + job.getLabel(), "FE: " + FrontendOptions.getLocalHostAddress(), LoadJobSourceType.FRONTEND, + job.getTimeoutSecond()); job.setTransactionId(transactionId); } createEtlRequest(); diff --git a/fe/src/main/java/org/apache/doris/task/RoutineLoadTask.java b/fe/src/main/java/org/apache/doris/task/RoutineLoadTask.java deleted file mode 100644 index 5e1decde1bfbc3..00000000000000 --- a/fe/src/main/java/org/apache/doris/task/RoutineLoadTask.java +++ /dev/null @@ -1,53 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.task; - -import org.apache.doris.catalog.Catalog; -import org.apache.doris.load.routineload.LoadDataSourceType; -import org.apache.doris.common.LoadException; -import org.apache.doris.load.RoutineLoadDesc; -import org.apache.doris.thrift.TResourceInfo; -import org.apache.doris.thrift.TTaskType; - -public class RoutineLoadTask extends AgentTask { - - private String id; - private long txnId; - private RoutineLoadDesc routineLoadDesc; - private LoadDataSourceType dataSourceType; - - - public RoutineLoadTask(TResourceInfo resourceInfo, long backendId, long dbId, long tableId, String id, - LoadDataSourceType dataSourceType, long txnId) { - super(resourceInfo, backendId, TTaskType.STREAM_LOAD, dbId, tableId, 0L, 0L, 0L, - Catalog.getCurrentCatalog().getNextId()); - this.id = id; - this.txnId = txnId; - this.dataSourceType = dataSourceType; - } - - public void setRoutineLoadDesc(RoutineLoadDesc routineLoadDesc) throws LoadException { - if (this.routineLoadDesc != null) { - throw new LoadException("Column separator has been initialized"); - } - this.routineLoadDesc = new RoutineLoadDesc(routineLoadDesc.getColumnSeparator(), - routineLoadDesc.getColumnsInfo(), - routineLoadDesc.getWherePredicate(), - null); - } -} diff --git a/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java b/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java new file mode 100644 index 00000000000000..c6ce9bcf5546dc --- /dev/null +++ b/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.task; + +import org.apache.doris.analysis.ColumnSeparator; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.ImportColumnDesc; +import org.apache.doris.analysis.ImportColumnsStmt; +import org.apache.doris.analysis.ImportWhereStmt; +import org.apache.doris.analysis.SqlParser; +import org.apache.doris.analysis.SqlScanner; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.thrift.TFileFormatType; +import org.apache.doris.thrift.TFileType; +import org.apache.doris.thrift.TStreamLoadPutRequest; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Joiner; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.StringReader; +import java.util.List; + +public class StreamLoadTask { + + private static final Logger LOG = LogManager.getLogger(StreamLoadTask.class); + + private TUniqueId id; + private long txnId; + private TFileType fileType; + private TFileFormatType formatType; + + // optional + private List columnExprDesc; + private Expr whereExpr; + private ColumnSeparator columnSeparator; + private String partitions; + private String path; + + public StreamLoadTask(TUniqueId id, long txnId, TFileType fileType, TFileFormatType formatType) { + this.id = id; + this.txnId = txnId; + this.fileType = fileType; + this.formatType = formatType; + } + + public TUniqueId getId() { + return id; + } + + public long getTxnId() { + return txnId; + } + + public TFileType getFileType() { + return fileType; + } + + public TFileFormatType getFormatType() { + return formatType; + } + + public List getColumnExprDesc() { + return columnExprDesc; + } + + public Expr getWhereExpr() { + return whereExpr; + } + + public ColumnSeparator getColumnSeparator() { + return columnSeparator; + } + + public String getPartitions() { + return partitions; + } + + public String getPath() { + return path; + } + + public static StreamLoadTask fromTStreamLoadPutRequest(TStreamLoadPutRequest request) throws UserException { + StreamLoadTask streamLoadTask = new StreamLoadTask(request.getLoadId(), request.getTxnId(), + request.getFileType(), request.getFormatType()); + streamLoadTask.setOptionalFromTSLPutRequest(request); + return streamLoadTask; + } + + private void setOptionalFromTSLPutRequest(TStreamLoadPutRequest request) throws UserException { + if (request.isSetColumns()) { + setColumnToColumnExpr(request.getColumns()); + } + if (request.isSetWhere()) { + setWhereExpr(request.getWhere()); + } + if (request.isSetColumnSeparator()) { + setColumnSeparator(request.getColumnSeparator()); + } + if (request.isSetPartitions()) { + partitions = request.getPartitions(); + } + switch (request.getFileType()) { + case FILE_STREAM: + path = request.getPath(); + break; + default: + throw new UserException("unsupported file type, type=" + request.getFileType()); + } + } + + public static StreamLoadTask fromRoutineLoadJob(RoutineLoadJob routineLoadJob) { + TUniqueId dummyId = new TUniqueId(); + StreamLoadTask streamLoadTask = new StreamLoadTask(dummyId, -1L /* dummy txn id*/, + TFileType.FILE_STREAM, TFileFormatType.FORMAT_CSV_PLAIN); + streamLoadTask.setOptionalFromRoutineLoadJob(routineLoadJob); + return streamLoadTask; + } + + private void setOptionalFromRoutineLoadJob(RoutineLoadJob routineLoadJob) { + columnExprDesc = routineLoadJob.getColumnDescs(); + whereExpr = routineLoadJob.getWhereExpr(); + columnSeparator = routineLoadJob.getColumnSeparator(); + partitions = routineLoadJob.getPartitions() == null ? null : Joiner.on(",").join(routineLoadJob.getPartitions()); + } + + private void setColumnToColumnExpr(String columns) throws UserException { + String columnsSQL = new String("COLUMNS (" + columns + ")"); + SqlParser parser = new SqlParser(new SqlScanner(new StringReader(columnsSQL))); + ImportColumnsStmt columnsStmt; + try { + columnsStmt = (ImportColumnsStmt) parser.parse().value; + } catch (Error e) { + LOG.warn("error happens when parsing columns, sql={}", columnsSQL, e); + throw new AnalysisException("failed to parsing columns' header, maybe contain unsupported character"); + } catch (AnalysisException e) { + LOG.warn("analyze columns' statement failed, sql={}, error={}", + columnsSQL, parser.getErrorMsg(columnsSQL), e); + String errorMessage = parser.getErrorMsg(columnsSQL); + if (errorMessage == null) { + throw e; + } else { + throw new AnalysisException(errorMessage, e); + } + } catch (Exception e) { + LOG.warn("failed to parse columns header, sql={}", columnsSQL, e); + throw new UserException("parse columns header failed", e); + } + + if (columnsStmt.getColumns() != null && !columnsStmt.getColumns().isEmpty()) { + columnExprDesc = columnsStmt.getColumns(); + } + } + + private void setWhereExpr(String whereString) throws UserException { + String whereSQL = new String("WHERE " + whereString); + SqlParser parser = new SqlParser(new SqlScanner(new StringReader(whereSQL))); + ImportWhereStmt whereStmt; + try { + whereStmt = (ImportWhereStmt) parser.parse().value; + } catch (Error e) { + LOG.warn("error happens when parsing where header, sql={}", whereSQL, e); + throw new AnalysisException("failed to parsing where header, maybe contain unsupported character"); + } catch (AnalysisException e) { + LOG.warn("analyze where statement failed, sql={}, error={}", + whereSQL, parser.getErrorMsg(whereSQL), e); + String errorMessage = parser.getErrorMsg(whereSQL); + if (errorMessage == null) { + throw e; + } else { + throw new AnalysisException(errorMessage, e); + } + } catch (Exception e) { + LOG.warn("failed to parse where header, sql={}", whereSQL, e); + throw new UserException("parse columns header failed", e); + } + whereExpr = whereStmt.getExpr(); + } + + private void setColumnSeparator(String oriSeparator) throws AnalysisException { + columnSeparator = new ColumnSeparator(oriSeparator); + columnSeparator.analyze(); + } +} diff --git a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index b022debb86e0f3..63e0f7391de8fa 100644 --- a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -27,7 +27,6 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.Replica; -import org.apache.doris.catalog.Table; import org.apache.doris.catalog.Tablet; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.AnalysisException; @@ -39,13 +38,14 @@ import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; import org.apache.doris.load.Load; -import org.apache.doris.load.TxnStateChangeListener; +import org.apache.doris.metric.MetricRepo; import org.apache.doris.persist.EditLog; import org.apache.doris.task.AgentTaskQueue; import org.apache.doris.task.PublishVersionTask; import org.apache.doris.thrift.TTaskType; import org.apache.doris.transaction.TransactionState.LoadJobSourceType; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Lists; @@ -88,9 +88,10 @@ public class GlobalTransactionMgr { private com.google.common.collect.Table dbIdToTxnLabels; private Map runningTxnNums; private TransactionIdGenerator idGenerator; + private TxnStateCallbackFactory callbackFactory = new TxnStateCallbackFactory(); private Catalog catalog; - + public GlobalTransactionMgr(Catalog catalog) { idToTransactionState = new HashMap<>(); dbIdToTxnLabels = HashBasedTable.create(); @@ -99,16 +100,20 @@ public GlobalTransactionMgr(Catalog catalog) { this.idGenerator = new TransactionIdGenerator(); } - public long beginTransaction(long dbId, String label, String coordinator, LoadJobSourceType sourceType) - throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException { - return beginTransaction(dbId, label, -1, coordinator, sourceType, null); + public TxnStateCallbackFactory getCallbackFactory() { + return callbackFactory; + } + + public long beginTransaction(long dbId, String label, String coordinator, LoadJobSourceType sourceType, + long timeoutSecond) throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException { + return beginTransaction(dbId, label, -1, coordinator, sourceType, -1, timeoutSecond); } /** * the app could specify the transaction id * * timestamp is used to judge that whether the request is a internal retry request - * if label already exist, and timestamps are equal, we return the exist tid, and consider this 'begin' + * if label already exist, and timestamp are equal, we return the exist tid, and consider this 'begin' * as success. * timestamp == -1 is for compatibility * @@ -117,14 +122,20 @@ public long beginTransaction(long dbId, String label, String coordinator, LoadJo * @throws IllegalTransactionParameterException */ public long beginTransaction(long dbId, String label, long timestamp, - String coordinator, LoadJobSourceType sourceType, - TxnStateChangeListener txnStateChangeListener) + String coordinator, LoadJobSourceType sourceType, long listenerId, long timeoutSecond) throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException { if (Config.disable_load_job) { throw new BeginTransactionException("disable_load_job is set to true, all load jobs are prevented"); } + if (timeoutSecond > Config.max_stream_load_timeout_second || + timeoutSecond < Config.min_stream_load_timeout_second) { + throw new AnalysisException("Invalid timeout. Timeout should between " + + Config.min_stream_load_timeout_second + " and " + Config.max_stream_load_timeout_second + + " seconds"); + } + writeLock(); try { Preconditions.checkNotNull(coordinator); @@ -140,20 +151,24 @@ public long beginTransaction(long dbId, String label, long timestamp, return txnLabels.get(label); } } - throw new LabelAlreadyUsedException(label); } if (runningTxnNums.get(dbId) != null && runningTxnNums.get(dbId) > Config.max_running_txn_num_per_db) { throw new BeginTransactionException("current running txns on db " + dbId + " is " - + runningTxnNums.get(dbId) + ", larger than limit " + Config.max_running_txn_num_per_db); + + runningTxnNums.get(dbId) + ", larger than limit " + Config.max_running_txn_num_per_db); } long tid = idGenerator.getNextTransactionId(); LOG.info("begin transaction: txn id {} with label {} from coordinator {}", tid, label, coordinator); TransactionState transactionState = new TransactionState(dbId, tid, label, timestamp, sourceType, - coordinator, txnStateChangeListener); + coordinator, listenerId, timeoutSecond * 1000); transactionState.setPrepareTime(System.currentTimeMillis()); unprotectUpsertTransactionState(transactionState); + + if (MetricRepo.isInit.get()) { + MetricRepo.COUNTER_TXN_SUCCESS.increase(1L); + } + return tid; } finally { writeUnlock(); @@ -192,7 +207,7 @@ public void deleteTransaction(long transactionId) { } public void commitTransaction(long dbId, long transactionId, List tabletCommitInfos) - throws MetaNotFoundException, TransactionException { + throws UserException { commitTransaction(dbId, transactionId, tabletCommitInfos, null); } @@ -208,14 +223,14 @@ public void commitTransaction(long dbId, long transactionId, List tabletCommitInfos, TxnCommitAttachment txnCommitAttachment) - throws MetaNotFoundException, TransactionException { + throws UserException { if (Config.disable_load_job) { throw new TransactionCommitFailedException("disable_load_job is set to true, all load jobs are prevented"); } @@ -352,57 +367,37 @@ public void commitTransaction(long dbId, long transactionId, List tabletCommitInfos, long timeoutMillis) - throws MetaNotFoundException, TransactionException { + throws UserException { return commitAndPublishTransaction(db, transactionId, tabletCommitInfos, timeoutMillis, null); } public boolean commitAndPublishTransaction(Database db, long transactionId, List tabletCommitInfos, long timeoutMillis, TxnCommitAttachment txnCommitAttachment) - throws MetaNotFoundException, TransactionException { + throws UserException { db.writeLock(); try { commitTransaction(db.getId(), transactionId, tabletCommitInfos, txnCommitAttachment); @@ -432,47 +427,58 @@ public boolean commitAndPublishTransaction(Database db, long transactionId, } return transactionState.getTransactionStatus() == TransactionStatus.VISIBLE; } - - public void abortTransaction(long transactionId, String reason) throws UserException { - if (transactionId < 0) { - LOG.info("transaction id is {}, less than 0, maybe this is an old type load job, ignore abort operation", transactionId); - return; - } - writeLock(); - try { - unprotectAbortTransaction(transactionId, reason); - } catch (Exception exception) { - LOG.info("transaction:[{}] reason:[{}] abort failure exception:{}", transactionId, reason, exception); - throw exception; - } finally { - writeUnlock(); - } - return; - } - + public void abortTransaction(Long dbId, String label, String reason) throws UserException { Preconditions.checkNotNull(label); + Long transactionId = null; writeLock(); try { Map dbTxns = dbIdToTxnLabels.row(dbId); if (dbTxns == null) { throw new UserException("transaction not found, label=" + label); } - Long transactionId = dbTxns.get(label); + transactionId = dbTxns.get(label); if (transactionId == null) { throw new UserException("transaction not found, label=" + label); } - unprotectAbortTransaction(transactionId, reason); } finally { writeUnlock(); } + abortTransaction(transactionId, reason); + } + + public void abortTransaction(long transactionId, String reason) throws UserException { + abortTransaction(transactionId, reason, null); + } + + public void abortTransaction(long transactionId, String reason, TxnCommitAttachment txnCommitAttachment) throws UserException { + if (transactionId < 0) { + LOG.info("transaction id is {}, less than 0, maybe this is an old type load job, ignore abort operation", transactionId); + return; + } + TransactionState transactionState = idToTransactionState.get(transactionId); + if (transactionState == null) { + throw new UserException("transaction not found"); + } + // before state transform + transactionState.beforeStateTransform(TransactionStatus.ABORTED); + boolean txnOperated = false; + writeLock(); + try { + unprotectAbortTransaction(transactionId, reason, txnCommitAttachment); + txnOperated = true; + } finally { + writeUnlock(); + transactionState.afterStateTransform(TransactionStatus.ABORTED, txnOperated, reason); + } + return; } /* * get all txns which is ready to publish * a ready-to-publish txn's partition's visible version should be ONE less than txn's commit version. */ - public List getReadyToPublishTransactions() { + public List getReadyToPublishTransactions() throws UserException { List readyPublishTransactionState = new ArrayList<>(); List allCommittedTransactionState = null; writeLock(); @@ -490,8 +496,6 @@ public List getReadyToPublishTransactions() { continue; } } - } catch (TransactionException e) { - LOG.warn("failed to update transaction {} status to aborted", e.getTransactionId()); } finally { writeUnlock(); } @@ -571,8 +575,6 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) { LOG.warn("db is dropped during transaction, abort transaction {}", transactionState); unprotectUpsertTransactionState(transactionState); return; - } catch (TransactionException e) { - LOG.warn("failed to change transaction {} status to aborted", transactionState.getTransactionId()); } finally { writeUnlock(); } @@ -692,8 +694,6 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) { transactionState.setFinishTime(System.currentTimeMillis()); transactionState.setTransactionStatus(TransactionStatus.VISIBLE); unprotectUpsertTransactionState(transactionState); - } catch (TransactionException e) { - LOG.warn("failed to change transaction {} status to visible", transactionState.getTransactionId()); } finally { writeUnlock(); } @@ -706,8 +706,8 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) { } // check if there exists a load job before the endTransactionId have all finished - // load job maybe started but could not know the affected tableid, so that we not check by table - public boolean hasPreviousTransactionsFinished(long endTransactionId, long dbId) { + // load job maybe started but could not know the affected table id, so that we not check by table + public boolean isPreviousTransactionsFinished(long endTransactionId, long dbId) { readLock(); try { for (Map.Entry entry : idToTransactionState.entrySet()) { @@ -715,6 +715,8 @@ public boolean hasPreviousTransactionsFinished(long endTransactionId, long dbId) continue; } if (entry.getKey() <= endTransactionId) { + LOG.info("txn is still running: {}, checking end txn id: {}", + entry.getValue(), endTransactionId); return false; } } @@ -729,7 +731,11 @@ public boolean hasPreviousTransactionsFinished(long endTransactionId, long dbId) */ public void removeOldTransactions() { long currentMillis = System.currentTimeMillis(); - + + // TODO(cmy): the following 3 steps are no needed anymore, we can only use the last step to check + // the timeout txn. Because, now we set timeout for each txn same as timeout of their job's. + // But we keep the 1 and 2 step for compatibility. They should be deleted in 0.11.0 + // to avoid dead lock (transaction lock and load lock), we do this in 3 phases // 1. get all related db ids of txn in idToTransactionState Set dbIds = Sets.newHashSet(); @@ -746,8 +752,7 @@ public void removeOldTransactions() { // streaming insert stmt not add to fe load job, should use this method to // recycle the timeout insert stmt load job if (transactionState.getTransactionStatus() == TransactionStatus.PREPARE - && (currentMillis - transactionState.getPrepareTime()) - / 1000 > Config.stream_load_default_timeout_second) { + && currentMillis - transactionState.getPrepareTime() > transactionState.getTimeoutMs()) { dbIds.add(transactionState.getDbId()); } } @@ -755,7 +760,7 @@ public void removeOldTransactions() { } finally { readUnlock(); } - + // 2. get all load jobs' txn id of these databases Map> dbIdToTxnIds = Maps.newHashMap(); Load loadInstance = Catalog.getCurrentCatalog().getLoadInstance(); @@ -763,7 +768,7 @@ public void removeOldTransactions() { Set txnIds = loadInstance.getTxnIdsByDb(dbId); dbIdToTxnIds.put(dbId, txnIds); } - + // 3. use dbIdToTxnIds to remove old transactions, without holding load locks again writeLock(); try { @@ -784,17 +789,10 @@ public void removeOldTransactions() { // streaming insert stmt not add to fe load job, should use this method to // recycle the timeout insert stmt load job if (transactionState.getTransactionStatus() == TransactionStatus.PREPARE - && (currentMillis - transactionState.getPrepareTime()) / 1000 > Config.stream_load_default_timeout_second) { - if (transactionState.getSourceType() != LoadJobSourceType.FRONTEND - || !checkTxnHasRelatedJob(transactionState, dbIdToTxnIds)) { - try { - transactionState.setTransactionStatus(TransactionStatus.ABORTED, - TransactionState.TxnStatusChangeReason.TIMEOUT); - } catch (TransactionException e) { - LOG.warn("txn {} could not be aborted with error message {}", - transactionState.getTransactionId(), e.getMessage()); - continue; - } + && currentMillis - transactionState.getPrepareTime() > transactionState.getTimeoutMs()) { + if ((transactionState.getSourceType() != LoadJobSourceType.FRONTEND + || !checkTxnHasRelatedJob(transactionState, dbIdToTxnIds))) { + transactionState.setTransactionStatus(TransactionStatus.ABORTED); transactionState.setFinishTime(System.currentTimeMillis()); transactionState.setReason("transaction is timeout and is cancelled automatically"); unprotectUpsertTransactionState(transactionState); @@ -821,10 +819,10 @@ private boolean checkTxnHasRelatedJob(TransactionState txnState, Map errorReplicaIds, + Map> tableToPartition, Set totalInvolvedBackends, + Database db) { + // transaction state is modified during check if the transaction could committed + if (transactionState.getTransactionStatus() != TransactionStatus.PREPARE) { + return; + } + // update transaction state version + transactionState.setCommitTime(System.currentTimeMillis()); + transactionState.setTransactionStatus(TransactionStatus.COMMITTED); + transactionState.setErrorReplicas(errorReplicaIds); + for (long tableId : tableToPartition.keySet()) { + TableCommitInfo tableCommitInfo = new TableCommitInfo(tableId); + for (long partitionId : tableToPartition.get(tableId)) { + OlapTable table = (OlapTable) db.getTable(tableId); + Partition partition = table.getPartition(partitionId); + PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(partitionId, + partition.getNextVersion(), + partition.getNextVersionHash()); + tableCommitInfo.addPartitionCommitInfo(partitionCommitInfo); + } + transactionState.putIdToTableCommitInfo(tableId, tableCommitInfo); + } + // persist transactionState + unprotectUpsertTransactionState(transactionState); + + // add publish version tasks. set task to null as a placeholder. + // tasks will be created when publishing version. + for (long backendId : totalInvolvedBackends) { + transactionState.addPublishVersionTask(backendId, null); + } + } + + private void unprotectAbortTransaction(long transactionId, String reason, TxnCommitAttachment txnCommitAttachment) + throws UserException { TransactionState transactionState = idToTransactionState.get(transactionId); if (transactionState == null) { throw new UserException("transaction not found"); @@ -875,6 +914,10 @@ private void unprotectAbortTransaction(long transactionId, String reason) throws || transactionState.getTransactionStatus() == TransactionStatus.VISIBLE) { throw new UserException("transaction's state is already committed or visible, could not abort"); } + // update transaction state extra if exists + if (txnCommitAttachment != null) { + transactionState.setTxnCommitAttachment(txnCommitAttachment); + } transactionState.setFinishTime(System.currentTimeMillis()); transactionState.setReason(reason); transactionState.setTransactionStatus(TransactionStatus.ABORTED); @@ -889,6 +932,8 @@ private void unprotectAbortTransaction(long transactionId, String reason) throws public void replayUpsertTransactionState(TransactionState transactionState) { writeLock(); try { + // set transaction status will call txn state change listener + transactionState.replaySetTransactionStatus(); Database db = catalog.getDb(transactionState.getDbId()); if (transactionState.getTransactionStatus() == TransactionStatus.COMMITTED) { LOG.debug("replay a committed transaction {}", transactionState); @@ -1019,7 +1064,7 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat newVersion = replica.getVersion(); newVersionHash = replica.getVersionHash(); } - + // success version always move forward lastSucessVersion = newCommitVersion; lastSuccessVersionHash = newCommitVersionHash; @@ -1054,7 +1099,7 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat partition.updateVisibleVersionAndVersionHash(version, versionHash); if (LOG.isDebugEnabled()) { LOG.debug("transaction state {} set partition {}'s version to [{}] and version hash to [{}]", - transactionState, partition.getId(), version, versionHash); + transactionState, partition.getId(), version, versionHash); } } } @@ -1119,17 +1164,32 @@ public List> getDbInfo() { return infos; } - public List> getDbTransInfo(long dbId) throws AnalysisException { + public List> getDbTransStateInfo(long dbId) { + List> infos = Lists.newArrayList(); + readLock(); + try { + infos.add(Lists.newArrayList("running", String.valueOf(runningTxnNums.getOrDefault(dbId, 0)))); + long finishedNum = idToTransactionState.values().stream().filter( + t -> (t.getDbId() == dbId && t.getTransactionStatus().isFinalStatus())).count(); + infos.add(Lists.newArrayList("finished", String.valueOf(finishedNum))); + } finally { + readUnlock(); + } + return infos; + } + + public List> getDbTransInfo(long dbId, boolean running, int limit) throws AnalysisException { List> infos = new ArrayList>(); - int limit = 2000; readLock(); try { Database db = Catalog.getInstance().getDb(dbId); if (db == null) { throw new AnalysisException("Database[" + dbId + "] does not exist"); } + idToTransactionState.values().stream() - .filter(t -> t.getDbId() == dbId) + .filter(t -> (t.getDbId() == dbId && (running ? !t.getTransactionStatus().isFinalStatus() + : t.getTransactionStatus().isFinalStatus()))) .limit(limit) .forEach(t -> { List info = new ArrayList(); @@ -1143,6 +1203,8 @@ public List> getDbTransInfo(long dbId) throws AnalysisException info.add(TimeUtils.longToTimeString(t.getFinishTime())); info.add(t.getReason()); info.add(t.getErrorReplicas().size()); + info.add(t.getCallbackId()); + info.add(t.getTimeoutMs()); infos.add(info); }); } finally { @@ -1151,7 +1213,7 @@ public List> getDbTransInfo(long dbId) throws AnalysisException return infos; } - public List> getTableTransInfo(long tid, Database db) throws AnalysisException { + public List> getTableTransInfo(long tid) throws AnalysisException { List> tableInfos = new ArrayList>(); readLock(); try { @@ -1159,26 +1221,13 @@ public List> getTableTransInfo(long tid, Database db) throws An if (null == transactionState) { throw new AnalysisException("Transaction[" + tid + "] does not exist."); } - db.readLock(); - try { - for (long tableId : transactionState.getIdToTableCommitInfos().keySet()) { - List tableInfo = new ArrayList(); - Table table = db.getTable(tableId); - if (null == table) { - throw new AnalysisException("Table[" + tableId + "] does not exist."); - } - int partitionNum = 1; - if (table.getType() == Table.TableType.OLAP) { - OlapTable olapTable = (OlapTable) table; - tableInfo.add(table.getId()); - tableInfo.add(table.getName()); - tableInfo.add(partitionNum); - tableInfo.add(olapTable.getState()); - tableInfos.add(tableInfo); - } - } - } finally { - db.readUnlock(); + + for (Map.Entry entry : transactionState.getIdToTableCommitInfos().entrySet()) { + List tableInfo = new ArrayList(); + tableInfo.add(entry.getKey()); + tableInfo.add(Joiner.on(", ").join(entry.getValue().getIdToPartitionCommitInfo().values().stream().map( + e -> e.getPartitionId()).collect(Collectors.toList()))); + tableInfos.add(tableInfo); } } finally { readUnlock(); @@ -1186,7 +1235,7 @@ public List> getTableTransInfo(long tid, Database db) throws An return tableInfos; } - public List> getPartitionTransInfo(long tid, Database db, OlapTable olapTable) + public List> getPartitionTransInfo(long tid, long tableId) throws AnalysisException { List> partitionInfos = new ArrayList>(); readLock(); @@ -1195,24 +1244,14 @@ public List> getPartitionTransInfo(long tid, Database db, OlapT if (null == transactionState) { throw new AnalysisException("Transaction[" + tid + "] does not exist."); } - TableCommitInfo tableCommitInfo = transactionState.getIdToTableCommitInfos().get(olapTable.getId()); - db.readLock(); - try { - Map idToPartitionCommitInfo = tableCommitInfo.getIdToPartitionCommitInfo(); - for (long partitionId : idToPartitionCommitInfo.keySet()) { - Partition partition = olapTable.getPartition(partitionId); - List partitionInfo = new ArrayList(); - String partitionName = partition.getName(); - partitionInfo.add(partitionId); - partitionInfo.add(partitionName); - PartitionCommitInfo partitionCommitInfo = idToPartitionCommitInfo.get(partitionId); - partitionInfo.add(partitionCommitInfo.getVersion()); - partitionInfo.add(partitionCommitInfo.getVersionHash()); - partitionInfo.add(partition.getState()); - partitionInfos.add(partitionInfo); - } - } finally { - db.readUnlock(); + TableCommitInfo tableCommitInfo = transactionState.getIdToTableCommitInfos().get(tableId); + Map idToPartitionCommitInfo = tableCommitInfo.getIdToPartitionCommitInfo(); + for (Map.Entry entry : idToPartitionCommitInfo.entrySet()) { + List partitionInfo = new ArrayList(); + partitionInfo.add(entry.getKey()); + partitionInfo.add(entry.getValue().getVersion()); + partitionInfo.add(entry.getValue().getVersionHash()); + partitionInfos.add(partitionInfo); } } finally { readUnlock(); diff --git a/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java b/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java index 4eece8705ccae5..24ee3d27582a73 100644 --- a/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java +++ b/fe/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.common.Config; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.Daemon; import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentTaskExecutor; @@ -57,7 +58,7 @@ protected void runOneCycle() { } } - private void publishVersion() { + private void publishVersion() throws UserException { GlobalTransactionMgr globalTransactionMgr = Catalog.getCurrentGlobalTransactionMgr(); List readyTransactionStates = globalTransactionMgr.getReadyToPublishTransactions(); if (readyTransactionStates == null || readyTransactionStates.isEmpty()) { diff --git a/fe/src/main/java/org/apache/doris/transaction/TransactionState.java b/fe/src/main/java/org/apache/doris/transaction/TransactionState.java index d39c0e085c7677..eb22dfbb3c1358 100644 --- a/fe/src/main/java/org/apache/doris/transaction/TransactionState.java +++ b/fe/src/main/java/org/apache/doris/transaction/TransactionState.java @@ -17,14 +17,17 @@ package org.apache.doris.transaction; +import org.apache.doris.catalog.Catalog; import org.apache.doris.common.Config; +import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; -import org.apache.doris.load.TxnStateChangeListener; import org.apache.doris.metric.MetricRepo; import org.apache.doris.task.PublishVersionTask; import com.google.common.base.Joiner; +import com.google.common.base.Strings; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -68,27 +71,33 @@ public static LoadJobSourceType valueOf(int flag) { return null; } } - + } + + public enum TxnStatusChangeReason { + DB_DROPPED, + TIMEOUT, + OFFSET_OUT_OF_RANGE; + + public static TxnStatusChangeReason fromString(String reasonString) { + for (TxnStatusChangeReason txnStatusChangeReason : TxnStatusChangeReason.values()) { + if (reasonString.contains(txnStatusChangeReason.toString())) { + return txnStatusChangeReason; + } + } + return null; + } + @Override public String toString() { switch (this) { - case FRONTEND: - return "frontend"; - case BACKEND_STREAMING: - return "backend_streaming"; - case INSERT_STREAMING: - return "insert_streaming"; + case OFFSET_OUT_OF_RANGE: + return "Offset out of range"; default: - return null; + return this.name(); } } } - public enum TxnStatusChangeReason { - DB_DROPPED, - TIMEOUT - } - private long dbId; private long transactionId; private String label; @@ -96,13 +105,14 @@ public enum TxnStatusChangeReason { // no need to persist it private long timestamp; private Map idToTableCommitInfos; + // coordinator is show who begin this txn (FE, or one of BE, etc...) private String coordinator; private TransactionStatus transactionStatus; private LoadJobSourceType sourceType; private long prepareTime; private long commitTime; private long finishTime; - private String reason; + private String reason = ""; // error replica ids private Set errorReplicas; private CountDownLatch latch; @@ -113,10 +123,14 @@ public enum TxnStatusChangeReason { private long publishVersionTime; private TransactionStatus preStatus = null; + private long callbackId = -1; + private long timeoutMs = Config.stream_load_default_timeout_second; + // optional private TxnCommitAttachment txnCommitAttachment; - private TxnStateChangeListener txnStateChangeListener; + private String errorLogUrl = null; + public TransactionState() { this.dbId = -1; this.transactionId = -1; @@ -137,7 +151,7 @@ public TransactionState() { } public TransactionState(long dbId, long transactionId, String label, long timestamp, - LoadJobSourceType sourceType, String coordinator) { + LoadJobSourceType sourceType, String coordinator, long callbackId, long timeoutMs) { this.dbId = dbId; this.transactionId = transactionId; this.label = label; @@ -154,14 +168,8 @@ public TransactionState(long dbId, long transactionId, String label, long timest this.publishVersionTasks = Maps.newHashMap(); this.hasSendTask = false; this.latch = new CountDownLatch(1); - } - - public TransactionState(long dbId, long transactionId, String label, long timestamp, - LoadJobSourceType sourceType, String coordinator, TxnStateChangeListener txnStateChangeListener) { - this(dbId, transactionId, label, timestamp, sourceType, coordinator); - if (txnStateChangeListener != null) { - this.txnStateChangeListener = txnStateChangeListener; - } + this.callbackId = callbackId; + this.timeoutMs = timeoutMs; } public void setErrorReplicas(Set newErrorReplicas) { @@ -184,7 +192,7 @@ public void setHasSendTask(boolean hasSendTask) { this.hasSendTask = hasSendTask; this.publishVersionTime = System.currentTimeMillis(); } - + public void updateSendTaskTime() { this.publishVersionTime = System.currentTimeMillis(); } @@ -201,69 +209,6 @@ public long getTimestamp() { return timestamp; } - @Override - public void write(DataOutput out) throws IOException { - out.writeLong(transactionId); - Text.writeString(out, label); - out.writeLong(dbId); - out.writeInt(idToTableCommitInfos.size()); - for (TableCommitInfo info : idToTableCommitInfos.values()) { - info.write(out); - } - Text.writeString(out, coordinator); - out.writeInt(transactionStatus.value()); - out.writeInt(sourceType.value()); - out.writeLong(prepareTime); - out.writeLong(commitTime); - out.writeLong(finishTime); - Text.writeString(out, reason); - out.writeInt(errorReplicas.size()); - for (long errorReplciaId : errorReplicas) { - out.writeLong(errorReplciaId); - } - // TODO(ml): persistent will be enable after all of routine load work finished. -// if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_46) { -// if (txnCommitAttachment == null) { -// out.writeBoolean(false); -// } else { -// out.writeBoolean(true); -// txnCommitAttachment.write(out); -// } -// } - } - - @Override - public void readFields(DataInput in) throws IOException { - transactionId = in.readLong(); - label = Text.readString(in); - dbId = in.readLong(); - int size = in.readInt(); - for (int i = 0; i < size; i++) { - TableCommitInfo info = new TableCommitInfo(); - info.readFields(in); - idToTableCommitInfos.put(info.getTableId(), info); - } - coordinator = Text.readString(in); - transactionStatus = TransactionStatus.valueOf(in.readInt()); - sourceType = LoadJobSourceType.valueOf(in.readInt()); - prepareTime = in.readLong(); - commitTime = in.readLong(); - finishTime = in.readLong(); - reason = Text.readString(in); - int errorReplicaNum = in.readInt(); - for (int i = 0; i < errorReplicaNum; ++i) { - errorReplicas.add(in.readLong()); - } - // TODO(ml): persistent will be enable after all of routine load work finished. -// if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_46) { -// if (in.readBoolean()) { -// txnCommitAttachment = TransactionStateExtra.readTxnCommitAttachment(in, sourceType); -// } -// } - - // TODO(ml): reload txnStateChangeListener by txnCommitAttachment - } - public long getTransactionId() { return transactionId; } @@ -300,31 +245,32 @@ public TransactionStatus getPreStatus() { return this.preStatus; } - public TxnCommitAttachment getTxnCommitAttachment() { return txnCommitAttachment; } - - public void setTransactionStatus(TransactionStatus transactionStatus) throws TransactionException { - setTransactionStatus(transactionStatus, null); + + public long getCallbackId() { + return callbackId; } - - public void setTransactionStatus(TransactionStatus transactionStatus, TxnStatusChangeReason txnStatusChangeReason) - throws TransactionException { - // before state changed - if (txnStateChangeListener != null) { - switch (transactionStatus) { - case ABORTED: - txnStateChangeListener.beforeAborted(this, txnStatusChangeReason); - break; - } - } - - // state changed + + public long getTimeoutMs() { + return timeoutMs; + } + + public void setErrorLogUrl(String errorLogUrl) { + this.errorLogUrl = errorLogUrl; + } + + public String getErrorLogUrl() { + return errorLogUrl; + } + + public void setTransactionStatus(TransactionStatus transactionStatus) { + // status changed this.preStatus = this.transactionStatus; this.transactionStatus = transactionStatus; - // after state changed + // after status changed if (transactionStatus == TransactionStatus.VISIBLE) { this.latch.countDown(); if (MetricRepo.isInit.get()) { @@ -334,14 +280,70 @@ public void setTransactionStatus(TransactionStatus transactionStatus, TxnStatusC if (MetricRepo.isInit.get()) { MetricRepo.COUNTER_TXN_FAILED.increase(1L); } - if (txnStateChangeListener != null) { - txnStateChangeListener.onAborted(this, txnStatusChangeReason); + } + } + + public void beforeStateTransform(TransactionStatus transactionStatus) throws TransactionException { + // before status changed + TxnStateChangeCallback callback = Catalog.getCurrentGlobalTransactionMgr() + .getCallbackFactory().getCallback(callbackId); + if (callback != null) { + switch (transactionStatus) { + case ABORTED: + callback.beforeAborted(this); + break; + case COMMITTED: + callback.beforeCommitted(this); + break; + default: + break; + } + } else if (callback == null && callbackId > 0) { + switch (transactionStatus) { + case COMMITTED: + // Maybe listener has been deleted. The txn need to be aborted later. + throw new TransactionException("Failed to commit txn when callback could not be found"); + default: + break; + } + } + } + + public void afterStateTransform(TransactionStatus transactionStatus, boolean txnOperated) throws UserException { + afterStateTransform(transactionStatus, txnOperated, null); + } + + public void afterStateTransform(TransactionStatus transactionStatus, boolean txnOperated, String txnStatusChangeReason) + throws UserException { + // after status changed + TxnStateChangeCallback callback = Catalog.getCurrentGlobalTransactionMgr() + .getCallbackFactory().getCallback(callbackId); + if (callback != null) { + switch (transactionStatus) { + case ABORTED: + callback.afterAborted(this, txnOperated, txnStatusChangeReason); + break; + case COMMITTED: + callback.afterCommitted(this, txnOperated); + break; + default: + break; } - } else if (transactionStatus == TransactionStatus.COMMITTED && txnStateChangeListener != null) { - txnStateChangeListener.onCommitted(this); } } + public void replaySetTransactionStatus() { + TxnStateChangeCallback callback = Catalog.getCurrentGlobalTransactionMgr().getCallbackFactory().getCallback( + callbackId); + if (callback != null) { + if (transactionStatus == TransactionStatus.ABORTED) { + callback.replayOnAborted(this); + } else if (transactionStatus == TransactionStatus.COMMITTED) { + callback.replayOnCommitted(this); + } + } + } + public void waitTransactionVisible(long timeoutMillis) throws InterruptedException { this.latch.await(timeoutMillis, TimeUnit.MILLISECONDS); } @@ -359,7 +361,7 @@ public void setFinishTime(long finishTime) { } public void setReason(String reason) { - this.reason = reason; + this.reason = Strings.nullToEmpty(reason); } public Set getErrorReplicas() { @@ -426,12 +428,66 @@ public boolean isPublishTimeout() { return System.currentTimeMillis() - publishVersionTime > timeoutMillis; } - public void setTxnStateChangeListener(TxnStateChangeListener txnStateChangeListener) { - this.txnStateChangeListener = txnStateChangeListener; + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(transactionId); + Text.writeString(out, label); + out.writeLong(dbId); + out.writeInt(idToTableCommitInfos.size()); + for (TableCommitInfo info : idToTableCommitInfos.values()) { + info.write(out); + } + Text.writeString(out, coordinator); + out.writeInt(transactionStatus.value()); + out.writeInt(sourceType.value()); + out.writeLong(prepareTime); + out.writeLong(commitTime); + out.writeLong(finishTime); + Text.writeString(out, reason); + out.writeInt(errorReplicas.size()); + for (long errorReplciaId : errorReplicas) { + out.writeLong(errorReplciaId); + } + + if (txnCommitAttachment == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + txnCommitAttachment.write(out); + } + out.writeLong(callbackId); + out.writeLong(timeoutMs); } - public TxnStateChangeListener getTxnStateChangeListener() { - return txnStateChangeListener; + @Override + public void readFields(DataInput in) throws IOException { + transactionId = in.readLong(); + label = Text.readString(in); + dbId = in.readLong(); + int size = in.readInt(); + for (int i = 0; i < size; i++) { + TableCommitInfo info = new TableCommitInfo(); + info.readFields(in); + idToTableCommitInfos.put(info.getTableId(), info); + } + coordinator = Text.readString(in); + transactionStatus = TransactionStatus.valueOf(in.readInt()); + sourceType = LoadJobSourceType.valueOf(in.readInt()); + prepareTime = in.readLong(); + commitTime = in.readLong(); + finishTime = in.readLong(); + reason = Text.readString(in); + int errorReplicaNum = in.readInt(); + for (int i = 0; i < errorReplicaNum; ++i) { + errorReplicas.add(in.readLong()); + } + + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_49) { + if (in.readBoolean()) { + txnCommitAttachment = TxnCommitAttachment.read(in); + } + callbackId = in.readLong(); + timeoutMs = in.readLong(); + } } - } diff --git a/fe/src/main/java/org/apache/doris/transaction/TransactionStatus.java b/fe/src/main/java/org/apache/doris/transaction/TransactionStatus.java index 5f88c6fdf0ae55..efbe0b1561c04e 100644 --- a/fe/src/main/java/org/apache/doris/transaction/TransactionStatus.java +++ b/fe/src/main/java/org/apache/doris/transaction/TransactionStatus.java @@ -51,6 +51,10 @@ public static TransactionStatus valueOf(int flag) { } } + public boolean isFinalStatus() { + return this == TransactionStatus.VISIBLE || this == TransactionStatus.ABORTED; + } + @Override public String toString() { switch (this) { diff --git a/fe/src/main/java/org/apache/doris/transaction/TxnCommitAttachment.java b/fe/src/main/java/org/apache/doris/transaction/TxnCommitAttachment.java index ee9ff2c2957e6c..206ea0959b2a02 100644 --- a/fe/src/main/java/org/apache/doris/transaction/TxnCommitAttachment.java +++ b/fe/src/main/java/org/apache/doris/transaction/TxnCommitAttachment.java @@ -17,15 +17,29 @@ package org.apache.doris.transaction; +import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; import org.apache.doris.thrift.TTxnCommitAttachment; +import org.apache.doris.transaction.TransactionState.LoadJobSourceType; import java.io.DataInput; +import java.io.DataOutput; import java.io.IOException; public abstract class TxnCommitAttachment implements Writable { + protected TransactionState.LoadJobSourceType sourceType; + protected boolean isTypeRead = false; + + public TxnCommitAttachment(TransactionState.LoadJobSourceType sourceType) { + this.sourceType = sourceType; + } + + public void setTypeRead(boolean isTypeRead) { + this.isTypeRead = isTypeRead; + } + public static TxnCommitAttachment readTxnCommitAttachment(DataInput in, TransactionState.LoadJobSourceType sourceType) throws IOException { @@ -41,8 +55,8 @@ public static TxnCommitAttachment readTxnCommitAttachment(DataInput in, public static TxnCommitAttachment fromThrift(TTxnCommitAttachment txnCommitAttachment) { if (txnCommitAttachment != null) { - switch (txnCommitAttachment.txnSourceType) { - case ROUTINE_LOAD_TASK: + switch (txnCommitAttachment.getLoadType()) { + case ROUTINE_LOAD: return new RLTaskTxnCommitAttachment(txnCommitAttachment.getRlTaskTxnCommitAttachment()); default: return null; @@ -51,4 +65,32 @@ public static TxnCommitAttachment fromThrift(TTxnCommitAttachment txnCommitAttac return null; } } + + public static TxnCommitAttachment read(DataInput in) throws IOException { + TxnCommitAttachment attachment = null; + LoadJobSourceType type = LoadJobSourceType.valueOf(Text.readString(in)); + if (type == LoadJobSourceType.ROUTINE_LOAD_TASK) { + attachment = new RLTaskTxnCommitAttachment(); + } else { + throw new IOException("Unknown load job source type: " + type.name()); + } + + attachment.setTypeRead(true); + attachment.readFields(in); + return attachment; + } + + @Override + public void write(DataOutput out) throws IOException { + // ATTN: must write type first + Text.writeString(out, sourceType.name()); + } + + @Override + public void readFields(DataInput in) throws IOException { + if (!isTypeRead) { + sourceType = LoadJobSourceType.valueOf(Text.readString(in)); + isTypeRead = true; + } + } } diff --git a/fe/src/main/java/org/apache/doris/transaction/TxnStateCallbackFactory.java b/fe/src/main/java/org/apache/doris/transaction/TxnStateCallbackFactory.java new file mode 100644 index 00000000000000..94a06f856fd45d --- /dev/null +++ b/fe/src/main/java/org/apache/doris/transaction/TxnStateCallbackFactory.java @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.transaction; + +import com.google.common.collect.Maps; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Map; + +/* + * Author: Chenmingyu + * Date: Mar 14, 2019 + */ + +// saves all TxnStateChangeListeners +public class TxnStateCallbackFactory { + private static final Logger LOG = LogManager.getLogger(TxnStateCallbackFactory.class); + + private Map callbacks = Maps.newHashMap(); + + public synchronized boolean addCallback(TxnStateChangeCallback callback) { + if (callbacks.containsKey(callback.getId())) { + return false; + } + callbacks.put(callback.getId(), callback); + LOG.info("add callback of txn state : {}", callback.getId()); + return true; + } + + public synchronized void removeCallback(long id) { + callbacks.remove(id); + LOG.info("remove callback of txn state : {}", id); + } + + public synchronized TxnStateChangeCallback getCallback(long id) { + return callbacks.get(id); + } +} diff --git a/fe/src/main/java/org/apache/doris/transaction/TxnStateChangeCallback.java b/fe/src/main/java/org/apache/doris/transaction/TxnStateChangeCallback.java new file mode 100644 index 00000000000000..d0a4ebb8cd60de --- /dev/null +++ b/fe/src/main/java/org/apache/doris/transaction/TxnStateChangeCallback.java @@ -0,0 +1,63 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.transaction; + +import org.apache.doris.common.UserException; + +public interface TxnStateChangeCallback { + + public long getId(); + + /** + * this interface is executed before txn committed, it will check if txn could be commit + * @param txnState + * @throws TransactionException if transaction could not be commit or there are some exception before committed, + * it will throw this exception. The txn will be committed failed. + */ + public void beforeCommitted(TransactionState txnState) throws TransactionException; + + /** + * this interface is executed before txn aborted, it will check if txn could be abort + * + * @param txnState + * @throws TransactionException if transaction could not be abort or there are some exception before aborted, + * it will throw this exception. The txn will be aborted failed. + */ + public void beforeAborted(TransactionState txnState) throws TransactionException; + + /** + * update catalog of job which has related txn after transaction has been committed + * + * @param txnState + */ + public void afterCommitted(TransactionState txnState, boolean txnOperated) throws UserException; + + public void replayOnCommitted(TransactionState txnState); + + /** + * this interface is executed when transaction has been aborted + * + * @param txnState + * @param txnStatusChangeReason + * maybe null + * @return + */ + public void afterAborted(TransactionState txnState, boolean txnOperated, String txnStatusChangeReason) throws UserException; + + public void replayOnAborted(TransactionState txnState); +} diff --git a/fe/src/main/jflex/sql_scanner.flex b/fe/src/main/jflex/sql_scanner.flex index 84c2197da3ca9d..1ad0c5326139ef 100644 --- a/fe/src/main/jflex/sql_scanner.flex +++ b/fe/src/main/jflex/sql_scanner.flex @@ -199,6 +199,7 @@ import org.apache.doris.common.util.SqlUtils; keywordMap.put("pause", new Integer(SqlParserSymbols.KW_PAUSE)); keywordMap.put("resume", new Integer(SqlParserSymbols.KW_RESUME)); keywordMap.put("stop", new Integer(SqlParserSymbols.KW_STOP)); + keywordMap.put("task", new Integer(SqlParserSymbols.KW_TASK)); keywordMap.put("local", new Integer(SqlParserSymbols.KW_LOCAL)); keywordMap.put("location", new Integer(SqlParserSymbols.KW_LOCATION)); keywordMap.put("max", new Integer(SqlParserSymbols.KW_MAX)); diff --git a/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java b/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java index d03a63d1aad00f..87b49af93f8ac2 100644 --- a/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java +++ b/fe/src/test/java/org/apache/doris/alter/RollupJobTest.java @@ -38,6 +38,7 @@ import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.meta.MetaContext; import org.apache.doris.task.AgentTask; @@ -195,7 +196,7 @@ public void testRollup2() throws Exception { // load one transaction with backend 2 has errors long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); // commit a transaction, backend 2 has errors TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); @@ -310,7 +311,7 @@ public void testRollup3() throws Exception { long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); // commit a transaction, backend 2 has errors TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); diff --git a/fe/src/test/java/org/apache/doris/alter/SchemaChangeJobTest.java b/fe/src/test/java/org/apache/doris/alter/SchemaChangeJobTest.java index b471e55219b35e..2a25424a8d4b86 100644 --- a/fe/src/test/java/org/apache/doris/alter/SchemaChangeJobTest.java +++ b/fe/src/test/java/org/apache/doris/alter/SchemaChangeJobTest.java @@ -45,6 +45,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.meta.MetaContext; import org.apache.doris.task.AgentTask; @@ -196,7 +197,7 @@ public void testSchemaChange2() throws Exception { long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); // commit a transaction, backend 2 has errors TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); diff --git a/fe/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java b/fe/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java index 6b210b1d9c4b15..574f51512279fd 100644 --- a/fe/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java +++ b/fe/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java @@ -17,15 +17,13 @@ package org.apache.doris.analysis; -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import mockit.Injectable; -import mockit.Mock; -import mockit.MockUp; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.load.routineload.LoadDataSourceType; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.Assert; @@ -35,6 +33,10 @@ import java.util.List; import java.util.Map; +import mockit.Injectable; +import mockit.Mock; +import mockit.MockUp; + public class CreateRoutineLoadStmtTest { private static final Logger LOG = LogManager.getLogger(CreateRoutineLoadStmtTest.class); @@ -43,6 +45,7 @@ public class CreateRoutineLoadStmtTest { public void testAnalyzeWithDuplicateProperty(@Injectable Analyzer analyzer) throws UserException { String jobName = "job1"; String dbName = "db1"; + LabelName labelName = new LabelName(dbName, jobName); String tableNameString = "table1"; String topicName = "topic1"; String serverAddress = "http://127.0.0.1:8080"; @@ -53,7 +56,6 @@ public void testAnalyzeWithDuplicateProperty(@Injectable Analyzer analyzer) thro ColumnSeparator columnSeparator = new ColumnSeparator(","); // duplicate load property - TableName tableName = new TableName(dbName, tableNameString); List loadPropertyList = new ArrayList<>(); loadPropertyList.add(columnSeparator); loadPropertyList.add(columnSeparator); @@ -63,10 +65,10 @@ public void testAnalyzeWithDuplicateProperty(@Injectable Analyzer analyzer) thro Map customProperties = Maps.newHashMap(); customProperties.put(CreateRoutineLoadStmt.KAFKA_TOPIC_PROPERTY, topicName); - customProperties.put(CreateRoutineLoadStmt.KAFKA_ENDPOINT_PROPERTY, serverAddress); + customProperties.put(CreateRoutineLoadStmt.KAFKA_BROKER_LIST_PROPERTY, serverAddress); customProperties.put(CreateRoutineLoadStmt.KAFKA_PARTITIONS_PROPERTY, kafkaPartitionString); - CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(jobName, tableName, + CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(labelName, tableNameString, loadPropertyList, properties, typeName, customProperties); @@ -89,6 +91,7 @@ public void analyze(Analyzer analyzer1) { public void testAnalyze(@Injectable Analyzer analyzer) throws UserException { String jobName = "job1"; String dbName = "db1"; + LabelName labelName = new LabelName(dbName, jobName); String tableNameString = "table1"; String topicName = "topic1"; String serverAddress = "127.0.0.1:8080"; @@ -109,10 +112,10 @@ public void testAnalyze(@Injectable Analyzer analyzer) throws UserException { Map customProperties = Maps.newHashMap(); customProperties.put(CreateRoutineLoadStmt.KAFKA_TOPIC_PROPERTY, topicName); - customProperties.put(CreateRoutineLoadStmt.KAFKA_ENDPOINT_PROPERTY, serverAddress); + customProperties.put(CreateRoutineLoadStmt.KAFKA_BROKER_LIST_PROPERTY, serverAddress); customProperties.put(CreateRoutineLoadStmt.KAFKA_PARTITIONS_PROPERTY, kafkaPartitionString); - CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(jobName, tableName, + CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(labelName, tableNameString, loadPropertyList, properties, typeName, customProperties); new MockUp() { @@ -129,9 +132,8 @@ public void analyze(Analyzer analyzer1) { Assert.assertEquals(partitionNames.getPartitionNames(), createRoutineLoadStmt.getRoutineLoadDesc().getPartitionNames()); Assert.assertEquals(2, createRoutineLoadStmt.getDesiredConcurrentNum()); Assert.assertEquals(0, createRoutineLoadStmt.getMaxErrorNum()); - Assert.assertEquals(serverAddress, createRoutineLoadStmt.getKafkaEndpoint()); + Assert.assertEquals(serverAddress, createRoutineLoadStmt.getKafkaBrokerList()); Assert.assertEquals(topicName, createRoutineLoadStmt.getKafkaTopic()); - Assert.assertEquals(kafkaPartitionString, Joiner.on(",").join(createRoutineLoadStmt.getKafkaPartitions())); } } diff --git a/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java b/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java index e078a6ace1f8f9..7a380c9ce245a8 100644 --- a/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java +++ b/fe/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java @@ -89,9 +89,9 @@ public static Catalog createTestCatalog() throws InstantiationException, Illegal Backend backend1 = createBackend(testBackendId1, "host1", 123, 124, 125); Backend backend2 = createBackend(testBackendId2, "host1", 123, 124, 125); Backend backend3 = createBackend(testBackendId3, "host1", 123, 124, 125); - catalog.getCurrentSystemInfo().addBackend(backend1); - catalog.getCurrentSystemInfo().addBackend(backend2); - catalog.getCurrentSystemInfo().addBackend(backend3); + Catalog.getCurrentSystemInfo().addBackend(backend1); + Catalog.getCurrentSystemInfo().addBackend(backend2); + Catalog.getCurrentSystemInfo().addBackend(backend3); catalog.initDefaultCluster(); Database db = createSimpleDb(testDbId1, testTableId1, testPartitionId1, testIndexId1, testTabletId1, testStartVersion, testStartVersionHash); diff --git a/fe/src/test/java/org/apache/doris/catalog/FakeEditLog.java b/fe/src/test/java/org/apache/doris/catalog/FakeEditLog.java index dcb74d4ac25fb7..62d5e3937b07e0 100644 --- a/fe/src/test/java/org/apache/doris/catalog/FakeEditLog.java +++ b/fe/src/test/java/org/apache/doris/catalog/FakeEditLog.java @@ -21,6 +21,7 @@ import org.apache.doris.alter.SchemaChangeJob; import org.apache.doris.cluster.Cluster; import org.apache.doris.persist.EditLog; +import org.apache.doris.persist.RoutineLoadOperation; import org.apache.doris.transaction.TransactionState; import java.util.HashMap; @@ -85,6 +86,11 @@ public void logStartSchemaChange(SchemaChangeJob schemaChangeJob) { public void logFinishingSchemaChange(SchemaChangeJob schemaChangeJob) { } + @Mock + public void logOpRoutineLoadJob(RoutineLoadOperation operation) { + + } + public TransactionState getTransaction(long transactionId) { return allTransactionState.get(transactionId); } diff --git a/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java b/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java index 64432e386477fc..a7d04e3c0be6e6 100644 --- a/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java +++ b/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java @@ -39,6 +39,7 @@ import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TPullLoadSubTaskInfo; import org.apache.doris.thrift.TResultBatch; +import org.apache.doris.thrift.TRoutineLoadTask; import org.apache.doris.thrift.TSnapshotRequest; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TTabletStatResult; @@ -217,6 +218,12 @@ public TTabletStatResult get_tablet_stat() throws TException { // TODO Auto-generated method stub return null; } + + @Override + public TStatus submit_routine_load_task(List tasks) throws TException { + // TODO Auto-generated method stub + return null; + } } @Test diff --git a/fe/src/test/java/org/apache/doris/load/routineload/KafkaProducerTest.java b/fe/src/test/java/org/apache/doris/load/routineload/KafkaProducerTest.java new file mode 100644 index 00000000000000..cc7318c6185691 --- /dev/null +++ b/fe/src/test/java/org/apache/doris/load/routineload/KafkaProducerTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.load.routineload; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.StringSerializer; + +import java.util.Properties; +import java.util.concurrent.ExecutionException; + +public class KafkaProducerTest { + + public Producer createProducer() { + Properties props = new Properties(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "xxx"); + props.put(ProducerConfig.CLIENT_ID_CONFIG, "client1"); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class.getName()); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + return new KafkaProducer<>(props); + } + + public static void main(String[] args) throws InterruptedException { + KafkaProducerTest kafkaProducerTest = new KafkaProducerTest(); + Producer kafkaProducer = kafkaProducerTest.createProducer(); + int i = 1; + while (true) { + String value = String.valueOf(i); + if (i % 10000 == 0) { + value = value + "\t" + value; + } + ProducerRecord record = new ProducerRecord<>("miaoling", value); + try { + RecordMetadata metadata = kafkaProducer.send(record).get(); + System.out.println("Record send with value " + value + " to partition " + + metadata.partition() + " with offset " + metadata.offset()); + } catch (ExecutionException e) { + System.out.println("Error in sending record " + value); + System.out.println(e); + } catch (InterruptedException e) { + System.out.println("Error in sending record " + value); + System.out.println(e); + } + i++; + } + } + +} diff --git a/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java b/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java index be923dd457f757..7735151cfa3b5b 100644 --- a/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java +++ b/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java @@ -17,37 +17,32 @@ package org.apache.doris.load.routineload; -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import mockit.Deencapsulation; -import mockit.Expectations; -import mockit.Injectable; -import mockit.Mock; -import mockit.MockUp; -import mockit.Mocked; -import mockit.Verifications; import org.apache.doris.analysis.ColumnSeparator; import org.apache.doris.analysis.CreateRoutineLoadStmt; +import org.apache.doris.analysis.LabelName; import org.apache.doris.analysis.ParseNode; import org.apache.doris.analysis.PartitionNames; import org.apache.doris.analysis.TableName; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.Table; -import org.apache.doris.common.LoadException; -import org.apache.doris.load.RoutineLoadDesc; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; -import org.apache.doris.common.SystemIdGenerator; +import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; +import org.apache.doris.load.RoutineLoadDesc; import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TResourceInfo; import org.apache.doris.transaction.BeginTransactionException; import org.apache.doris.transaction.GlobalTransactionMgr; -import org.apache.doris.transaction.TransactionState; + +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; @@ -62,6 +57,13 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.UUID; + +import mockit.Deencapsulation; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; +import mockit.Verifications; public class KafkaRoutineLoadJobTest { @@ -71,6 +73,7 @@ public class KafkaRoutineLoadJobTest { private String jobName = "job1"; private String dbName = "db1"; + private LabelName labelName = new LabelName(dbName, jobName); private String tableNameString = "table1"; private String topicName = "topic1"; private String serverAddress = "http://127.0.0.1:8080"; @@ -84,6 +87,8 @@ public class KafkaRoutineLoadJobTest { ConnectContext connectContext; @Mocked TResourceInfo tResourceInfo; + @Mocked + KafkaConsumer kafkaConsumer; @Before public void init() { @@ -100,36 +105,56 @@ public void testBeNumMin(@Mocked KafkaConsumer kafkaConsumer, @Mocked SystemInfoService systemInfoService, @Mocked Database database, @Mocked RoutineLoadDesc routineLoadDesc) throws MetaNotFoundException { - List partitionInfoList = new ArrayList<>(); - partitionInfoList.add(partitionInfo1); - partitionInfoList.add(partitionInfo2); - List beIds = Lists.newArrayList(1L); + List partitionList1 = Lists.newArrayList(1, 2); + List partitionList2 = Lists.newArrayList(1, 2, 3); + List partitionList3 = Lists.newArrayList(1, 2, 3, 4); + List partitionList4 = Lists.newArrayList(1, 2, 3, 4, 5, 6, 7); + List beIds1 = Lists.newArrayList(1L); + List beIds2 = Lists.newArrayList(1L, 2L, 3L, 4L); - String clusterName = "clusterA"; + String clusterName1 = "default1"; + String clusterName2 = "default2"; new Expectations() { { - kafkaConsumer.partitionsFor(anyString, (Duration) any); - result = partitionInfoList; Catalog.getCurrentSystemInfo(); result = systemInfoService; - Catalog.getCurrentCatalog(); - result = catalog; - catalog.getDb(anyLong); - result = database; - systemInfoService.getBackendIds(true); - result = beIds; - connectContext.toResourceCtx(); - result = tResourceInfo; + systemInfoService.getClusterBackendIds(clusterName1, true); + result = beIds1; + systemInfoService.getClusterBackendIds(clusterName2, true); + result = beIds2; + minTimes = 0; } }; - KafkaRoutineLoadJob kafkaRoutineLoadJob = - new KafkaRoutineLoadJob("1", "kafka_routine_load_job", 1L, - 1L, routineLoadDesc, 3, 0, - "", "", new KafkaProgress()); - Deencapsulation.setField(kafkaRoutineLoadJob, "consumer", kafkaConsumer); - Assert.assertEquals(1, kafkaRoutineLoadJob.calculateCurrentConcurrentTaskNum()); + // 2 partitions, 1 be + RoutineLoadJob routineLoadJob = + new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName1, 1L, + 1L, "127.0.0.1:9020", "topic1"); + Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); + Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList1); + Assert.assertEquals(1, routineLoadJob.calculateCurrentConcurrentTaskNum()); + + // 3 partitions, 4 be + routineLoadJob = new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName2, 1L, + 1L, "127.0.0.1:9020", "topic1"); + Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); + Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList2); + Assert.assertEquals(3, routineLoadJob.calculateCurrentConcurrentTaskNum()); + + // 4 partitions, 4 be + routineLoadJob = new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName2, 1L, + 1L, "127.0.0.1:9020", "topic1"); + Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); + Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList3); + Assert.assertEquals(4, routineLoadJob.calculateCurrentConcurrentTaskNum()); + + // 7 partitions, 4 be + routineLoadJob = new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName2, 1L, + 1L, "127.0.0.1:9020", "topic1"); + Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); + Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList4); + Assert.assertEquals(4, routineLoadJob.calculateCurrentConcurrentTaskNum()); } @@ -138,40 +163,34 @@ public void testDivideRoutineLoadJob(@Mocked KafkaConsumer kafkaConsumer, @Injectable GlobalTransactionMgr globalTransactionMgr, @Mocked Catalog catalog, @Injectable RoutineLoadManager routineLoadManager, + @Injectable RoutineLoadTaskScheduler routineLoadTaskScheduler, @Mocked RoutineLoadDesc routineLoadDesc) - throws BeginTransactionException, LabelAlreadyUsedException, AnalysisException { - - new Expectations() { - { - connectContext.toResourceCtx(); - result = tResourceInfo; - } - }; + throws UserException { - KafkaRoutineLoadJob kafkaRoutineLoadJob = - new KafkaRoutineLoadJob("1", "kafka_routine_load_job", 1L, - 1L, routineLoadDesc, 3, 0, - "", "", null); + RoutineLoadJob routineLoadJob = + new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", "default", 1L, + 1L, "127.0.0.1:9020", "topic1"); new Expectations() { { - globalTransactionMgr.beginTransaction(anyLong, anyString, anyLong, anyString, - TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, (KafkaRoutineLoadJob) any); - result = 0L; catalog.getRoutineLoadManager(); result = routineLoadManager; + catalog.getRoutineLoadTaskScheduler(); + result = routineLoadTaskScheduler; } }; - Deencapsulation.setField(kafkaRoutineLoadJob, "currentKafkaPartitions", Arrays.asList(1, 4, 6)); - Deencapsulation.setField(kafkaRoutineLoadJob, "consumer", kafkaConsumer); + Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", Arrays.asList(1, 4, 6)); + Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); - kafkaRoutineLoadJob.divideRoutineLoadJob(2); + routineLoadJob.divideRoutineLoadJob(2); - List result = kafkaRoutineLoadJob.getNeedScheduleTaskInfoList(); - Assert.assertEquals(2, result.size()); - for (RoutineLoadTaskInfo routineLoadTaskInfo : result) { + // todo(ml): assert + List routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); + Assert.assertEquals(2, routineLoadTaskInfoList.size()); + for (RoutineLoadTaskInfo routineLoadTaskInfo : routineLoadTaskInfoList) { KafkaTaskInfo kafkaTaskInfo = (KafkaTaskInfo) routineLoadTaskInfo; + Assert.assertEquals(false, kafkaTaskInfo.isRunning()); if (kafkaTaskInfo.getPartitions().size() == 2) { Assert.assertTrue(kafkaTaskInfo.getPartitions().contains(1)); Assert.assertTrue(kafkaTaskInfo.getPartitions().contains(6)); @@ -184,58 +203,33 @@ public void testDivideRoutineLoadJob(@Mocked KafkaConsumer kafkaConsumer, } @Test - public void testProcessTimeOutTasks(@Mocked KafkaConsumer kafkaConsumer, - @Injectable GlobalTransactionMgr globalTransactionMgr, + public void testProcessTimeOutTasks(@Injectable GlobalTransactionMgr globalTransactionMgr, @Mocked Catalog catalog, @Injectable RoutineLoadManager routineLoadManager, @Mocked RoutineLoadDesc routineLoadDesc) throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException { - new Expectations() { - { - connectContext.toResourceCtx(); - result = tResourceInfo; - } - }; - RoutineLoadJob routineLoadJob = - new KafkaRoutineLoadJob("1", "kafka_routine_load_job", 1L, - 1L, routineLoadDesc, 3, 0, - "", "", null); + new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", "default", 1L, + 1L, "127.0.0.1:9020", "topic1"); + long maxBatchIntervalS = 10; + Deencapsulation.setField(routineLoadJob, "maxBatchIntervalS", maxBatchIntervalS); new Expectations() { { - globalTransactionMgr.beginTransaction(anyLong, anyString, anyLong, anyString, - TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, routineLoadJob); - result = 0L; catalog.getRoutineLoadManager(); result = routineLoadManager; } }; List routineLoadTaskInfoList = new ArrayList<>(); - KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo("1", "1"); - kafkaTaskInfo.addKafkaPartition(100); - kafkaTaskInfo.setLoadStartTimeMs(System.currentTimeMillis() - DEFAULT_TASK_TIMEOUT_SECONDS * 60 * 1000); + Map partitionIdsToOffset = Maps.newHashMap(); + partitionIdsToOffset.put(100, 0L); + KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(new UUID(1, 1), 1L, "default_cluster", partitionIdsToOffset); + kafkaTaskInfo.setExecuteStartTimeMs(System.currentTimeMillis() - maxBatchIntervalS * 2 * 1000 - 1); routineLoadTaskInfoList.add(kafkaTaskInfo); Deencapsulation.setField(routineLoadJob, "routineLoadTaskInfoList", routineLoadTaskInfoList); - Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); - - new MockUp() { - @Mock - public long getNextId() { - return 2L; - } - }; - - new Expectations() { - { - routineLoadManager.getJob("1"); - result = routineLoadJob; - } - }; - routineLoadJob.processTimeoutTasks(); new Verifications() { @@ -244,42 +238,8 @@ public long getNextId() { Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); Assert.assertNotEquals("1", idToRoutineLoadTask.get(0).getId()); Assert.assertEquals(1, idToRoutineLoadTask.size()); - List needScheduleTask = - Deencapsulation.getField(routineLoadJob, "needScheduleTaskInfoList"); - Assert.assertEquals(1, needScheduleTask.size()); - Assert.assertEquals(100, (int) ((KafkaTaskInfo) (needScheduleTask.get(0))) - .getPartitions().get(0)); - } - }; - } - - @Test - public void testFromCreateStmtWithErrorPartition(@Mocked Catalog catalog, - @Injectable Database database, - @Injectable OlapTable table) throws LoadException { - CreateRoutineLoadStmt createRoutineLoadStmt = initCreateRoutineLoadStmt(); - RoutineLoadDesc routineLoadDesc = new RoutineLoadDesc(columnSeparator, null, null, partitionNames.getPartitionNames()); - Deencapsulation.setField(createRoutineLoadStmt, "routineLoadDesc", routineLoadDesc); - - new Expectations() { - { - catalog.getDb(dbName); - result = database; - database.getTable(tableNameString); - result = table; - table.getPartition("p1"); - result = null; - table.getType(); - result = Table.TableType.OLAP; } }; - - try { - KafkaRoutineLoadJob kafkaRoutineLoadJob = KafkaRoutineLoadJob.fromCreateStmt(createRoutineLoadStmt); - Assert.fail(); - } catch (AnalysisException e) { - LOG.info(e.getMessage()); - } } @Test @@ -291,8 +251,6 @@ public void testFromCreateStmtWithErrorTable(@Mocked Catalog catalog, new Expectations() { { - catalog.getDb(dbName); - result = database; database.getTable(tableNameString); result = null; } @@ -301,36 +259,33 @@ public void testFromCreateStmtWithErrorTable(@Mocked Catalog catalog, try { KafkaRoutineLoadJob kafkaRoutineLoadJob = KafkaRoutineLoadJob.fromCreateStmt(createRoutineLoadStmt); Assert.fail(); - } catch (AnalysisException e) { + } catch (UserException e) { LOG.info(e.getMessage()); } } @Test public void testFromCreateStmt(@Mocked Catalog catalog, - @Mocked KafkaConsumer kafkaConsumer, @Injectable Database database, - @Injectable OlapTable table) throws LoadException, AnalysisException { + @Injectable OlapTable table) throws UserException { CreateRoutineLoadStmt createRoutineLoadStmt = initCreateRoutineLoadStmt(); RoutineLoadDesc routineLoadDesc = new RoutineLoadDesc(columnSeparator, null, null, partitionNames.getPartitionNames()); Deencapsulation.setField(createRoutineLoadStmt, "routineLoadDesc", routineLoadDesc); - List kafkaIntegerList = Lists.newArrayList(); + List> partitionIdToOffset = Lists.newArrayList(); List kafkaPartitionInfoList = Lists.newArrayList(); for (String s : kafkaPartitionString.split(",")) { - kafkaIntegerList.add(Integer.valueOf(s)); + partitionIdToOffset.add(new Pair<>(Integer.valueOf(s), 0l)); PartitionInfo partitionInfo = new PartitionInfo(topicName, Integer.valueOf(s), null, null, null); kafkaPartitionInfoList.add(partitionInfo); } - Deencapsulation.setField(createRoutineLoadStmt, "kafkaPartitions", kafkaIntegerList); - Deencapsulation.setField(createRoutineLoadStmt, "kafkaEndpoint", serverAddress); + Deencapsulation.setField(createRoutineLoadStmt, "kafkaPartitionOffsets", partitionIdToOffset); + Deencapsulation.setField(createRoutineLoadStmt, "kafkaBrokerList", serverAddress); Deencapsulation.setField(createRoutineLoadStmt, "kafkaTopic", topicName); long dbId = 1l; long tableId = 2L; new Expectations() { { - catalog.getDb(dbName); - result = database; database.getTable(tableNameString); result = table; database.getId(); @@ -348,11 +303,10 @@ public void testFromCreateStmt(@Mocked Catalog catalog, Assert.assertEquals(jobName, kafkaRoutineLoadJob.getName()); Assert.assertEquals(dbId, kafkaRoutineLoadJob.getDbId()); Assert.assertEquals(tableId, kafkaRoutineLoadJob.getTableId()); - Assert.assertEquals(serverAddress, Deencapsulation.getField(kafkaRoutineLoadJob, "serverAddress")); + Assert.assertEquals(serverAddress, Deencapsulation.getField(kafkaRoutineLoadJob, "brokerList")); Assert.assertEquals(topicName, Deencapsulation.getField(kafkaRoutineLoadJob, "topic")); List kafkaPartitionResult = Deencapsulation.getField(kafkaRoutineLoadJob, "customKafkaPartitions"); Assert.assertEquals(kafkaPartitionString, Joiner.on(",").join(kafkaPartitionResult)); - Assert.assertEquals(routineLoadDesc, kafkaRoutineLoadJob.getRoutineLoadDesc()); } private CreateRoutineLoadStmt initCreateRoutineLoadStmt() { @@ -366,12 +320,13 @@ private CreateRoutineLoadStmt initCreateRoutineLoadStmt() { Map customProperties = Maps.newHashMap(); customProperties.put(CreateRoutineLoadStmt.KAFKA_TOPIC_PROPERTY, topicName); - customProperties.put(CreateRoutineLoadStmt.KAFKA_ENDPOINT_PROPERTY, serverAddress); + customProperties.put(CreateRoutineLoadStmt.KAFKA_BROKER_LIST_PROPERTY, serverAddress); customProperties.put(CreateRoutineLoadStmt.KAFKA_PARTITIONS_PROPERTY, kafkaPartitionString); - CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(jobName, tableName, + CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(labelName, tableNameString, loadPropertyList, properties, typeName, customProperties); + Deencapsulation.setField(createRoutineLoadStmt, "name", jobName); return createRoutineLoadStmt; } } diff --git a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java new file mode 100644 index 00000000000000..c86ca50689d6c7 --- /dev/null +++ b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +package org.apache.doris.load.routineload; + + +import org.apache.doris.analysis.CreateRoutineLoadStmt; +import org.apache.doris.analysis.SqlParser; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.UserException; +import org.apache.doris.persist.EditLog; +import org.apache.doris.transaction.TransactionException; +import org.apache.doris.transaction.TransactionState; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; + +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.junit.Assert; +import org.junit.Test; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import java_cup.runtime.Symbol; +import mockit.Deencapsulation; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; + +public class RoutineLoadJobTest { + + @Mocked + Catalog catalog; + @Mocked + EditLog editLog; + @Mocked + SqlParser sqlParser; + @Mocked + CreateRoutineLoadStmt createRoutineLoadStmt; + @Mocked + Symbol symbol; + @Mocked + KafkaConsumer kafkaConsumer; + + @Test + public void testAfterAbortedReasonOffsetOutOfRange(@Injectable TransactionState transactionState, + @Injectable RoutineLoadTaskInfo routineLoadTaskInfo, + @Injectable ReentrantReadWriteLock lock) + throws UserException { + + List routineLoadTaskInfoList = Lists.newArrayList(); + routineLoadTaskInfoList.add(routineLoadTaskInfo); + long txnId = 1L; + + new Expectations() { + { + transactionState.getTransactionId(); + result = txnId; + routineLoadTaskInfo.getTxnId(); + result = txnId; + } + }; + + String txnStatusChangeReasonString = TransactionState.TxnStatusChangeReason.OFFSET_OUT_OF_RANGE.toString(); + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "routineLoadTaskInfoList", routineLoadTaskInfoList); + Deencapsulation.setField(routineLoadJob, "lock", lock); + routineLoadJob.afterAborted(transactionState, true, txnStatusChangeReasonString); + + Assert.assertEquals(RoutineLoadJob.JobState.PAUSED, routineLoadJob.getState()); + } + + @Test + public void testAfterAborted(@Injectable TransactionState transactionState, + @Injectable KafkaTaskInfo routineLoadTaskInfo, + @Injectable KafkaProgress progress, + @Injectable ReentrantReadWriteLock lock) throws UserException { + List routineLoadTaskInfoList = Lists.newArrayList(); + routineLoadTaskInfoList.add(routineLoadTaskInfo); + long txnId = 1L; + + new Expectations() { + { + transactionState.getTransactionId(); + result = txnId; + routineLoadTaskInfo.getTxnId(); + result = txnId; + transactionState.getTxnCommitAttachment(); + result = new RLTaskTxnCommitAttachment(); + routineLoadTaskInfo.getPartitions(); + result = Lists.newArrayList(); + } + }; + + String txnStatusChangeReasonString = "no data"; + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.RUNNING); + Deencapsulation.setField(routineLoadJob, "routineLoadTaskInfoList", routineLoadTaskInfoList); + Deencapsulation.setField(routineLoadJob, "progress", progress); + Deencapsulation.setField(routineLoadJob, "lock", lock); + routineLoadJob.afterAborted(transactionState, true, txnStatusChangeReasonString); + + Assert.assertEquals(RoutineLoadJob.JobState.RUNNING, routineLoadJob.getState()); + Assert.assertEquals(new Long(1), Deencapsulation.getField(routineLoadJob, "abortedTaskNum")); + } + + @Test + public void testAfterCommittedWhileTaskAborted(@Injectable TransactionState transactionState, + @Injectable KafkaProgress progress, + @Injectable ReentrantReadWriteLock lock) throws UserException { + List routineLoadTaskInfoList = Lists.newArrayList(); + long txnId = 1L; + + new Expectations() { + { + transactionState.getTransactionId(); + result = txnId; + } + }; + + String txnStatusChangeReasonString = "no data"; + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.RUNNING); + Deencapsulation.setField(routineLoadJob, "routineLoadTaskInfoList", routineLoadTaskInfoList); + Deencapsulation.setField(routineLoadJob, "progress", progress); + Deencapsulation.setField(routineLoadJob, "lock", lock); + try { + routineLoadJob.afterCommitted(transactionState, true); + Assert.assertEquals(RoutineLoadJob.JobState.PAUSED, routineLoadJob.getState()); + } catch (TransactionException e) { + Assert.fail(); + } + } + + @Test + public void testGetShowInfo(@Mocked KafkaProgress kafkaProgress) { + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.PAUSED); + Deencapsulation.setField(routineLoadJob, "pauseReason", + TransactionState.TxnStatusChangeReason.OFFSET_OUT_OF_RANGE.toString()); + Deencapsulation.setField(routineLoadJob, "progress", kafkaProgress); + + List showInfo = routineLoadJob.getShowInfo(); + Assert.assertEquals(true, showInfo.stream().filter(entity -> !Strings.isNullOrEmpty(entity)) + .anyMatch(entity -> entity.equals(TransactionState.TxnStatusChangeReason.OFFSET_OUT_OF_RANGE.toString()))); + } + + @Test + public void testUpdateWhileDbDeleted() throws UserException { + new Expectations() { + { + catalog.getDb(anyLong); + result = null; + } + }; + + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + routineLoadJob.update(); + + Assert.assertEquals(RoutineLoadJob.JobState.CANCELLED, routineLoadJob.getState()); + } + + @Test + public void testUpdateWhileTableDeleted(@Injectable Database database) throws UserException { + new Expectations() { + { + catalog.getDb(anyLong); + result = database; + database.getTable(anyLong); + result = null; + } + }; + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + routineLoadJob.update(); + + Assert.assertEquals(RoutineLoadJob.JobState.CANCELLED, routineLoadJob.getState()); + } + + @Test + public void testUpdateWhilePartitionChanged(@Injectable Database database, + @Injectable Table table, + @Injectable PartitionInfo partitionInfo, + @Injectable KafkaProgress kafkaProgress) throws UserException { + List partitionInfoList = Lists.newArrayList(); + partitionInfoList.add(partitionInfo); + + new Expectations() { + { + catalog.getDb(anyLong); + result = database; + database.getTable(anyLong); + result = table; + kafkaConsumer.partitionsFor(anyString, (Duration) any); + result = partitionInfoList; + partitionInfo.partition(); + result = 1; + } + }; + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.RUNNING); + Deencapsulation.setField(routineLoadJob, "progress", kafkaProgress); + routineLoadJob.update(); + + Assert.assertEquals(RoutineLoadJob.JobState.NEED_SCHEDULE, routineLoadJob.getState()); + } + + @Test + public void testUpdateNumOfDataErrorRowMoreThanMax() { + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "maxErrorNum", 0); + Deencapsulation.setField(routineLoadJob, "maxBatchRows", 0); + Deencapsulation.invoke(routineLoadJob, "updateNumOfData", 1L, 1L, 0L, 1L, 1L, false); + + Assert.assertEquals(RoutineLoadJob.JobState.PAUSED, Deencapsulation.getField(routineLoadJob, "state")); + + } + + @Test + public void testUpdateTotalMoreThanBatch() { + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.RUNNING); + Deencapsulation.setField(routineLoadJob, "maxErrorNum", 10); + Deencapsulation.setField(routineLoadJob, "maxBatchRows", 10); + Deencapsulation.setField(routineLoadJob, "currentErrorRows", 1); + Deencapsulation.setField(routineLoadJob, "currentTotalRows", 99); + Deencapsulation.invoke(routineLoadJob, "updateNumOfData", 2L, 0L, 0L, 1L, 1L, false); + + Assert.assertEquals(RoutineLoadJob.JobState.RUNNING, Deencapsulation.getField(routineLoadJob, "state")); + Assert.assertEquals(new Long(0), Deencapsulation.getField(routineLoadJob, "currentErrorRows")); + Assert.assertEquals(new Long(0), Deencapsulation.getField(routineLoadJob, "currentTotalRows")); + + } + + @Test + public void testGetBeIdToConcurrentTaskNum(@Injectable RoutineLoadTaskInfo routineLoadTaskInfo, + @Injectable RoutineLoadTaskInfo routineLoadTaskInfo1) { + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + List routineLoadTaskInfoList = Lists.newArrayList(); + routineLoadTaskInfoList.add(routineLoadTaskInfo); + routineLoadTaskInfoList.add(routineLoadTaskInfo1); + Deencapsulation.setField(routineLoadJob, "routineLoadTaskInfoList", routineLoadTaskInfoList); + + new Expectations() { + { + routineLoadTaskInfo.getBeId(); + result = 1L; + routineLoadTaskInfo1.getBeId(); + result = 1L; + } + }; + + Map beIdConcurrentTasksNum = routineLoadJob.getBeIdToConcurrentTaskNum(); + Assert.assertEquals(2, (int) beIdConcurrentTasksNum.get(1L)); + } + +} diff --git a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java index 8681411a857c7d..33d6586ec80a7c 100644 --- a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java +++ b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java @@ -17,27 +17,34 @@ package org.apache.doris.load.routineload; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import mockit.Deencapsulation; -import mockit.Expectations; -import mockit.Injectable; -import mockit.Mock; -import mockit.MockUp; -import mockit.Mocked; +import static mockit.Deencapsulation.invoke; + import org.apache.doris.analysis.ColumnSeparator; import org.apache.doris.analysis.CreateRoutineLoadStmt; +import org.apache.doris.analysis.LabelName; import org.apache.doris.analysis.ParseNode; +import org.apache.doris.analysis.PauseRoutineLoadStmt; +import org.apache.doris.analysis.ResumeRoutineLoadStmt; +import org.apache.doris.analysis.StopRoutineLoadStmt; import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Database; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.LoadException; +import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; import org.apache.doris.mysql.privilege.PaloAuth; import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.persist.EditLog; +import org.apache.doris.persist.RoutineLoadOperation; import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TResourceInfo; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.junit.Assert; @@ -48,11 +55,18 @@ import java.util.Map; import java.util.UUID; +import mockit.Deencapsulation; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mock; +import mockit.MockUp; +import mockit.Mocked; + public class RoutineLoadManagerTest { private static final Logger LOG = LogManager.getLogger(RoutineLoadManagerTest.class); - private static final int DEFAULT_BE_CONCURRENT_TASK_NUM = 100; + private static final int DEFAULT_BE_CONCURRENT_TASK_NUM = 10; @Mocked private SystemInfoService systemInfoService; @@ -61,9 +75,10 @@ public class RoutineLoadManagerTest { public void testAddJobByStmt(@Injectable PaloAuth paloAuth, @Injectable TResourceInfo tResourceInfo, @Mocked ConnectContext connectContext, - @Mocked Catalog catalog) throws DdlException, LoadException, AnalysisException { + @Mocked Catalog catalog) throws UserException { String jobName = "job1"; String dbName = "db1"; + LabelName labelName = new LabelName(dbName, jobName); String tableNameString = "table1"; TableName tableName = new TableName(dbName, tableNameString); List loadPropertyList = new ArrayList<>(); @@ -76,13 +91,13 @@ public void testAddJobByStmt(@Injectable PaloAuth paloAuth, String topicName = "topic1"; customProperties.put(CreateRoutineLoadStmt.KAFKA_TOPIC_PROPERTY, topicName); String serverAddress = "http://127.0.0.1:8080"; - customProperties.put(CreateRoutineLoadStmt.KAFKA_ENDPOINT_PROPERTY, serverAddress); - CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(jobName, tableName, + customProperties.put(CreateRoutineLoadStmt.KAFKA_BROKER_LIST_PROPERTY, serverAddress); + CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(labelName, tableNameString, loadPropertyList, properties, typeName, customProperties); - KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(jobName, 1L, 1L, serverAddress, topicName); - + KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(1L, jobName, "default_cluster", 1L, 1L, + serverAddress, topicName); new MockUp() { @Mock @@ -95,12 +110,12 @@ public KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) { { catalog.getAuth(); result = paloAuth; - paloAuth.checkTblPriv((ConnectContext) any, dbName, tableNameString, PrivPredicate.LOAD); + paloAuth.checkTblPriv((ConnectContext) any, anyString, anyString, PrivPredicate.LOAD); result = true; } }; RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.addRoutineLoadJob(createRoutineLoadStmt); + routineLoadManager.createRoutineLoadJob(createRoutineLoadStmt, "dummy"); Map idToRoutineLoadJob = Deencapsulation.getField(routineLoadManager, "idToRoutineLoadJob"); @@ -110,6 +125,7 @@ public KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) { Assert.assertEquals(jobName, routineLoadJob.getName()); Assert.assertEquals(1L, routineLoadJob.getTableId()); Assert.assertEquals(RoutineLoadJob.JobState.NEED_SCHEDULE, routineLoadJob.getState()); + Assert.assertEquals(true, routineLoadJob instanceof KafkaRoutineLoadJob); Map>> dbToNameToRoutineLoadJob = Deencapsulation.getField(routineLoadManager, "dbToNameToRoutineLoadJob"); @@ -119,8 +135,6 @@ public KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) { Assert.assertEquals(jobName, nameToRoutineLoadJob.keySet().iterator().next()); Assert.assertEquals(1, nameToRoutineLoadJob.values().size()); Assert.assertEquals(routineLoadJob, nameToRoutineLoadJob.values().iterator().next().get(0)); - - } @Test @@ -130,6 +144,7 @@ public void testCreateJobAuthDeny(@Injectable PaloAuth paloAuth, @Mocked Catalog catalog) { String jobName = "job1"; String dbName = "db1"; + LabelName labelName = new LabelName(dbName, jobName); String tableNameString = "table1"; TableName tableName = new TableName(dbName, tableNameString); List loadPropertyList = new ArrayList<>(); @@ -142,8 +157,8 @@ public void testCreateJobAuthDeny(@Injectable PaloAuth paloAuth, String topicName = "topic1"; customProperties.put(CreateRoutineLoadStmt.KAFKA_TOPIC_PROPERTY, topicName); String serverAddress = "http://127.0.0.1:8080"; - customProperties.put(CreateRoutineLoadStmt.KAFKA_ENDPOINT_PROPERTY, serverAddress); - CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(jobName, tableName, + customProperties.put(CreateRoutineLoadStmt.KAFKA_BROKER_LIST_PROPERTY, serverAddress); + CreateRoutineLoadStmt createRoutineLoadStmt = new CreateRoutineLoadStmt(labelName, tableNameString, loadPropertyList, properties, typeName, customProperties); @@ -152,18 +167,20 @@ public void testCreateJobAuthDeny(@Injectable PaloAuth paloAuth, { catalog.getAuth(); result = paloAuth; - paloAuth.checkTblPriv((ConnectContext) any, dbName, tableNameString, PrivPredicate.LOAD); + paloAuth.checkTblPriv((ConnectContext) any, anyString, anyString, PrivPredicate.LOAD); result = false; } }; RoutineLoadManager routineLoadManager = new RoutineLoadManager(); try { - routineLoadManager.addRoutineLoadJob(createRoutineLoadStmt); + routineLoadManager.createRoutineLoadJob(createRoutineLoadStmt, "dummy"); Assert.fail(); } catch (LoadException | DdlException e) { Assert.fail(); } catch (AnalysisException e) { LOG.info("Access deny"); + } catch (UserException e) { + e.printStackTrace(); } } @@ -172,21 +189,24 @@ public void testCreateWithSameName(@Mocked ConnectContext connectContext) { String jobName = "job1"; String topicName = "topic1"; String serverAddress = "http://127.0.0.1:8080"; - KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(jobName, 1L, 1L, serverAddress, topicName); + KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(1L, jobName, "default_cluster", 1L, 1L, + serverAddress, + topicName); RoutineLoadManager routineLoadManager = new RoutineLoadManager(); Map>> dbToNameToRoutineLoadJob = Maps.newConcurrentMap(); Map> nameToRoutineLoadJob = Maps.newConcurrentMap(); List routineLoadJobList = Lists.newArrayList(); - KafkaRoutineLoadJob kafkaRoutineLoadJobWithSameName = new KafkaRoutineLoadJob(jobName, 1L, 1L, serverAddress, topicName); + KafkaRoutineLoadJob kafkaRoutineLoadJobWithSameName = new KafkaRoutineLoadJob(1L, jobName, "default_cluster", + 1L, 1L, serverAddress, topicName); routineLoadJobList.add(kafkaRoutineLoadJobWithSameName); nameToRoutineLoadJob.put(jobName, routineLoadJobList); dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); try { - routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob); + routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob, "db"); Assert.fail(); } catch (DdlException e) { LOG.info(e.getMessage()); @@ -194,18 +214,29 @@ public void testCreateWithSameName(@Mocked ConnectContext connectContext) { } @Test - public void testCreateWithSameNameOfStoppedJob(@Mocked ConnectContext connectContext) throws DdlException { + public void testCreateWithSameNameOfStoppedJob(@Mocked ConnectContext connectContext, + @Mocked Catalog catalog, + @Mocked EditLog editLog) throws DdlException { String jobName = "job1"; String topicName = "topic1"; String serverAddress = "http://127.0.0.1:8080"; - KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(jobName, 1L, 1L, serverAddress, topicName); + KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(1L, jobName, "default_cluster", 1L, 1L, + serverAddress, topicName); RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + new Expectations() { + { + catalog.getEditLog(); + result = editLog; + } + }; + Map>> dbToNameToRoutineLoadJob = Maps.newConcurrentMap(); Map> nameToRoutineLoadJob = Maps.newConcurrentMap(); List routineLoadJobList = Lists.newArrayList(); - KafkaRoutineLoadJob kafkaRoutineLoadJobWithSameName = new KafkaRoutineLoadJob(jobName, 1L, 1L, serverAddress, topicName); + KafkaRoutineLoadJob kafkaRoutineLoadJobWithSameName = new KafkaRoutineLoadJob(1L, jobName, "default_cluster", + 1L, 1L, serverAddress, topicName); Deencapsulation.setField(kafkaRoutineLoadJobWithSameName, "state", RoutineLoadJob.JobState.STOPPED); routineLoadJobList.add(kafkaRoutineLoadJobWithSameName); nameToRoutineLoadJob.put(jobName, routineLoadJobList); @@ -215,7 +246,7 @@ public void testCreateWithSameNameOfStoppedJob(@Mocked ConnectContext connectCon Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); Deencapsulation.setField(routineLoadManager, "idToRoutineLoadJob", idToRoutineLoadJob); - routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob); + routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob, "db"); Map>> result = Deencapsulation.getField(routineLoadManager, "dbToNameToRoutineLoadJob"); @@ -236,16 +267,81 @@ public void testGetMinTaskBeId() throws LoadException { new Expectations() { { + systemInfoService.getClusterBackendIds(anyString, true); + result = beIds; + systemInfoService.getBackendIds(true); + result = beIds; + Catalog.getCurrentSystemInfo(); + result = systemInfoService; + } + }; + + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map beIdToConcurrentTaskMap = Maps.newHashMap(); + beIdToConcurrentTaskMap.put(1L, 1); + + new Expectations(routineLoadManager) {{ + invoke(routineLoadManager, "getBeIdConcurrentTaskMaps"); + result = beIdToConcurrentTaskMap; + }}; + Assert.assertEquals(2L, routineLoadManager.getMinTaskBeId("default")); + } + + @Test + public void testGetMinTaskBeIdWhileClusterDeleted() { + new Expectations() { + { + systemInfoService.getClusterBackendIds(anyString, true); + result = null; + Catalog.getCurrentSystemInfo(); + result = systemInfoService; + } + }; + + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + try { + routineLoadManager.getMinTaskBeId("default"); + Assert.fail(); + } catch (LoadException e) { + // do nothing + } + + } + + @Test + public void testGetMinTaskBeIdWhileNoSlot(@Injectable RoutineLoadJob routineLoadJob) { + List beIds = Lists.newArrayList(); + beIds.add(1L); + Map beIdToConcurrentTaskMap = Maps.newHashMap(); + beIdToConcurrentTaskMap.put(1L, 11); + + new Expectations() { + { + systemInfoService.getClusterBackendIds(anyString, true); + result = beIds; systemInfoService.getBackendIds(true); result = beIds; Catalog.getCurrentSystemInfo(); result = systemInfoService; + routineLoadJob.getBeIdToConcurrentTaskNum(); + result = beIdToConcurrentTaskMap; + routineLoadJob.getState(); + result = RoutineLoadJob.JobState.RUNNING; } }; RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.addNumOfConcurrentTasksByBeId(1L); - Assert.assertEquals(2L, routineLoadManager.getMinTaskBeId()); + Deencapsulation.setField(RoutineLoadManager.class, "DEFAULT_BE_CONCURRENT_TASK_NUM", 0); + Map routineLoadJobMap = Maps.newHashMap(); + routineLoadJobMap.put(1l, routineLoadJob); + Deencapsulation.setField(routineLoadManager, "idToRoutineLoadJob", routineLoadJobMap); + + try { + routineLoadManager.getMinTaskBeId("default"); + Assert.fail(); + } catch (LoadException e) { + // do nothing + } } @Test @@ -264,7 +360,12 @@ public void testGetTotalIdleTaskNum() { }; RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.addNumOfConcurrentTasksByBeId(1L); + Map beIdToConcurrentTaskMap = Maps.newHashMap(); + beIdToConcurrentTaskMap.put(1L, 1); + new Expectations(routineLoadManager) {{ + invoke(routineLoadManager, "getBeIdConcurrentTaskMaps"); + result = beIdToConcurrentTaskMap; + }}; Assert.assertEquals(DEFAULT_BE_CONCURRENT_TASK_NUM * 2 - 1, routineLoadManager.getClusterIdleSlotNum()); } @@ -288,7 +389,367 @@ public void testUpdateBeIdTaskMaps() { }; RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.updateBeIdTaskMaps(); + routineLoadManager.updateBeIdToMaxConcurrentTasks(); + } + + @Test + public void testGetJobByName(@Injectable RoutineLoadJob routineLoadJob1, + @Injectable RoutineLoadJob routineLoadJob2, + @Injectable RoutineLoadJob routineLoadJob3) { + String jobName = "ilovedoris"; + List routineLoadJobList1 = Lists.newArrayList(); + routineLoadJobList1.add(routineLoadJob1); + routineLoadJobList1.add(routineLoadJob2); + Map> nameToRoutineLoadList1 = Maps.newHashMap(); + nameToRoutineLoadList1.put(jobName, routineLoadJobList1); + + List routineLoadJobList2 = Lists.newArrayList(); + routineLoadJobList2.add(routineLoadJob3); + Map> nameToRoutineLoadList2 = Maps.newHashMap(); + nameToRoutineLoadList2.put(jobName, routineLoadJobList2); + + Map>> dbToNameRoutineLoadList = Maps.newHashMap(); + dbToNameRoutineLoadList.put("db1", nameToRoutineLoadList1); + dbToNameRoutineLoadList.put("db2", nameToRoutineLoadList2); + + new Expectations() { + { + routineLoadJob1.isFinal(); + result = true; + routineLoadJob2.isFinal(); + result = false; + + } + }; + + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameRoutineLoadList); + List result = routineLoadManager.getJobByName(jobName); + + Assert.assertEquals(3, result.size()); + Assert.assertEquals(routineLoadJob2, result.get(0)); + Assert.assertEquals(routineLoadJob1, result.get(1)); + Assert.assertEquals(routineLoadJob3, result.get(2)); + + } + + @Test + public void testGetJob(@Injectable RoutineLoadJob routineLoadJob1, + @Injectable RoutineLoadJob routineLoadJob2, + @Injectable RoutineLoadJob routineLoadJob3) throws MetaNotFoundException { + + new Expectations() { + { + routineLoadJob1.isFinal(); + result = true; + routineLoadJob2.isFinal(); + result = false; + routineLoadJob3.isFinal(); + result = true; + } + }; + + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map idToRoutineLoadJob = Maps.newHashMap(); + idToRoutineLoadJob.put(1L, routineLoadJob1); + idToRoutineLoadJob.put(2L, routineLoadJob2); + idToRoutineLoadJob.put(3L, routineLoadJob3); + Deencapsulation.setField(routineLoadManager, "idToRoutineLoadJob", idToRoutineLoadJob); + List result = routineLoadManager.getJob(null, null, true); + + Assert.assertEquals(3, result.size()); + Assert.assertEquals(routineLoadJob2, result.get(0)); + Assert.assertEquals(routineLoadJob1, result.get(1)); + Assert.assertEquals(routineLoadJob3, result.get(2)); + } + + @Test + public void testGetJobIncludeHistory(@Injectable RoutineLoadJob routineLoadJob1, + @Injectable RoutineLoadJob routineLoadJob2, + @Injectable RoutineLoadJob routineLoadJob3, + @Mocked Catalog catalog, + @Mocked Database database) throws MetaNotFoundException { + new Expectations() { + { + routineLoadJob1.isFinal(); + result = true; + routineLoadJob2.isFinal(); + result = false; + routineLoadJob3.isFinal(); + result = true; + catalog.getDb(anyString); + result = database; + database.getId(); + result = 1L; + } + }; + + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map>> dbToNameToRoutineLoadJob = Maps.newHashMap(); + Map> nameToRoutineLoadJob = Maps.newHashMap(); + List routineLoadJobList = Lists.newArrayList(); + routineLoadJobList.add(routineLoadJob1); + routineLoadJobList.add(routineLoadJob2); + routineLoadJobList.add(routineLoadJob3); + nameToRoutineLoadJob.put("", routineLoadJobList); + dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); + List result = routineLoadManager.getJob("", "", true); + + Assert.assertEquals(3, result.size()); + Assert.assertEquals(routineLoadJob2, result.get(0)); + Assert.assertEquals(routineLoadJob1, result.get(1)); + Assert.assertEquals(routineLoadJob3, result.get(2)); + } + + @Test + public void testPauseRoutineLoadJob(@Injectable PauseRoutineLoadStmt pauseRoutineLoadStmt, + @Mocked Catalog catalog, + @Mocked Database database, + @Mocked PaloAuth paloAuth, + @Mocked ConnectContext connectContext) throws UserException { + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map>> dbToNameToRoutineLoadJob = Maps.newHashMap(); + Map> nameToRoutineLoadJob = Maps.newHashMap(); + List routineLoadJobList = Lists.newArrayList(); + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + routineLoadJobList.add(routineLoadJob); + nameToRoutineLoadJob.put("", routineLoadJobList); + dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); + + new Expectations() { + { + pauseRoutineLoadStmt.getDbFullName(); + result = ""; + pauseRoutineLoadStmt.getName(); + result = ""; + catalog.getDb(""); + result = database; + database.getId(); + result = 1L; + catalog.getAuth(); + result = paloAuth; + paloAuth.checkTblPriv((ConnectContext) any, anyString, anyString, (PrivPredicate) any); + result = true; + } + }; + + routineLoadManager.pauseRoutineLoadJob(pauseRoutineLoadStmt); + + Assert.assertEquals(RoutineLoadJob.JobState.PAUSED, routineLoadJob.getState()); + } + + @Test + public void testResumeRoutineLoadJob(@Injectable ResumeRoutineLoadStmt resumeRoutineLoadStmt, + @Mocked Catalog catalog, + @Mocked Database database, + @Mocked PaloAuth paloAuth, + @Mocked ConnectContext connectContext) throws UserException { + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map>> dbToNameToRoutineLoadJob = Maps.newHashMap(); + Map> nameToRoutineLoadJob = Maps.newHashMap(); + List routineLoadJobList = Lists.newArrayList(); + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + routineLoadJobList.add(routineLoadJob); + nameToRoutineLoadJob.put("", routineLoadJobList); + dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); + + new Expectations() { + { + resumeRoutineLoadStmt.getDbFullName(); + result = ""; + resumeRoutineLoadStmt.getName(); + result = ""; + catalog.getDb(""); + result = database; + database.getId(); + result = 1L; + catalog.getAuth(); + result = paloAuth; + paloAuth.checkTblPriv((ConnectContext) any, anyString, anyString, (PrivPredicate) any); + result = true; + } + }; + + routineLoadManager.resumeRoutineLoadJob(resumeRoutineLoadStmt); + + Assert.assertEquals(RoutineLoadJob.JobState.NEED_SCHEDULE, routineLoadJob.getState()); + } + + @Test + public void testStopRoutineLoadJob(@Injectable StopRoutineLoadStmt stopRoutineLoadStmt, + @Mocked Catalog catalog, + @Mocked Database database, + @Mocked PaloAuth paloAuth, + @Mocked ConnectContext connectContext) throws UserException { + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map>> dbToNameToRoutineLoadJob = Maps.newHashMap(); + Map> nameToRoutineLoadJob = Maps.newHashMap(); + List routineLoadJobList = Lists.newArrayList(); + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + routineLoadJobList.add(routineLoadJob); + nameToRoutineLoadJob.put("", routineLoadJobList); + dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); + + new Expectations() { + { + stopRoutineLoadStmt.getDbFullName(); + result = ""; + stopRoutineLoadStmt.getName(); + result = ""; + catalog.getDb(""); + result = database; + database.getId(); + result = 1L; + catalog.getAuth(); + result = paloAuth; + paloAuth.checkTblPriv((ConnectContext) any, anyString, anyString, (PrivPredicate) any); + result = true; + } + }; + + routineLoadManager.stopRoutineLoadJob(stopRoutineLoadStmt); + + Assert.assertEquals(RoutineLoadJob.JobState.STOPPED, routineLoadJob.getState()); + } + + @Test + public void testCheckBeToTask(@Mocked Catalog catalog, + @Mocked SystemInfoService systemInfoService) throws LoadException { + List beIdsInCluster = Lists.newArrayList(); + beIdsInCluster.add(1L); + Map beIdToMaxConcurrentTasks = Maps.newHashMap(); + beIdToMaxConcurrentTasks.put(1L, 10); + new Expectations() { + { + systemInfoService.getClusterBackendIds("default", true); + result = beIdsInCluster; + } + }; + + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Deencapsulation.setField(routineLoadManager, "beIdToMaxConcurrentTasks", beIdToMaxConcurrentTasks); + Assert.assertEquals(true, routineLoadManager.checkBeToTask(1L, "default")); + } + + @Test + public void testCleanOldRoutineLoadJobs(@Injectable RoutineLoadJob routineLoadJob, + @Mocked Catalog catalog, + @Mocked EditLog editLog) { + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map>> dbToNameToRoutineLoadJob = Maps.newHashMap(); + Map> nameToRoutineLoadJob = Maps.newHashMap(); + List routineLoadJobList = Lists.newArrayList(); + routineLoadJobList.add(routineLoadJob); + nameToRoutineLoadJob.put("", routineLoadJobList); + dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); + Map idToRoutineLoadJob = Maps.newHashMap(); + idToRoutineLoadJob.put(1L, routineLoadJob); + Deencapsulation.setField(routineLoadManager, "idToRoutineLoadJob", idToRoutineLoadJob); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); + + new Expectations() { + { + routineLoadJob.needRemove(); + result = true; + routineLoadJob.getDbId(); + result = 1L; + routineLoadJob.getName(); + result = ""; + catalog.getEditLog(); + result = editLog; + } + }; + routineLoadManager.cleanOldRoutineLoadJobs(); + + Assert.assertEquals(0, dbToNameToRoutineLoadJob.size()); + Assert.assertEquals(0, idToRoutineLoadJob.size()); + } + + @Test + public void testGetBeIdConcurrentTaskMaps(@Injectable RoutineLoadJob routineLoadJob) { + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map idToRoutineLoadJob = Maps.newHashMap(); + idToRoutineLoadJob.put(1L, routineLoadJob); + Deencapsulation.setField(routineLoadManager, "idToRoutineLoadJob", idToRoutineLoadJob); + Map beIdToConcurrenTaskNum = Maps.newHashMap(); + beIdToConcurrenTaskNum.put(1L, 1); + + new Expectations() { + { + routineLoadJob.getState(); + result = RoutineLoadJob.JobState.RUNNING; + routineLoadJob.getBeIdToConcurrentTaskNum(); + result = beIdToConcurrenTaskNum; + } + }; + + Map result = Deencapsulation.invoke(routineLoadManager, "getBeIdConcurrentTaskMaps"); + Assert.assertEquals(1, (int) result.get(1l)); + + } + + @Test + public void testReplayRemoveOldRoutineLoad(@Injectable RoutineLoadOperation operation, + @Injectable RoutineLoadJob routineLoadJob) { + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + Map idToRoutineLoadJob = Maps.newHashMap(); + idToRoutineLoadJob.put(1L, routineLoadJob); + Deencapsulation.setField(routineLoadManager, "idToRoutineLoadJob", idToRoutineLoadJob); + Map>> dbToNameToRoutineLoadJob = Maps.newHashMap(); + Map> nameToRoutineLoadJob = Maps.newHashMap(); + List routineLoadJobList = Lists.newArrayList(); + routineLoadJobList.add(routineLoadJob); + nameToRoutineLoadJob.put("", routineLoadJobList); + dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); + + new Expectations() { + { + routineLoadJob.getName(); + result = ""; + routineLoadJob.getDbId(); + result = 1L; + operation.getId(); + result = 1L; + } + }; + + routineLoadManager.replayRemoveOldRoutineLoad(operation); + Assert.assertEquals(0, idToRoutineLoadJob.size()); + } + + @Test + public void testReplayChangeRoutineLoadJob(@Injectable RoutineLoadOperation operation) { + RoutineLoadManager routineLoadManager = new RoutineLoadManager(); + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); + Deencapsulation.setField(routineLoadJob, "name", ""); + Deencapsulation.setField(routineLoadJob, "dbId", 1L); + Map idToRoutineLoadJob = Maps.newHashMap(); + idToRoutineLoadJob.put(1L, routineLoadJob); + Deencapsulation.setField(routineLoadManager, "idToRoutineLoadJob", idToRoutineLoadJob); + Map>> dbToNameToRoutineLoadJob = Maps.newHashMap(); + Map> nameToRoutineLoadJob = Maps.newHashMap(); + List routineLoadJobList = Lists.newArrayList(); + routineLoadJobList.add(routineLoadJob); + nameToRoutineLoadJob.put("", routineLoadJobList); + dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); + Deencapsulation.setField(routineLoadManager, "dbToNameToRoutineLoadJob", dbToNameToRoutineLoadJob); + + new Expectations() { + { + operation.getId(); + result = 1L; + operation.getJobState(); + result = RoutineLoadJob.JobState.PAUSED; + } + }; + + routineLoadManager.replayChangeRoutineLoadJob(operation); + Assert.assertEquals(RoutineLoadJob.JobState.PAUSED, routineLoadJob.getState()); } } diff --git a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java index b0e54a032cb937..e14f2383597a46 100644 --- a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java +++ b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java @@ -17,21 +17,20 @@ package org.apache.doris.load.routineload; -import com.google.common.collect.Lists; -import mockit.Deencapsulation; -import mockit.Expectations; -import mockit.Injectable; -import mockit.Mocked; -import org.apache.doris.common.DdlException; -import org.apache.doris.load.RoutineLoadDesc; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.common.DdlException; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.load.RoutineLoadDesc; +import org.apache.doris.planner.StreamLoadPlanner; import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TResourceInfo; +import com.google.common.collect.Lists; + import org.apache.kafka.clients.consumer.KafkaConsumer; import org.junit.Assert; import org.junit.Test; @@ -41,6 +40,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import mockit.Deencapsulation; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; + public class RoutineLoadSchedulerTest { @Mocked @@ -54,9 +58,12 @@ public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, @Injectable RoutineLoadManager routineLoadManager, @Injectable SystemInfoService systemInfoService, @Injectable Database database, - @Injectable RoutineLoadDesc routineLoadDesc) + @Injectable RoutineLoadDesc routineLoadDesc, + @Injectable RoutineLoadTaskScheduler routineLoadTaskScheduler, + @Mocked StreamLoadPlanner planner, + @Injectable OlapTable olapTable) throws LoadException, MetaNotFoundException { - String clusterName = "cluster1"; + String clusterName = "default"; List beIds = Lists.newArrayList(); beIds.add(1L); beIds.add(2L); @@ -66,24 +73,15 @@ public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, partitions.add(200); partitions.add(300); - new Expectations(){ - { - connectContext.toResourceCtx(); - result = tResourceInfo; - } - }; - - RoutineLoadJob routineLoadJob = - new KafkaRoutineLoadJob("1", "kafka_routine_load_job", 1L, - 1L, routineLoadDesc ,3, 0, - "", "", new KafkaProgress()); - routineLoadJob.setState(RoutineLoadJob.JobState.NEED_SCHEDULE); + KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(1L, "test", clusterName, 1L, 1L, + "xxx", "test"); + Deencapsulation.setField(kafkaRoutineLoadJob,"state", RoutineLoadJob.JobState.NEED_SCHEDULE); List routineLoadJobList = new ArrayList<>(); - routineLoadJobList.add(routineLoadJob); + routineLoadJobList.add(kafkaRoutineLoadJob); - Deencapsulation.setField(routineLoadJob, "customKafkaPartitions", partitions); - Deencapsulation.setField(routineLoadJob, "desireTaskConcurrentNum", 3); - Deencapsulation.setField(routineLoadJob, "consumer", consumer); + Deencapsulation.setField(kafkaRoutineLoadJob, "customKafkaPartitions", partitions); + Deencapsulation.setField(kafkaRoutineLoadJob, "desireTaskConcurrentNum", 3); + Deencapsulation.setField(kafkaRoutineLoadJob, "consumer", consumer); new Expectations() { { @@ -93,12 +91,19 @@ public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, result = routineLoadJobList; catalog.getDb(anyLong); result = database; - systemInfoService.getBackendIds( true); + database.getTable(1L); + result = olapTable; + systemInfoService.getClusterBackendIds(clusterName, true); result = beIds; routineLoadManager.getSizeOfIdToRoutineLoadTask(); result = 1; + minTimes = 0; routineLoadManager.getTotalMaxConcurrentTaskNum(); result = 10; + minTimes = 0; + catalog.getRoutineLoadTaskScheduler(); + result = routineLoadTaskScheduler; + minTimes = 0; } }; @@ -106,8 +111,9 @@ public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, Deencapsulation.setField(routineLoadScheduler, "routineLoadManager", routineLoadManager); routineLoadScheduler.runOneCycle(); - Assert.assertEquals(2, routineLoadJob.getNeedScheduleTaskInfoList().size()); - for (RoutineLoadTaskInfo routineLoadTaskInfo : routineLoadJob.getNeedScheduleTaskInfoList()) { + List routineLoadTaskInfoList = + Deencapsulation.getField(kafkaRoutineLoadJob, "routineLoadTaskInfoList"); + for (RoutineLoadTaskInfo routineLoadTaskInfo : routineLoadTaskInfoList) { KafkaTaskInfo kafkaTaskInfo = (KafkaTaskInfo) routineLoadTaskInfo; if (kafkaTaskInfo.getPartitions().size() == 2) { Assert.assertTrue(kafkaTaskInfo.getPartitions().contains(100)); @@ -118,7 +124,6 @@ public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, } } - public void functionTest(@Mocked Catalog catalog, @Mocked SystemInfoService systemInfoService, @Injectable Database database) throws DdlException, InterruptedException { @@ -129,9 +134,10 @@ public void functionTest(@Mocked Catalog catalog, } }; - KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob("test", 1L, 1L, "10.74.167.16:8092", "test"); + KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(1L, "test", "default_cluster", 1L, 1L, + "10.74.167.16:8092", "test"); RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob); + routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob, "db"); List backendIds = new ArrayList<>(); backendIds.add(1L); @@ -156,13 +162,12 @@ public void functionTest(@Mocked Catalog catalog, executorService.submit(routineLoadScheduler); executorService.submit(routineLoadTaskScheduler); - - - KafkaRoutineLoadJob kafkaRoutineLoadJob1 = new KafkaRoutineLoadJob("test_custom_partition", 1L, 1L, "10.74.167.16:8092", "test_1"); + KafkaRoutineLoadJob kafkaRoutineLoadJob1 = new KafkaRoutineLoadJob(1L, "test_custom_partition", + "default_cluster", 1L, 1L, "xxx", "test_1"); List customKafkaPartitions = new ArrayList<>(); customKafkaPartitions.add(2); Deencapsulation.setField(kafkaRoutineLoadJob1, "customKafkaPartitions", customKafkaPartitions); - routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob1); + routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob1, "db"); Thread.sleep(10000); } diff --git a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java index c9a4daadc39ecc..b96b59dc5b4167 100644 --- a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java +++ b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java @@ -17,34 +17,36 @@ package org.apache.doris.load.routineload; +import mockit.Verifications; import org.apache.doris.analysis.LoadColumnsInfo; import org.apache.doris.catalog.Catalog; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ClientPool; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.load.RoutineLoadDesc; -import org.apache.doris.task.AgentTask; import org.apache.doris.task.AgentTaskExecutor; -import org.apache.doris.task.AgentTaskQueue; -import org.apache.doris.task.KafkaRoutineLoadTask; -import org.apache.doris.thrift.TTaskType; +import org.apache.doris.thrift.BackendService; import org.apache.doris.transaction.BeginTransactionException; import com.google.common.collect.Maps; import com.google.common.collect.Queues; -import org.junit.Assert; +import org.apache.doris.transaction.GlobalTransactionMgr; import org.junit.Test; +import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.UUID; import mockit.Deencapsulation; import mockit.Expectations; import mockit.Injectable; import mockit.Mocked; -import mockit.Verifications; + +import static mockit.Deencapsulation.invoke; public class RoutineLoadTaskSchedulerTest { @@ -59,26 +61,26 @@ public class RoutineLoadTaskSchedulerTest { public void testRunOneCycle(@Injectable KafkaRoutineLoadJob kafkaRoutineLoadJob1, @Injectable KafkaRoutineLoadJob routineLoadJob, @Injectable RoutineLoadDesc routineLoadDesc, - @Injectable LoadColumnsInfo loadColumnsInfo) throws LoadException, + @Injectable LoadColumnsInfo loadColumnsInfo, + @Mocked GlobalTransactionMgr globalTransactionMgr, + @Mocked BackendService.Client client, + @Mocked ClientPool clientPool) throws LoadException, MetaNotFoundException, AnalysisException, LabelAlreadyUsedException, BeginTransactionException { long beId = 100L; + Map partitionIdToOffset = Maps.newHashMap(); + partitionIdToOffset.put(1, 100L); + partitionIdToOffset.put(2, 200L); + KafkaProgress kafkaProgress = new KafkaProgress(); + Deencapsulation.setField(kafkaProgress, "partitionIdToOffset", partitionIdToOffset); + Queue routineLoadTaskInfoQueue = Queues.newLinkedBlockingQueue(); - KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo("1", "1"); - routineLoadTaskInfo1.addKafkaPartition(1); - routineLoadTaskInfo1.addKafkaPartition(2); + KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo(new UUID(1, 1), 1l, "default_cluster", partitionIdToOffset); routineLoadTaskInfoQueue.add(routineLoadTaskInfo1); - Map idToRoutineLoadTask = Maps.newHashMap(); idToRoutineLoadTask.put(1L, routineLoadTaskInfo1); - Map partitionIdToOffset = Maps.newHashMap(); - partitionIdToOffset.put(1, 100L); - partitionIdToOffset.put(2, 200L); - KafkaProgress kafkaProgress = new KafkaProgress(); - kafkaProgress.setPartitionIdToOffset(partitionIdToOffset); - Map idToRoutineLoadJob = Maps.newConcurrentMap(); idToRoutineLoadJob.put("1", routineLoadJob); @@ -92,63 +94,26 @@ public void testRunOneCycle(@Injectable KafkaRoutineLoadJob kafkaRoutineLoadJob1 result = routineLoadManager; Catalog.getCurrentCatalog(); result = catalog; - catalog.getNextId(); - result = 2L; routineLoadManager.getClusterIdleSlotNum(); result = 1; times = 1; + routineLoadManager.checkTaskInJob((UUID) any); + result = true; kafkaRoutineLoadJob1.getDbId(); result = 1L; kafkaRoutineLoadJob1.getTableId(); result = 1L; - routineLoadDesc.getColumnsInfo(); - result = loadColumnsInfo; - routineLoadDesc.getColumnSeparator(); - result = ""; - kafkaRoutineLoadJob1.getProgress(); - result = kafkaProgress; - - routineLoadManager.getNeedScheduleTasksQueue(); - result = routineLoadTaskInfoQueue; - routineLoadManager.getMinTaskBeId(); + routineLoadManager.getMinTaskBeId(anyString); result = beId; - routineLoadManager.getJobByTaskId(anyString); - result = kafkaRoutineLoadJob1; - routineLoadManager.getJob(anyString); + routineLoadManager.getJob(anyLong); result = kafkaRoutineLoadJob1; } }; - KafkaRoutineLoadTask kafkaRoutineLoadTask = new KafkaRoutineLoadTask(kafkaRoutineLoadJob1.getResourceInfo(), - beId, kafkaRoutineLoadJob1.getDbId(), kafkaRoutineLoadJob1.getTableId(), - "", 0L, partitionIdToOffset); -// -// new Expectations() { -// { -// routineLoadTaskInfo1.createStreamLoadTask(anyLong); -// result = kafkaRoutineLoadTask; -// } -// }; - RoutineLoadTaskScheduler routineLoadTaskScheduler = new RoutineLoadTaskScheduler(); + Deencapsulation.setField(routineLoadTaskScheduler, "needScheduleTasksQueue", routineLoadTaskInfoQueue); routineLoadTaskScheduler.runOneCycle(); - - new Verifications() { - { - AgentTask routineLoadTask = - AgentTaskQueue.getTask(beId, TTaskType.STREAM_LOAD, 2L); - - Assert.assertEquals(beId, routineLoadTask.getBackendId()); - Assert.assertEquals(100L, - (long) ((KafkaRoutineLoadTask) routineLoadTask).getPartitionIdToOffset().get(1)); - Assert.assertEquals(200L, - (long) ((KafkaRoutineLoadTask) routineLoadTask).getPartitionIdToOffset().get(2)); - - routineLoadManager.addNumOfConcurrentTasksByBeId(beId); - times = 1; - } - }; } } diff --git a/fe/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java b/fe/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java index aff31fc986e4ed..0457937892d1e4 100644 --- a/fe/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java +++ b/fe/src/test/java/org/apache/doris/planner/StreamLoadPlannerTest.java @@ -23,6 +23,9 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.common.UserException; +import org.apache.doris.task.StreamLoadTask; +import org.apache.doris.thrift.TFileFormatType; +import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TStreamLoadPutRequest; import org.apache.doris.thrift.TUniqueId; @@ -70,7 +73,10 @@ public void testNormalPlan() throws UserException { TStreamLoadPutRequest request = new TStreamLoadPutRequest(); request.setTxnId(1); request.setLoadId(new TUniqueId(2, 3)); - StreamLoadPlanner planner = new StreamLoadPlanner(db, destTable, request); + request.setFileType(TFileType.FILE_STREAM); + request.setFormatType(TFileFormatType.FORMAT_CSV_PLAIN); + StreamLoadPlanner planner = new StreamLoadPlanner(db, destTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); planner.plan(); } } \ No newline at end of file diff --git a/fe/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java b/fe/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java index bf26a8cdcd49b5..928b8f151e30c3 100644 --- a/fe/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java +++ b/fe/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java @@ -35,7 +35,9 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.task.StreamLoadTask; import org.apache.doris.thrift.TExplainLevel; +import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TFileType; import org.apache.doris.thrift.TPlanNode; import org.apache.doris.thrift.TStreamLoadPutRequest; @@ -71,6 +73,7 @@ public class StreamLoadScanNodeTest { TStreamLoadPutRequest getBaseRequest() { TStreamLoadPutRequest request = new TStreamLoadPutRequest(); request.setFileType(TFileType.FILE_STREAM); + request.setFormatType(TFileFormatType.FORMAT_CSV_PLAIN); return request; } @@ -140,7 +143,8 @@ public void testNormal() throws UserException { } TStreamLoadPutRequest request = getBaseRequest(); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); new Expectations() {{ dstTable.getBaseSchema(); result = columns; }}; @@ -174,7 +178,8 @@ public void testLostV2() throws UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1, k2, v1"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -203,7 +208,8 @@ public void testBadColumns() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1 k2 v1"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -247,9 +253,9 @@ public void testColumnsNormal() throws UserException, UserException { }; TStreamLoadPutRequest request = getBaseRequest(); - request.setFileType(TFileType.FILE_LOCAL); request.setColumns("k1,k2,v1, v2=k2"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -295,9 +301,10 @@ public void testHllColumnsNormal() throws UserException { }; TStreamLoadPutRequest request = getBaseRequest(); - request.setFileType(TFileType.FILE_LOCAL); + request.setFileType(TFileType.FILE_STREAM); request.setColumns("k1,k2, v1=hll_hash(k2)"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -324,28 +331,35 @@ public void testHllColumnsNoHllHash() throws UserException { } } - new Expectations() {{ - catalog.getFunction((Function) any, (Function.CompareMode) any); - result = new ScalarFunction(new FunctionName("hll_hash1"), Lists.newArrayList(), Type.BIGINT, false); - }}; + new Expectations() { + { + catalog.getFunction((Function) any, (Function.CompareMode) any); + result = new ScalarFunction(new FunctionName("hll_hash1"), Lists.newArrayList(), Type.BIGINT, false); + minTimes = 0; + } + }; new Expectations() { { dstTable.getColumn("k1"); result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get(); + minTimes = 0; dstTable.getColumn("k2"); result = null; + minTimes = 0; dstTable.getColumn("v1"); result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get(); + minTimes = 0; } }; TStreamLoadPutRequest request = getBaseRequest(); request.setFileType(TFileType.FILE_LOCAL); request.setColumns("k1,k2, v1=hll_hash1(k2)"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -375,7 +389,8 @@ public void testHllColumnsFail() throws UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setFileType(TFileType.FILE_LOCAL); request.setColumns("k1,k2, v1=k2"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -405,7 +420,8 @@ public void testUnsupportedFType() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setFileType(TFileType.FILE_BROKER); request.setColumns("k1,k2,v1, v2=k2"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -434,7 +450,8 @@ public void testColumnsUnknownRef() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1,k2,v1, v2=k3"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -480,7 +497,8 @@ public void testWhereNormal() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1,k2,v1, v2=k1"); request.setWhere("k1 = 1"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -511,22 +529,27 @@ public void testWhereBad() throws UserException, UserException { { dstTable.getColumn("k1"); result = columns.stream().filter(c -> c.getName().equals("k1")).findFirst().get(); + minTimes = 0; dstTable.getColumn("k2"); result = columns.stream().filter(c -> c.getName().equals("k2")).findFirst().get(); + minTimes = 0; dstTable.getColumn("v1"); result = columns.stream().filter(c -> c.getName().equals("v1")).findFirst().get(); + minTimes = 0; dstTable.getColumn("v2"); result = columns.stream().filter(c -> c.getName().equals("v2")).findFirst().get(); + minTimes = 0; } }; TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1,k2,v1, v2=k2"); request.setWhere("k1 1"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -556,7 +579,8 @@ public void testWhereUnknownRef() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1,k2,v1, v2=k1"); request.setWhere("k5 = 1"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); @@ -586,7 +610,8 @@ public void testWhereNotBool() throws UserException, UserException { TStreamLoadPutRequest request = getBaseRequest(); request.setColumns("k1,k2,v1, v2=k1"); request.setWhere("k1 + v2"); - StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, request); + StreamLoadScanNode scanNode = new StreamLoadScanNode(new PlanNodeId(1), dstDesc, dstTable, + StreamLoadTask.fromTStreamLoadPutRequest(request)); scanNode.init(analyzer); scanNode.finalize(analyzer); diff --git a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java index 3b12236e856644..a8f2b93320d741 100644 --- a/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/StmtExecutorTest.java @@ -453,7 +453,7 @@ public void testDdl() throws Exception { // Mock ddl PowerMock.mockStatic(DdlExecutor.class); - DdlExecutor.execute(EasyMock.isA(Catalog.class), EasyMock.isA(DdlStmt.class)); + DdlExecutor.execute(EasyMock.isA(Catalog.class), EasyMock.isA(DdlStmt.class), EasyMock.anyString()); EasyMock.expectLastCall().anyTimes(); PowerMock.replay(DdlExecutor.class); @@ -481,7 +481,7 @@ public void testDdlFail() throws Exception { // Mock ddl PowerMock.mockStatic(DdlExecutor.class); - DdlExecutor.execute(EasyMock.isA(Catalog.class), EasyMock.isA(DdlStmt.class)); + DdlExecutor.execute(EasyMock.isA(Catalog.class), EasyMock.isA(DdlStmt.class), EasyMock.anyString()); EasyMock.expectLastCall().andThrow(new DdlException("ddl fail")); PowerMock.replay(DdlExecutor.class); @@ -509,7 +509,7 @@ public void testDdlFail2() throws Exception { // Mock ddl PowerMock.mockStatic(DdlExecutor.class); - DdlExecutor.execute(EasyMock.isA(Catalog.class), EasyMock.isA(DdlStmt.class)); + DdlExecutor.execute(EasyMock.isA(Catalog.class), EasyMock.isA(DdlStmt.class), EasyMock.anyString()); EasyMock.expectLastCall().andThrow(new Exception("bug")); PowerMock.replay(DdlExecutor.class); @@ -527,7 +527,6 @@ public void testUse() throws Exception { EasyMock.expect(useStmt.getDatabase()).andReturn("testDb").anyTimes(); EasyMock.expect(useStmt.getRedirectStatus()).andReturn(RedirectStatus.NO_FORWARD).anyTimes(); EasyMock.expect(useStmt.getClusterName()).andReturn("testCluster").anyTimes(); - EasyMock.replay(useStmt); Symbol symbol = new Symbol(0, useStmt); @@ -535,9 +534,6 @@ public void testUse() throws Exception { EasyMock.expect(parser.parse()).andReturn(symbol).anyTimes(); EasyMock.replay(parser); - PowerMock.expectNew(SqlParser.class, EasyMock.isA(SqlScanner.class)).andReturn(parser); - PowerMock.replay(SqlParser.class); - StmtExecutor executor = new StmtExecutor(ctx, ""); executor.execute(); diff --git a/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index a25da4b87f4c0e..5abc132e8ab415 100644 --- a/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -22,25 +22,18 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import com.google.common.collect.Maps; -import mockit.Deencapsulation; -import mockit.Expectations; -import mockit.Injectable; -import mockit.Mocked; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.CatalogTestUtil; -import org.apache.doris.catalog.Database; import org.apache.doris.catalog.FakeCatalog; import org.apache.doris.catalog.FakeEditLog; -import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.DdlException; +import org.apache.doris.common.Config; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.LabelAlreadyUsedException; -import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.UserException; import org.apache.doris.load.routineload.KafkaProgress; import org.apache.doris.load.routineload.KafkaRoutineLoadJob; import org.apache.doris.load.routineload.KafkaTaskInfo; @@ -49,14 +42,15 @@ import org.apache.doris.load.routineload.RoutineLoadManager; import org.apache.doris.load.routineload.RoutineLoadTaskInfo; import org.apache.doris.meta.MetaContext; -import org.apache.doris.qe.ConnectContext; +import org.apache.doris.persist.EditLog; import org.apache.doris.thrift.TKafkaRLTaskProgress; +import org.apache.doris.thrift.TLoadSourceType; import org.apache.doris.thrift.TRLTaskTxnCommitAttachment; -import org.apache.doris.thrift.TResourceInfo; -import org.apache.doris.thrift.TRoutineLoadType; +import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TransactionState.LoadJobSourceType; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -65,10 +59,14 @@ import org.junit.Test; import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; + +import mockit.Deencapsulation; +import mockit.Injectable; +import mockit.Mocked; public class GlobalTransactionMgrTest { @@ -82,7 +80,6 @@ public class GlobalTransactionMgrTest { private String transactionSource = "localfe"; - @Before public void setUp() throws InstantiationException, IllegalAccessException, IllegalArgumentException, InvocationTargetException, NoSuchMethodException, SecurityException { @@ -111,8 +108,8 @@ public void testBeginTransaction() throws LabelAlreadyUsedException, AnalysisExc long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); - TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); + TransactionState transactionState = masterTransMgr.getTransactionState(transactionId); assertNotNull(transactionState); assertEquals(transactionId, transactionState.getTransactionId()); assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); @@ -125,18 +122,17 @@ public void testBeginTransactionWithSameLabel() throws LabelAlreadyUsedException BeginTransactionException { FakeCatalog.setCatalog(masterCatalog); long transactionId = 0; - Throwable throwable = null; try { transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); } catch (AnalysisException e) { e.printStackTrace(); } catch (LabelAlreadyUsedException e) { e.printStackTrace(); } - TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + TransactionState transactionState = masterTransMgr.getTransactionState(transactionId); assertNotNull(transactionState); assertEquals(transactionId, transactionState.getTransactionId()); assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); @@ -147,7 +143,7 @@ public void testBeginTransactionWithSameLabel() throws LabelAlreadyUsedException transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); } catch (Exception e) { // TODO: handle exception } @@ -155,15 +151,12 @@ public void testBeginTransactionWithSameLabel() throws LabelAlreadyUsedException // all replica committed success @Test - public void testCommitTransaction1() throws MetaNotFoundException, - TransactionException, - IllegalTransactionParameterException, LabelAlreadyUsedException, - AnalysisException, BeginTransactionException { + public void testCommitTransaction1() throws UserException { FakeCatalog.setCatalog(masterCatalog); long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); // commit a transaction TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); @@ -198,16 +191,13 @@ public void testCommitTransaction1() throws MetaNotFoundException, // commit with only two replicas @Test - public void testCommitTransactionWithOneFailed() throws MetaNotFoundException, - TransactionException, - IllegalTransactionParameterException, LabelAlreadyUsedException, - AnalysisException, BeginTransactionException { + public void testCommitTransactionWithOneFailed() throws UserException { TransactionState transactionState = null; FakeCatalog.setCatalog(masterCatalog); long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); // commit a transaction with 1,2 success TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); @@ -229,7 +219,7 @@ public void testCommitTransactionWithOneFailed() throws MetaNotFoundException, long transactionId2 = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable2, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId3); @@ -237,7 +227,7 @@ public void testCommitTransactionWithOneFailed() throws MetaNotFoundException, transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); - transactionState = fakeEditLog.getTransaction(transactionId2); + transactionState = masterTransMgr.getTransactionState(transactionId2); // check status is prepare, because the commit failed assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); // check replica version @@ -303,160 +293,140 @@ public void testCommitTransactionWithOneFailed() throws MetaNotFoundException, @Test public void testCommitRoutineLoadTransaction(@Injectable TabletCommitInfo tabletCommitInfo, - @Injectable Database database, - @Injectable KafkaTaskInfo routineLoadTaskInfo, - @Injectable TResourceInfo tResourceInfo, - @Injectable OlapTable olapTable, - @Mocked Catalog catalog, - @Mocked ConnectContext connectContext, - @Mocked KafkaConsumer kafkaConsumer) - throws MetaNotFoundException, TransactionException, DdlException { - List tabletCommitInfoList = new ArrayList<>(); - tabletCommitInfoList.add(tabletCommitInfo); - - KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob("test", 1L, 1L, "host:port", "topic"); + @Mocked KafkaConsumer kafkaConsumer, + @Mocked EditLog editLog) + throws UserException { + FakeCatalog.setCatalog(masterCatalog); + + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId2); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId3); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + + KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(1L, "test", "default_cluster", 1L, 1L, "host:port", "topic"); List routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); + Map partitionIdToOffset = Maps.newHashMap(); + partitionIdToOffset.put(1, 0L); + KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "defualt_cluster", partitionIdToOffset); + Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L); routineLoadTaskInfoList.add(routineLoadTaskInfo); - TransactionState transactionState = new TransactionState(1L, 1L, "label", 1L, LoadJobSourceType.ROUTINE_LOAD_TASK, "be1"); + TransactionState transactionState = new TransactionState(1L, 1L, "label", 1L, + LoadJobSourceType.ROUTINE_LOAD_TASK, "be1", routineLoadJob.getId(), + Config.stream_load_default_timeout_second); transactionState.setTransactionStatus(TransactionStatus.PREPARE); - transactionState.setTxnStateChangeListener(routineLoadJob); + masterTransMgr.getCallbackFactory().addCallback(routineLoadJob); + // Deencapsulation.setField(transactionState, "txnStateChangeListener", routineLoadJob); Map idToTransactionState = Maps.newHashMap(); idToTransactionState.put(1L, transactionState); Deencapsulation.setField(routineLoadJob, "maxErrorNum", 10); Map oldKafkaProgressMap = Maps.newHashMap(); oldKafkaProgressMap.put(1, 0L); KafkaProgress oldkafkaProgress = new KafkaProgress(); - oldkafkaProgress.setPartitionIdToOffset(oldKafkaProgressMap); + Deencapsulation.setField(oldkafkaProgress, "partitionIdToOffset", oldKafkaProgressMap); Deencapsulation.setField(routineLoadJob, "progress", oldkafkaProgress); - routineLoadJob.setState(RoutineLoadJob.JobState.RUNNING); + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.RUNNING); TRLTaskTxnCommitAttachment rlTaskTxnCommitAttachment = new TRLTaskTxnCommitAttachment(); - rlTaskTxnCommitAttachment.setBackendId(1L); - rlTaskTxnCommitAttachment.setTaskSignature(1L); - rlTaskTxnCommitAttachment.setNumOfTotalData(100); - rlTaskTxnCommitAttachment.setNumOfErrorData(1); - rlTaskTxnCommitAttachment.setTaskId("label"); + rlTaskTxnCommitAttachment.setId(new TUniqueId()); + rlTaskTxnCommitAttachment.setLoadedRows(100); + rlTaskTxnCommitAttachment.setFilteredRows(1); rlTaskTxnCommitAttachment.setJobId(Deencapsulation.getField(routineLoadJob, "id")); - rlTaskTxnCommitAttachment.setRoutineLoadType(TRoutineLoadType.KAFKA); + rlTaskTxnCommitAttachment.setLoadSourceType(TLoadSourceType.KAFKA); TKafkaRLTaskProgress tKafkaRLTaskProgress = new TKafkaRLTaskProgress(); Map kafkaProgress = Maps.newHashMap(); kafkaProgress.put(1, 10L); - tKafkaRLTaskProgress.setPartitionIdToOffset(kafkaProgress); + tKafkaRLTaskProgress.setPartitionCmtOffset(kafkaProgress); rlTaskTxnCommitAttachment.setKafkaRLTaskProgress(tKafkaRLTaskProgress); TxnCommitAttachment txnCommitAttachment = new RLTaskTxnCommitAttachment(rlTaskTxnCommitAttachment); - RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.addRoutineLoadJob(routineLoadJob); - - - new Expectations() { - { - catalog.getDb(1L); - result = database; - routineLoadTaskInfo.getId(); - result = "label"; - catalog.getRoutineLoadManager(); - result = routineLoadManager; - database.getTable(anyLong); - result = olapTable; - routineLoadTaskInfo.getJobId(); - result = Deencapsulation.getField(routineLoadJob, "id"); - routineLoadTaskInfo.getPartitions(); - result = Lists.newArrayList().add(1); - } - }; - Deencapsulation.setField(masterTransMgr, "idToTransactionState", idToTransactionState); - masterTransMgr.commitTransaction(1L, 1L, tabletCommitInfoList, txnCommitAttachment); - - Assert.assertEquals(Integer.valueOf(100), Deencapsulation.getField(routineLoadJob, "currentTotalNum")); - Assert.assertEquals(Integer.valueOf(1), Deencapsulation.getField(routineLoadJob, "currentErrorNum")); - Assert.assertEquals(Long.valueOf(10L), ((KafkaProgress) routineLoadJob.getProgress()).getPartitionIdToOffset().get(1)); - Assert.assertEquals(1, routineLoadJob.getNeedScheduleTaskInfoList().size()); - Assert.assertNotEquals("label", routineLoadJob.getNeedScheduleTaskInfoList().get(0)); - Assert.assertEquals(1, routineLoadManager.getNeedScheduleTasksQueue().size()); - Assert.assertNotEquals("label", routineLoadManager.getNeedScheduleTasksQueue().peek().getId()); + routineLoadManager.addRoutineLoadJob(routineLoadJob, "db"); + Deencapsulation.setField(masterTransMgr, "idToTransactionState", idToTransactionState); + masterTransMgr.commitTransaction(1L, 1L, transTablets, txnCommitAttachment); + + Assert.assertEquals(Long.valueOf(101), Deencapsulation.getField(routineLoadJob, "currentTotalRows")); + Assert.assertEquals(Long.valueOf(1), Deencapsulation.getField(routineLoadJob, "currentErrorRows")); + Assert.assertEquals(Long.valueOf(11L), ((KafkaProgress) routineLoadJob.getProgress()).getOffsetByPartition(1)); + // todo(ml): change to assert queue + // Assert.assertEquals(1, routineLoadManager.getNeedScheduleTasksQueue().size()); + // Assert.assertNotEquals("label", routineLoadManager.getNeedScheduleTasksQueue().peek().getId()); } @Test public void testCommitRoutineLoadTransactionWithErrorMax(@Injectable TabletCommitInfo tabletCommitInfo, - @Injectable Database database, - @Injectable KafkaTaskInfo routineLoadTaskInfo, - @Injectable TResourceInfo tResourceInfo, - @Injectable OlapTable olapTable, - @Mocked Catalog catalog, - @Mocked ConnectContext connectContext, - @Mocked KafkaConsumer kafkaConsumer) - throws TransactionException, MetaNotFoundException, DdlException { - List tabletCommitInfoList = new ArrayList<>(); - tabletCommitInfoList.add(tabletCommitInfo); - - KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob("test", 1L, 1L, "host:port", "topic"); + @Mocked EditLog editLog, + @Mocked KafkaConsumer kafkaConsumer) + throws UserException { + + FakeCatalog.setCatalog(masterCatalog); + + TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); + TabletCommitInfo tabletCommitInfo2 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId2); + TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId3); + List transTablets = Lists.newArrayList(); + transTablets.add(tabletCommitInfo1); + transTablets.add(tabletCommitInfo2); + transTablets.add(tabletCommitInfo3); + + KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(1L, "test", "default_cluster", 1L, 1L, "host:port", "topic"); List routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); + Map partitionIdToOffset = Maps.newHashMap(); + partitionIdToOffset.put(1, 0L); + KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "defualt_cluster", partitionIdToOffset); + Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L); routineLoadTaskInfoList.add(routineLoadTaskInfo); - TransactionState transactionState = new TransactionState(1L, 1L, "label", 1L, LoadJobSourceType.ROUTINE_LOAD_TASK, "be1"); + TransactionState transactionState = new TransactionState(1L, 1L, "label", 1L, + LoadJobSourceType.ROUTINE_LOAD_TASK, "be1", routineLoadJob.getId(), + Config.stream_load_default_timeout_second); transactionState.setTransactionStatus(TransactionStatus.PREPARE); - transactionState.setTxnStateChangeListener(routineLoadJob); + masterTransMgr.getCallbackFactory().addCallback(routineLoadJob); Map idToTransactionState = Maps.newHashMap(); idToTransactionState.put(1L, transactionState); Deencapsulation.setField(routineLoadJob, "maxErrorNum", 10); Map oldKafkaProgressMap = Maps.newHashMap(); oldKafkaProgressMap.put(1, 0L); KafkaProgress oldkafkaProgress = new KafkaProgress(); - oldkafkaProgress.setPartitionIdToOffset(oldKafkaProgressMap); + Deencapsulation.setField(oldkafkaProgress, "partitionIdToOffset", oldKafkaProgressMap); Deencapsulation.setField(routineLoadJob, "progress", oldkafkaProgress); - routineLoadJob.setState(RoutineLoadJob.JobState.RUNNING); + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.RUNNING); TRLTaskTxnCommitAttachment rlTaskTxnCommitAttachment = new TRLTaskTxnCommitAttachment(); - rlTaskTxnCommitAttachment.setBackendId(1L); - rlTaskTxnCommitAttachment.setTaskSignature(1L); - rlTaskTxnCommitAttachment.setNumOfTotalData(100); - rlTaskTxnCommitAttachment.setNumOfErrorData(11); - rlTaskTxnCommitAttachment.setTaskId("label"); + rlTaskTxnCommitAttachment.setId(new TUniqueId()); + rlTaskTxnCommitAttachment.setLoadedRows(100); + rlTaskTxnCommitAttachment.setFilteredRows(11); rlTaskTxnCommitAttachment.setJobId(Deencapsulation.getField(routineLoadJob, "id")); - rlTaskTxnCommitAttachment.setRoutineLoadType(TRoutineLoadType.KAFKA); + rlTaskTxnCommitAttachment.setLoadSourceType(TLoadSourceType.KAFKA); TKafkaRLTaskProgress tKafkaRLTaskProgress = new TKafkaRLTaskProgress(); Map kafkaProgress = Maps.newHashMap(); kafkaProgress.put(1, 10L); - tKafkaRLTaskProgress.setPartitionIdToOffset(kafkaProgress); + tKafkaRLTaskProgress.setPartitionCmtOffset(kafkaProgress); rlTaskTxnCommitAttachment.setKafkaRLTaskProgress(tKafkaRLTaskProgress); TxnCommitAttachment txnCommitAttachment = new RLTaskTxnCommitAttachment(rlTaskTxnCommitAttachment); - RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.addRoutineLoadJob(routineLoadJob); - - - new Expectations() { - { - catalog.getDb(1L); - result = database; - routineLoadTaskInfo.getId(); - result = "label"; - database.getTable(anyLong); - result = olapTable; - } - }; - Deencapsulation.setField(masterTransMgr, "idToTransactionState", idToTransactionState); - masterTransMgr.commitTransaction(1L, 1L, tabletCommitInfoList, txnCommitAttachment); + routineLoadManager.addRoutineLoadJob(routineLoadJob, "db"); - Assert.assertEquals(Integer.valueOf(0), Deencapsulation.getField(routineLoadJob, "currentTotalNum")); - Assert.assertEquals(Integer.valueOf(0), Deencapsulation.getField(routineLoadJob, "currentErrorNum")); - Assert.assertEquals(Long.valueOf(10L), ((KafkaProgress) routineLoadJob.getProgress()).getPartitionIdToOffset().get(1)); - Assert.assertEquals(0, routineLoadJob.getNeedScheduleTaskInfoList().size()); - Assert.assertEquals(0, routineLoadManager.getNeedScheduleTasksQueue().size()); + Deencapsulation.setField(masterTransMgr, "idToTransactionState", idToTransactionState); + masterTransMgr.commitTransaction(1L, 1L, transTablets, txnCommitAttachment); + + Assert.assertEquals(Long.valueOf(0), Deencapsulation.getField(routineLoadJob, "currentTotalRows")); + Assert.assertEquals(Long.valueOf(0), Deencapsulation.getField(routineLoadJob, "currentErrorRows")); + Assert.assertEquals(Long.valueOf(11L), + ((KafkaProgress) routineLoadJob.getProgress()).getOffsetByPartition(1)); + // todo(ml): change to assert queue + // Assert.assertEquals(0, routineLoadManager.getNeedScheduleTasksQueue().size()); Assert.assertEquals(RoutineLoadJob.JobState.PAUSED, routineLoadJob.getState()); } - - public void testFinishTransaction() throws MetaNotFoundException, TransactionException, - IllegalTransactionParameterException, LabelAlreadyUsedException, - AnalysisException, BeginTransactionException { + public void testFinishTransaction() throws UserException { long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); // commit a transaction TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); @@ -493,10 +463,7 @@ public void testFinishTransaction() throws MetaNotFoundException, TransactionExc } @Test - public void testFinishTransactionWithOneFailed() throws MetaNotFoundException, - TransactionException, - IllegalTransactionParameterException, LabelAlreadyUsedException, - AnalysisException, BeginTransactionException { + public void testFinishTransactionWithOneFailed() throws UserException { TransactionState transactionState = null; Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); @@ -505,7 +472,7 @@ public void testFinishTransactionWithOneFailed() throws MetaNotFoundException, long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); // commit a transaction with 1,2 success TabletCommitInfo tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); @@ -559,7 +526,7 @@ public void testFinishTransactionWithOneFailed() throws MetaNotFoundException, long transactionId2 = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable2, transactionSource, - LoadJobSourceType.FRONTEND); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); tabletCommitInfo1 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId1); TabletCommitInfo tabletCommitInfo3 = new TabletCommitInfo(CatalogTestUtil.testTabletId1, CatalogTestUtil.testBackendId3); @@ -567,7 +534,7 @@ public void testFinishTransactionWithOneFailed() throws MetaNotFoundException, transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo3); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, transactionId2, transTablets); - transactionState = fakeEditLog.getTransaction(transactionId2); + transactionState = masterTransMgr.getTransactionState(transactionId2); // check status is prepare, because the commit failed assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); @@ -627,8 +594,8 @@ public void testDeleteTransaction() throws LabelAlreadyUsedException, long transactionId = masterTransMgr.beginTransaction(CatalogTestUtil.testDbId1, CatalogTestUtil.testTxnLable1, transactionSource, - LoadJobSourceType.FRONTEND); - TransactionState transactionState = fakeEditLog.getTransaction(transactionId); + LoadJobSourceType.FRONTEND, Config.stream_load_default_timeout_second); + TransactionState transactionState = masterTransMgr.getTransactionState(transactionId); assertNotNull(transactionState); assertEquals(transactionId, transactionState.getTransactionId()); assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); diff --git a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java index fe3a89bceee696..09072818133e15 100644 --- a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java +++ b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java @@ -269,6 +269,7 @@ public BrokerFileSystem getFileSystem(String path, Map propertie "invalid " + dfsHaNameNodesKey + " configuration"); } else { for (String nameNode : nameNodes) { + nameNode = nameNode.trim(); String nameNodeRpcAddress = DFS_HA_NAMENODE_RPC_ADDRESS_PREFIX + dfsNameServices + "." + nameNode; if (!properties.containsKey(nameNodeRpcAddress)) { diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index a608b98559e73f..ad298f5eddc08c 100755 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -469,9 +469,15 @@ [['strleft'], 'VARCHAR', ['VARCHAR', 'INT'], '_ZN5doris15StringFunctions4leftEPN9doris_udf' '15FunctionContextERKNS1_9StringValERKNS1_6IntValE'], + [['left'], 'VARCHAR', ['VARCHAR', 'INT'], + '_ZN5doris15StringFunctions4leftEPN9doris_udf' + '15FunctionContextERKNS1_9StringValERKNS1_6IntValE'], [['strright'], 'VARCHAR', ['VARCHAR', 'INT'], '_ZN5doris15StringFunctions5rightEPN9doris_udf' '15FunctionContextERKNS1_9StringValERKNS1_6IntValE'], + [['right'], 'VARCHAR', ['VARCHAR', 'INT'], + '_ZN5doris15StringFunctions5rightEPN9doris_udf' + '15FunctionContextERKNS1_9StringValERKNS1_6IntValE'], [['space'], 'VARCHAR', ['INT'], '_ZN5doris15StringFunctions5spaceEPN9doris_udf15FunctionContextERKNS1_6IntValE'], [['repeat'], 'VARCHAR', ['VARCHAR', 'INT'], diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index b27d10ce337a8e..79443dddc0aeb9 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -192,7 +192,7 @@ struct TRecoverTabletReq { 3: optional Types.TVersion version 4: optional Types.TVersionHash version_hash } - + struct TAgentTaskRequest { 1: required TAgentServiceVersion protocol_version 2: required Types.TTaskType task_type diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index 798607c20c7ba0..5a197f9d9ae900 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -63,6 +63,28 @@ struct TTabletStatResult { 1: required map tablets_stats } +struct TKafkaLoadInfo { + 1: required string brokers; + 2: required string topic; + 3: required map partition_begin_offset; +} + +struct TRoutineLoadTask { + 1: required Types.TLoadSourceType type + 2: required i64 job_id + 3: required Types.TUniqueId id + 4: required i64 txn_id + 5: required i64 auth_code + 6: optional string db + 7: optional string tbl + 8: optional string label + 9: optional i64 max_interval_s + 10: optional i64 max_batch_rows + 11: optional i64 max_batch_size + 12: optional TKafkaLoadInfo kafka_load_info + 13: optional PaloInternalService.TExecPlanFragmentParams params +} + service BackendService { // Called by coord to start asynchronous execution of plan fragment in backend. // Returns as soon as all incoming data streams have been set up. @@ -102,7 +124,7 @@ service BackendService { Status.TStatus register_pull_load_task(1: Types.TUniqueId id, 2: i32 num_senders) // Call by task coordinator to unregister this task. - // This task may be failed because load task have been finished or this task + // This task may be failed because load task have been finished or this task // has been canceled by coordinator. Status.TStatus deregister_pull_load_task(1: Types.TUniqueId id) @@ -119,4 +141,6 @@ service BackendService { Status.TStatus erase_export_task(1:Types.TUniqueId task_id); TTabletStatResult get_tablet_stat(); + + Status.TStatus submit_routine_load_task(1:list tasks); } diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index eeb846464a0c74..644a904ac9c46a 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -458,6 +458,8 @@ struct TLoadTxnBeginRequest { 6: optional string user_ip 7: required string label 8: optional i64 timestamp + 9: optional i64 auth_code + 10: optional i64 timeout } struct TLoadTxnBeginResult { @@ -493,6 +495,7 @@ struct TStreamLoadPutRequest { 14: optional string columnSeparator 15: optional string partitions + 16: optional i64 auth_code } struct TStreamLoadPutResult { @@ -501,31 +504,26 @@ struct TStreamLoadPutResult { 2: optional PaloInternalService.TExecPlanFragmentParams params } -enum TRoutineLoadType { - KAFKA = 1 -} - struct TKafkaRLTaskProgress { - 1: required map partitionIdToOffset -} - -enum TTxnSourceType { - ROUTINE_LOAD_TASK = 1 + 1: required map partitionCmtOffset } struct TRLTaskTxnCommitAttachment { - 1: required TRoutineLoadType routineLoadType - 2: required i64 backendId - 3: required i64 taskSignature - 4: required i32 numOfErrorData - 5: required i32 numOfTotalData - 6: required string taskId - 7: required string jobId - 8: optional TKafkaRLTaskProgress kafkaRLTaskProgress + 1: required Types.TLoadSourceType loadSourceType + 2: required Types.TUniqueId id + 3: required i64 jobId + 4: optional i64 loadedRows + 5: optional i64 filteredRows + 6: optional i64 unselectedRows + 7: optional i64 receivedBytes + 8: optional i64 loadedBytes + 9: optional i64 loadCostMs + 10: optional TKafkaRLTaskProgress kafkaRLTaskProgress + 11: optional string errorLogUrl } struct TTxnCommitAttachment { - 1: required TTxnSourceType txnSourceType + 1: required Types.TLoadType loadType 2: optional TRLTaskTxnCommitAttachment rlTaskTxnCommitAttachment } @@ -539,7 +537,8 @@ struct TLoadTxnCommitRequest { 7: required i64 txnId 8: required bool sync 9: optional list commitInfos - 10: optional TTxnCommitAttachment txnCommitAttachment + 10: optional i64 auth_code + 11: optional TTxnCommitAttachment txnCommitAttachment } struct TLoadTxnCommitResult { @@ -555,6 +554,8 @@ struct TLoadTxnRollbackRequest { 6: optional string user_ip 7: required i64 txnId 8: optional string reason + 9: optional i64 auth_code + 10: optional TTxnCommitAttachment txnCommitAttachment } struct TLoadTxnRollbackResult { diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 69476511499d69..67fbef8f807116 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -43,7 +43,8 @@ enum TPlanNodeType { BROKER_SCAN_NODE, EMPTY_SET_NODE, UNION_NODE, - ES_SCAN_NODE + ES_SCAN_NODE, + ES_HTTP_SCAN_NODE } // phases of an execution node diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 81374e685e16ee..519c460dd641dc 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -354,3 +354,12 @@ struct TTabletCommitInfo { 2: required i64 backendId } +enum TLoadType { + MANUL_LOAD, + ROUTINE_LOAD, +} + +enum TLoadSourceType { + RAW, + KAFKA, +} diff --git a/run-ut.sh b/run-ut.sh index 96ed3f544e43c4..bd6a6983a3db4a 100755 --- a/run-ut.sh +++ b/run-ut.sh @@ -163,6 +163,10 @@ ${DORIS_TEST_BINARY_DIR}/exec/plain_text_line_reader_lzop_test ${DORIS_TEST_BINARY_DIR}/exec/broker_scanner_test ${DORIS_TEST_BINARY_DIR}/exec/broker_scan_node_test ${DORIS_TEST_BINARY_DIR}/exec/es_scan_node_test +${DORIS_TEST_BINARY_DIR}/exec/es_http_scan_node_test +${DORIS_TEST_BINARY_DIR}/exec/es_predicate_test +${DORIS_TEST_BINARY_DIR}/exec/es_scan_reader_test +${DORIS_TEST_BINARY_DIR}/exec/es_query_builder_test ${DORIS_TEST_BINARY_DIR}/exec/olap_table_info_test ${DORIS_TEST_BINARY_DIR}/exec/olap_table_sink_test @@ -210,6 +214,10 @@ ${DORIS_TEST_BINARY_DIR}/olap/olap_header_manager_test ${DORIS_TEST_BINARY_DIR}/olap/olap_meta_test ${DORIS_TEST_BINARY_DIR}/olap/delta_writer_test +# Running routine load test +${DORIS_TEST_BINARY_DIR}/runtime/kafka_consumer_pipe_test +${DORIS_TEST_BINARY_DIR}/runtime/routine_load_task_executor_test + ## Running agent unittest # Prepare agent testdata if [ -d ${DORIS_TEST_BINARY_DIR}/agent/test_data ]; then diff --git a/samples/stream_load/java/DorisStreamLoad.java b/samples/stream_load/java/DorisStreamLoad.java new file mode 100644 index 00000000000000..67974bff7ba3c0 --- /dev/null +++ b/samples/stream_load/java/DorisStreamLoad.java @@ -0,0 +1,141 @@ +// 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. + +import org.apache.commons.codec.binary.Base64; +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +/** + * This class is a java demo for doris stream load + * + * The pom.xml dependency: + * + * + * org.apache.httpcomponents + * httpclient + * 4.5.3 + * + * + * How to use: + * + * 1 create a table in doris with any mysql client + * + * CREATE TABLE `stream_test` ( + * `id` bigint(20) COMMENT "", + * `id2` bigint(20) COMMENT "" + * ) ENGINE=OLAP + * DUPLICATE KEY(`id`) + * DISTRIBUTED BY HASH(`id`) BUCKETS 20; + * + * + * 2 change the Doris cluster, db, user config in this class + * + * 3 run this class, you should see the following output: + * + * { + * "Status": "Success", + * "Message": "OK", + * "NumberLoadedRows": 10, + * "NumberFilteredRows": 0, + * "LoadBytes": 50, + * "LoadTimeMs": 151, + * "Label": "39c25a5c-7000-496e-a98e-348a264c81de" + * } + * + */ +public class DorisStreamLoad { + private final static String DORIS_HOST = "xxx.com"; + private final static String DORIS_DB = "test"; + private final static String DORIS_TABLE = "stream_test"; + private final static String DORIS_USER = "root"; + private final static String DORIS_PASSWORD = "xxx"; + private final static int DORIS_HTTP_PORT = 8410; + + private void sendData(String content) throws Exception { + final String loadUrl = String.format("http://%s:%s/api/%s/%s/_stream_load", + DORIS_HOST, + DORIS_HTTP_PORT, + DORIS_DB, + DORIS_TABLE); + + final HttpClientBuilder httpClientBuilder = HttpClients + .custom() + .setRedirectStrategy(new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }); + + try (CloseableHttpClient client = httpClientBuilder.build()) { + HttpPut put = new HttpPut(loadUrl); + StringEntity entity = new StringEntity(content); + put.setHeader(HttpHeaders.EXPECT, "100-continue"); + put.setHeader(HttpHeaders.AUTHORIZATION, basicAuthHeader(DORIS_USER, DORIS_PASSWORD)); + put.setEntity(entity); + + try (CloseableHttpResponse response = client.execute(put)) { + String loadResult = ""; + if (response.getEntity() != null) { + loadResult = EntityUtils.toString(response.getEntity()); + } + final int statusCode = response.getStatusLine().getStatusCode(); + if (statusCode != 200) { + throw new IOException( + String.format("Stream load failed, statusCode=%s load result=%s", statusCode, loadResult)); + } + + System.out.println(loadResult); + } + } + } + + private String basicAuthHeader(String username, String password) { + final String tobeEncode = username + ":" + password; + byte[] encoded = Base64.encodeBase64(tobeEncode.getBytes(StandardCharsets.UTF_8)); + return "Basic " + new String(encoded); + } + + public static void main(String[] args) throws Exception { + int id1 = 1; + int id2 = 10; + int rowNumber = 10; + String oneRow = id1 + "\t" + id2 + "\n"; + + StringBuilder stringBuilder = new StringBuilder(); + for (int i = 0; i < rowNumber; i++) { + stringBuilder.append(oneRow); + } + + //in doris 0.9 version, you need to comment this line + //refer to https://github.com/apache/incubator-doris/issues/783 + stringBuilder.deleteCharAt(stringBuilder.length() - 1); + + String loadData = stringBuilder.toString(); + DorisStreamLoad dorisStreamLoad = new DorisStreamLoad(); + dorisStreamLoad.sendData(loadData); + } +} diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh index 1068cc1b57068f..b36d820ff8bec8 100755 --- a/thirdparty/build-thirdparty.sh +++ b/thirdparty/build-thirdparty.sh @@ -319,12 +319,16 @@ build_snappy() { check_if_source_exist $SNAPPY_SOURCE cd $TP_SOURCE_DIR/$SNAPPY_SOURCE - CPPFLAGS="-I${TP_INCLUDE_DIR}" \ - LDFLAGS="-L${TP_LIB_DIR}" \ - CFLAGS="-fPIC" \ - ./configure --prefix=$TP_INSTALL_DIR --disable-shared --enable-static \ - --includedir=$TP_INCLUDE_DIR/snappy + mkdir build -p && cd build + rm -rf CMakeCache.txt CMakeFiles/ + $CMAKE_CMD -DCMAKE_INSTALL_PREFIX=$TP_INSTALL_DIR \ + -DCMAKE_POSITION_INDEPENDENT_CODE=On \ + -DCMAKE_INSTALL_INCLUDEDIR=$TP_INCLUDE_DIR/snappy \ + -DSNAPPY_BUILD_TESTS=0 ../ make -j$PARALLEL && make install + if [ -f $TP_INSTALL_DIR/lib64/libsnappy.a ]; then + mkdir -p $TP_INSTALL_DIR/lib && ln -s $TP_INSTALL_DIR/lib64/libsnappy.a $TP_INSTALL_DIR/lib/libsnappy.a + fi } # gperftools @@ -339,6 +343,9 @@ build_gperftools() { LDFLAGS="-L${TP_LIB_DIR}" \ LD_LIBRARY_PATH="${TP_LIB_DIR}" \ CFLAGS="-fPIC" \ + LDFLAGS="-L${TP_LIB_DIR}" \ + LD_LIBRARY_PATH="${TP_LIB_DIR}" \ + CFLAGS="-fPIC" \ ./configure --prefix=$TP_INSTALL_DIR/gperftools --disable-shared --enable-static --disable-libunwind --with-pic --enable-frame-pointers make -j$PARALLEL && make install } @@ -456,7 +463,6 @@ build_mysql() { #leveldb build_leveldb() { check_if_source_exist $LEVELDB_SOURCE - cd $TP_SOURCE_DIR/$LEVELDB_SOURCE CXXFLAGS="-fPIC" make -j$PARALLEL cp out-static/libleveldb.a ../../installed/lib/libleveldb.a @@ -507,7 +513,7 @@ build_librdkafka() { CPPFLAGS="-I${TP_INCLUDE_DIR}" \ LDFLAGS="-L${TP_LIB_DIR}" CFLAGS="-fPIC" \ - ./configure --prefix=$TP_INSTALL_DIR --enable-static + ./configure --prefix=$TP_INSTALL_DIR --enable-static --disable-ssl --disable-sasl make -j$PARALLEL && make install } @@ -521,8 +527,8 @@ build_openssl build_boost # must before thrift build_protobuf build_gflags -build_glog build_gtest +build_glog build_rapidjson build_snappy build_gperftools diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh index 2454ca8a4e2a25..003d96412f2512 100644 --- a/thirdparty/vars.sh +++ b/thirdparty/vars.sh @@ -115,10 +115,10 @@ GTEST_SOURCE=googletest-release-1.8.0 GTEST_MD5SUM="16877098823401d1bf2ed7891d7dce36" # snappy -SNAPPY_DOWNLOAD="https://github.com/google/snappy/releases/download/1.1.4/snappy-1.1.4.tar.gz" -SNAPPY_NAME=snappy-1.1.4.tar.gz -SNAPPY_SOURCE=snappy-1.1.4 -SNAPPY_MD5SUM="c328993b68afe3e5bd87c8ea9bdeb028" +SNAPPY_DOWNLOAD="https://github.com/google/snappy/archive/1.1.7.tar.gz" +SNAPPY_NAME=snappy-1.1.7.tar.gz +SNAPPY_SOURCE=snappy-1.1.7 +SNAPPY_MD5SUM="ee9086291c9ae8deb4dac5e0b85bf54a" # gperftools GPERFTOOLS_DOWNLOAD="https://github.com/gperftools/gperftools/archive/gperftools-2.7.tar.gz"