From 52f2df532a6d397420ae24c395d28475a306f5c7 Mon Sep 17 00:00:00 2001 From: morningman Date: Wed, 13 Mar 2019 09:27:34 +0800 Subject: [PATCH 1/2] Add persist operations for routine load job --- be/src/runtime/routine_load/data_consumer.cpp | 16 +- .../routine_load_task_executor.cpp | 4 + .../runtime/stream_load/stream_load_context.h | 21 +- fe/src/main/cup/sql_parser.cup | 13 +- .../doris/analysis/CreateRoutineLoadStmt.java | 247 ++++++----- .../doris/analysis/LoadColumnsInfo.java | 26 +- .../doris/analysis/ShowVariablesStmt.java | 10 +- .../org/apache/doris/catalog/Catalog.java | 68 +-- .../org/apache/doris/common/FeConstants.java | 2 +- .../apache/doris/common/FeMetaVersion.java | 4 +- .../apache/doris/common/LoadException.java | 5 +- .../apache/doris/journal/JournalEntity.java | 12 + .../apache/doris/load/RoutineLoadDesc.java | 17 +- .../doris/load/TxnStateChangeListener.java | 53 --- .../doris/load/routineload/KafkaProgress.java | 8 +- .../load/routineload/KafkaRoutineLoadJob.java | 137 +++--- .../doris/load/routineload/KafkaTaskInfo.java | 7 +- .../RLTaskTxnCommitAttachment.java | 4 - .../load/routineload/RoutineLoadJob.java | 406 ++++++++++++------ .../load/routineload/RoutineLoadManager.java | 144 ++++--- .../routineload/RoutineLoadScheduler.java | 16 +- .../load/routineload/RoutineLoadTaskInfo.java | 2 +- .../routineload/RoutineLoadTaskScheduler.java | 14 +- .../org/apache/doris/persist/EditLog.java | 18 + .../apache/doris/persist/OperationType.java | 4 + .../apache/doris/planner/OlapScanNode.java | 3 +- .../java/org/apache/doris/qe/DdlExecutor.java | 4 +- .../org/apache/doris/qe/ShowExecutor.java | 2 +- .../org/apache/doris/qe/StmtExecutor.java | 16 +- .../doris/service/FrontendServiceImpl.java | 3 +- .../doris/task/KafkaRoutineLoadTask.java | 40 -- .../apache/doris/task/RoutineLoadTask.java | 53 --- .../org/apache/doris/task/StreamLoadTask.java | 21 +- .../transaction/GlobalTransactionMgr.java | 41 +- .../doris/transaction/TransactionState.java | 191 ++++---- .../analysis/CreateRoutineLoadStmtTest.java | 15 +- .../routineload/KafkaRoutineLoadJobTest.java | 41 +- .../routineload/RoutineLoadManagerTest.java | 49 ++- .../routineload/RoutineLoadSchedulerTest.java | 26 +- .../RoutineLoadTaskSchedulerTest.java | 33 -- .../org/apache/doris/qe/StmtExecutorTest.java | 8 +- .../transaction/GlobalTransactionMgrTest.java | 8 +- gensrc/thrift/BackendService.thrift | 12 +- 43 files changed, 969 insertions(+), 855 deletions(-) delete mode 100644 fe/src/main/java/org/apache/doris/load/TxnStateChangeListener.java delete mode 100644 fe/src/main/java/org/apache/doris/task/KafkaRoutineLoadTask.java delete mode 100644 fe/src/main/java/org/apache/doris/task/RoutineLoadTask.java diff --git a/be/src/runtime/routine_load/data_consumer.cpp b/be/src/runtime/routine_load/data_consumer.cpp index e52d3332d4af36..f96b24cc942aa7 100644 --- a/be/src/runtime/routine_load/data_consumer.cpp +++ b/be/src/runtime/routine_load/data_consumer.cpp @@ -133,9 +133,9 @@ Status KafkaDataConsumer::start(StreamLoadContext* ctx) { } } - int64_t left_time = ctx->kafka_info->max_interval_s; - int64_t left_rows = ctx->kafka_info->max_batch_rows; - int64_t left_bytes = ctx->kafka_info->max_batch_size; + int64_t left_time = ctx->max_interval_s; + int64_t left_rows = ctx->max_batch_rows; + int64_t left_bytes = ctx->max_batch_size; std::shared_ptr kakfa_pipe = std::static_pointer_cast(ctx->body_sink); @@ -145,7 +145,7 @@ Status KafkaDataConsumer::start(StreamLoadContext* ctx) { << ", batch size: " << left_bytes << ". " << ctx->brief(); -// copy one + // copy one std::map cmt_offset = ctx->kafka_info->cmt_offset; MonotonicStopWatch watch; watch.start(); @@ -169,15 +169,15 @@ Status KafkaDataConsumer::start(StreamLoadContext* ctx) { << ", left rows=" << left_rows << ", left bytes=" << left_bytes; - if (left_bytes == ctx->kafka_info->max_batch_size) { + if (left_bytes == ctx->max_batch_size) { // nothing to be consumed, cancel it // we do not allow finishing stream load pipe without data kakfa_pipe->cancel(); _cancelled = true; return Status::CANCELLED; } else { - DCHECK(left_bytes < ctx->kafka_info->max_batch_size); - DCHECK(left_rows < ctx->kafka_info->max_batch_rows); + DCHECK(left_bytes < ctx->max_batch_size); + DCHECK(left_rows < ctx->max_batch_rows); kakfa_pipe->finish(); ctx->kafka_info->cmt_offset = std::move(cmt_offset); _finished = true; @@ -223,7 +223,7 @@ Status KafkaDataConsumer::start(StreamLoadContext* ctx) { return st; } - left_time = ctx->kafka_info->max_interval_s - watch.elapsed_time() / 1000 / 1000 / 1000; + left_time = ctx->max_interval_s - watch.elapsed_time() / 1000 / 1000 / 1000; } return Status::OK; diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp index 4a5a040512a7ee..b8dd206ccaa64b 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.cpp +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -53,6 +53,10 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { 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; diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index cc4aa3ae89b5dd..b11bd6412dbbf6 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -48,9 +48,6 @@ class KafkaLoadInfo { for (auto& p : t_info.partition_begin_offset) { cmt_offset[p.first] = p.second -1; } - if (t_info.__isset.max_interval_s) { max_interval_s = t_info.max_interval_s; } - if (t_info.__isset.max_batch_rows) { max_batch_rows = t_info.max_batch_rows; } - if (t_info.__isset.max_batch_size) { max_batch_size = t_info.max_batch_size; } } public: @@ -121,6 +118,24 @@ class StreamLoadContext { 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 = 1024; + int64_t max_batch_size = 100 * 1024 * 1024; // 100MB + + // 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 + + // 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; diff --git a/fe/src/main/cup/sql_parser.cup b/fe/src/main/cup/sql_parser.cup index b54ae1b2e3ad2b..6febb436eafc76 100644 --- a/fe/src/main/cup/sql_parser.cup +++ b/fe/src/main/cup/sql_parser.cup @@ -386,7 +386,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 +448,7 @@ query ::= ; import_columns_stmt ::= - KW_COLUMNS import_column_descs:columns + KW_COLUMNS LPAREN import_column_descs:columns RPAREN {: RESULT = new ImportColumnsStmt(columns); :} @@ -1177,7 +1176,7 @@ load_property ::= {: RESULT = colSep; :} - | load_columns_info:columnsInfo + | import_columns_stmt:columnsInfo {: RESULT = columnsInfo; :} @@ -1191,14 +1190,6 @@ 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 {: 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 6826f3aaaac11d..3e920ebfd503a7 100644 --- a/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java @@ -17,16 +17,18 @@ 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.FeNameFormat; +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.load.routineload.LoadDataSourceType; + +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; @@ -57,16 +59,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,6 +78,10 @@ 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_SECOND = "max_batch_interval"; + public static final String MAX_BATCH_ROWS = "max_batch_rows"; + public static final String MAX_BATCH_SIZE = "max_batch_size"; // kafka type properties public static final String KAFKA_BROKER_LIST_PROPERTY = "kafka_broker_list"; @@ -86,11 +92,13 @@ public class CreateRoutineLoadStmt extends DdlStmt { 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_SECOND) + .add(MAX_BATCH_ROWS) + .add(MAX_BATCH_SIZE) .build(); private static final ImmutableSet KAFKA_PROPERTIES_SET = new ImmutableSet.Builder() @@ -103,29 +111,34 @@ public class CreateRoutineLoadStmt extends DdlStmt { private final String name; private final TableName dbTableName; 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 RoutineLoadDesc routineLoadDesc; - private int desiredConcurrentNum; - private int maxErrorNum; + private int desiredConcurrentNum = 1; + private int maxErrorNum = -1; + private int maxBatchIntervalS = -1; + private int maxBatchRows = -1; + private int maxBatchSizeBytes = -1; + + // kafka related properties private String kafkaBrokerList; private String kafkaTopic; - private List kafkaPartitions; - private List kafkaOffsets; + // pair + private List> kafkaPartitionOffsets = Lists.newArrayList(); public CreateRoutineLoadStmt(String name, TableName dbTableName, List loadPropertyList, - Map properties, - String typeName, Map customProperties) { + Map jobProperties, + String typeName, Map dataSourceProperties) { this.name = name; this.dbTableName = dbTableName; this.loadPropertyList = loadPropertyList; - this.properties = properties; + this.jobProperties = jobProperties == null ? Maps.newHashMap() : jobProperties; this.typeName = typeName.toUpperCase(); - this.customProperties = customProperties; + this.dataSourceProperties = dataSourceProperties; } public String getName() { @@ -136,19 +149,10 @@ public TableName getDBTableName() { return dbTableName; } - public Map getProperties() { - return properties; - } - public String getTypeName() { return typeName; } - public Map getCustomProperties() { - return customProperties; - } - - // nullable public RoutineLoadDesc getRoutineLoadDesc() { return routineLoadDesc; } @@ -161,6 +165,18 @@ public int getMaxErrorNum() { return maxErrorNum; } + public int getMaxBatchIntervalS() { + return maxBatchIntervalS; + } + + public int getMaxBatchRows() { + return maxBatchRows; + } + + public int getMaxBatchSize() { + return maxBatchSizeBytes; + } + public String getKafkaBrokerList() { return kafkaBrokerList; } @@ -169,50 +185,32 @@ public String getKafkaTopic() { return kafkaTopic; } - public List getKafkaPartitions() { - return kafkaPartitions; - } - - public List getKafkaOffsets(){ - return kafkaOffsets; + 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 name FeNameFormat.checkCommonName(NAME_TYPE, name); // check dbName and tableName - checkDBTableName(); dbTableName.analyze(analyzer); // check load properties include column separator etc. checkLoadProperties(analyzer); // check routine load properties include desired concurrent number etc. - checkRoutineLoadProperties(); - // check custom properties - checkCustomProperties(); - } - - 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"); - } + checkJobProperties(); + // check data load source properties + checkLoadSourceProperties(); } - 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; + ImportColumnsStmt importColumnsStmt = null; + ImportWhereStmt importWhereStmt = null; PartitionNames partitionNames = null; for (ParseNode parseNode : loadPropertyList) { if (parseNode instanceof ColumnSeparator) { @@ -221,65 +219,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.analyze(null); } } - routineLoadDesc = new RoutineLoadDesc(columnSeparator, columnsInfo, wherePredicate, + routineLoadDesc = new RoutineLoadDesc(columnSeparator, importColumnsStmt, importWhereStmt, partitionNames.getPartitionNames()); } - private void checkRoutineLoadProperties() throws AnalysisException { - if (properties != null) { - Optional optional = properties.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"); - } - } + 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 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 = getIntegetPropertyOrDefault(DESIRED_CONCURRENT_NUMBER_PROPERTY, + "must be greater then 0", desiredConcurrentNum); + maxErrorNum = getIntegetPropertyOrDefault(MAX_ERROR_NUMBER_PROPERTY, + "must be greater then or equal to 0", maxErrorNum); + maxBatchIntervalS = getIntegetPropertyOrDefault(MAX_BATCH_INTERVAL_SECOND, + "must be greater then 0", maxBatchIntervalS); + maxBatchRows = getIntegetPropertyOrDefault(MAX_BATCH_ROWS, "must be greater then 0", maxBatchRows); + maxBatchSizeBytes = getIntegetPropertyOrDefault(MAX_BATCH_SIZE, "must be greater then 0", maxBatchSizeBytes); + } + private int getIntegetPropertyOrDefault(String propName, String hintMsg, int defaultVal) throws AnalysisException { + final String propVal = jobProperties.get(propName); + if (propVal != null) { + int intVal = getIntegerValueFromString(propVal, propName); + if (intVal <= 0) { + throw new AnalysisException(propName + " " + hintMsg); } + return intVal; } + return defaultVal; } - private void checkCustomProperties() throws AnalysisException { + private void checkLoadSourceProperties() throws AnalysisException { LoadDataSourceType type; try { type = LoadDataSourceType.valueOf(typeName); @@ -288,23 +283,24 @@ 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 - kafkaBrokerList = customProperties.get(KAFKA_BROKER_LIST_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 required property"); + throw new AnalysisException(KAFKA_BROKER_LIST_PROPERTY + " is a required property"); } String[] kafkaBrokerList = this.kafkaBrokerList.split(","); for (String broker : kafkaBrokerList) { @@ -313,42 +309,52 @@ private void checkKafkaCustomProperties() throws AnalysisException { + " 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), 0L)); } catch (AnalysisException e) { throw new AnalysisException(KAFKA_PARTITIONS_PROPERTY + " must be a number string with comma-separated"); } } } - // check offsets - // Todo(ml) - final String kafkaOffsetsString = customProperties.get(KAFKA_OFFSETS_PROPERTY); + + // check offset + final String kafkaOffsetsString = dataSourceProperties.get(KAFKA_OFFSETS_PROPERTY); if (kafkaOffsetsString != null) { - kafkaOffsets = new ArrayList<>(); - String[] kafkaOffsetsStringList = customProperties.get(KAFKA_OFFSETS_PROPERTY).split(","); - for (String s : kafkaOffsetsStringList) { - kafkaOffsets.add(Long.valueOf(s)); + 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++) { + kafkaPartitionOffsets.get(i).second = getLongValueFromString(kafkaOffsetsStringList[i], + KAFKA_OFFSETS_PROPERTY); } } } 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; @@ -359,4 +365,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/LoadColumnsInfo.java b/fe/src/main/java/org/apache/doris/analysis/LoadColumnsInfo.java index 3851f261ece0eb..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; @@ -91,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/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/catalog/Catalog.java b/fe/src/main/java/org/apache/doris/catalog/Catalog.java index 9f7af9a013f00c..9124fcc8b25b58 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/src/main/java/org/apache/doris/catalog/Catalog.java @@ -100,7 +100,6 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MarkedCountDownLatch; -import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; @@ -1294,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); @@ -1313,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); @@ -1397,24 +1392,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 { @@ -1700,10 +1698,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 { @@ -1725,6 +1726,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 @@ -1769,6 +1777,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(); @@ -2000,6 +2009,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) { 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/journal/JournalEntity.java b/fe/src/main/java/org/apache/doris/journal/JournalEntity.java index 64f9f1de6776c7..59e6a79ce823e4 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,16 @@ 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: { + 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/RoutineLoadDesc.java b/fe/src/main/java/org/apache/doris/load/RoutineLoadDesc.java index 4faaca6d2a736d..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,24 +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; @@ -43,11 +42,11 @@ public ColumnSeparator getColumnSeparator() { return columnSeparator; } - public LoadColumnsInfo getColumnsInfo() { + public ImportColumnsStmt getColumnsInfo() { return columnsInfo; } - public Expr getWherePredicate() { + public ImportWhereStmt getWherePredicate() { return wherePredicate; } 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 8ec5d940dfe2e9..00000000000000 --- a/fe/src/main/java/org/apache/doris/load/TxnStateChangeListener.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.load; - -import org.apache.doris.transaction.AbortTransactionException; -import org.apache.doris.transaction.TransactionException; -import org.apache.doris.transaction.TransactionState; - -public interface TxnStateChangeListener { - - void beforeCommitted(TransactionState txnState) throws TransactionException; - - /** - * update catalog of job which has related txn after transaction has been committed - * - * @param txnState - */ - void onCommitted(TransactionState txnState) throws TransactionException; - - /** - * 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, String txnStatusChangeReason) - throws AbortTransactionException; - - /** - * this interface is executed when transaction has been aborted - * - * @param txnState - * @param txnStatusChangeReason maybe null - */ - void onAborted(TransactionState txnState, String 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 c344f08c3f0e1a..573cb42575ddb3 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,6 +17,7 @@ package org.apache.doris.load.routineload; +import org.apache.doris.common.Pair; import org.apache.doris.thrift.TKafkaRLTaskProgress; import com.google.common.base.Joiner; @@ -36,10 +37,9 @@ public class KafkaProgress extends RoutineLoadProgress { // (partition id, begin offset) - private Map partitionIdToOffset; + private Map partitionIdToOffset = Maps.newHashMap(); public KafkaProgress() { - partitionIdToOffset = Maps.newHashMap(); } public KafkaProgress(TKafkaRLTaskProgress tKafkaRLTaskProgress) { @@ -50,6 +50,10 @@ public Map getPartitionIdToOffset() { return partitionIdToOffset; } + public void addPartitionOffset(Pair partitionOffset) { + partitionIdToOffset.put(partitionOffset.first, partitionOffset.second); + } + public void setPartitionIdToOffset(Map partitionIdToOffset) { this.partitionIdToOffset = 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 08e03f0a956ebb..1d798a4eb3e960 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 @@ -17,16 +17,18 @@ package org.apache.doris.load.routineload; -import com.google.common.base.Joiner; 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.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.LabelAlreadyUsedException; 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.common.io.Text; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; @@ -35,13 +37,17 @@ 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 org.apache.doris.transaction.TransactionState; 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.List; @@ -55,26 +61,28 @@ 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 = 10; 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; - public KafkaRoutineLoadJob(String name, long dbId, long tableId, String brokerList, String topic) { - super(name, dbId, tableId, LoadDataSourceType.KAFKA); + public KafkaRoutineLoadJob() { + // for serialization, id is dummy + super(-1, LoadDataSourceType.KAFKA); + } + + public KafkaRoutineLoadJob(Long id, String name, long dbId, long tableId, String brokerList, String topic) { + super(id, name, dbId, tableId, LoadDataSourceType.KAFKA); this.brokerList = brokerList; this.topic = topic; this.progress = new KafkaProgress(); - this.customKafkaPartitions = new ArrayList<>(); - this.currentKafkaPartitions = new ArrayList<>(); setConsumer(); } @@ -103,26 +111,6 @@ public String getBrokerList() { return brokerList; } - // this is a unprotected method which is called in the initialization function - private void setCustomKafkaPartitions(List kafkaPartitions) throws LoadException { - 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; - } - @Override public void divideRoutineLoadJob(int currentConcurrentTaskNum) { List result = new ArrayList<>(); @@ -189,8 +177,8 @@ boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { } @Override - protected void updateProgress(RLTaskTxnCommitAttachment attachment) { - super.updateProgress(attachment); + protected void updateProgress(RLTaskTxnCommitAttachment attachment, boolean isReplay) { + super.updateProgress(attachment, isReplay); this.progress.update(attachment.getProgress()); } @@ -233,7 +221,7 @@ protected boolean unprotectNeedReschedule() { .build(), e); if (this.state == JobState.NEED_SCHEDULE) { unprotectUpdateState(JobState.PAUSED, - "Job failed to fetch all current partition with error " + e.getMessage()); + "Job failed to fetch all current partition with error " + e.getMessage(), false); } return false; } @@ -270,32 +258,32 @@ protected boolean unprotectNeedReschedule() { private List getAllKafkaPartitions() { List result = new ArrayList<>(); List partitionList = consumer.partitionsFor( - topic, Duration.ofSeconds(FETCH_PARTITIONS_TIMEOUT)); + topic, Duration.ofSeconds(FETCH_PARTITIONS_TIMEOUT_SECOND)); for (PartitionInfo partitionInfo : partitionList) { result.add(partitionInfo.partition()); } return result; } - public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) throws AnalysisException, - LoadException { + public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) throws UserException { // check db and table - Database database = Catalog.getCurrentCatalog().getDb(stmt.getDBTableName().getDb()); - if (database == null) { - throw new AnalysisException("There is no database named " + stmt.getDBTableName().getDb()); + Database db = Catalog.getCurrentCatalog().getDb(stmt.getDBTableName().getDb()); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDBTableName().getDb()); } - database.readLock(); - Table table; + db.readLock(); + long tableId = -1L; try { - unprotectCheckCreate(stmt); - table = database.getTable(stmt.getDBTableName().getTbl()); + unprotectedCheckMeta(db, stmt.getDBTableName().getTbl(), stmt.getRoutineLoadDesc()); + tableId = db.getTable(stmt.getDBTableName().getTbl()).getId(); } finally { - database.readUnlock(); + db.readUnlock(); } // init kafka routine load job + long id = Catalog.getInstance().getNextId(); KafkaRoutineLoadJob kafkaRoutineLoadJob = - new KafkaRoutineLoadJob(stmt.getName(), database.getId(), table.getId(), + new KafkaRoutineLoadJob(id, stmt.getName(), db.getId(), tableId, stmt.getKafkaBrokerList(), stmt.getKafkaTopic()); kafkaRoutineLoadJob.setOptional(stmt); @@ -319,30 +307,55 @@ private void updateNewPartitionProgress() { private void setConsumer() { Properties props = new Properties(); props.put("bootstrap.servers", this.brokerList); - props.put("group.id", FE_GROUP_ID); + 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 + 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()); - if (stmt.getKafkaOffsets() != null) { - for (int i = 0; i < customKafkaPartitions.size(); i++) { - ((KafkaProgress) progress).getPartitionIdToOffset() - .put(customKafkaPartitions.get(i), stmt.getKafkaOffsets().get(i)); - } - } + } + + @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()); } } } 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 f665206195deba..550f645afe13b4 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 @@ -17,7 +17,6 @@ package org.apache.doris.load.routineload; -import com.google.common.base.Joiner; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.common.AnalysisException; @@ -32,6 +31,7 @@ import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.BeginTransactionException; +import com.google.common.base.Joiner; import com.google.common.collect.Maps; import java.util.ArrayList; @@ -64,7 +64,7 @@ public List getPartitions() { return partitions; } - // todo: reuse plan fragment of stream load + // TODO: reuse plan fragment of stream load @Override public TRoutineLoadTask createRoutineLoadTask() throws LoadException, UserException { KafkaRoutineLoadJob routineLoadJob = (KafkaRoutineLoadJob) routineLoadManager.getJob(jobId); @@ -101,6 +101,9 @@ public TRoutineLoadTask createRoutineLoadTask() throws LoadException, UserExcept 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; } 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 a41ac13f999064..2094fc94389c82 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 @@ -54,10 +54,6 @@ public RLTaskTxnCommitAttachment(TRLTaskTxnCommitAttachment rlTaskTxnCommitAttac } } - public long getJobId() { - return jobId; - } - public TUniqueId getTaskId() { return taskId; } 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 09a49fba96695d..0d619ff75801d1 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,24 +17,28 @@ package org.apache.doris.load.routineload; -import com.google.common.collect.Maps; import org.apache.doris.analysis.CreateRoutineLoadStmt; +import org.apache.doris.analysis.SqlParser; +import org.apache.doris.analysis.SqlScanner; import org.apache.doris.analysis.TableName; 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.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.LoadException; 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.DebugUtil; import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; import org.apache.doris.load.RoutineLoadDesc; -import org.apache.doris.load.TxnStateChangeListener; +import org.apache.doris.persist.RoutineLoadOperation; import org.apache.doris.planner.StreamLoadPlanner; import org.apache.doris.qe.ConnectContext; import org.apache.doris.service.ExecuteEnv; @@ -42,13 +46,15 @@ import org.apache.doris.task.StreamLoadTask; import org.apache.doris.thrift.TExecPlanFragmentParams; import org.apache.doris.thrift.TLoadTxnCommitRequest; -import org.apache.doris.thrift.TResourceInfo; import org.apache.doris.transaction.AbortTransactionException; import org.apache.doris.transaction.BeginTransactionException; import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; +import org.apache.doris.transaction.TxnStateChangeListener; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -57,6 +63,7 @@ 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.Map; @@ -71,13 +78,16 @@ * 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 extends TxnStateChangeListener implements 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 int DEFAULT_MAX_ERROR_NUM = (int) (BASE_OF_ERROR_RATE * 0.5); + private static final int DEFAULT_MAX_INTERVAL_SECOND = 5; + private static final int DEFAULT_MAX_BATCH_ROWS = 100000; + private static final int DEFAULT_MAX_BATCH_SIZE = 100 * 1024 * 1024; // 100MB private static final String STAR_STRING = "*"; protected static final int DEFAULT_TASK_MAX_CONCURRENT_NUM = 3; @@ -108,16 +118,13 @@ public enum JobState { NEED_SCHEDULE, RUNNING, PAUSED, - STOPPED, - CANCELLED; + STOPPED, CANCELLED; public boolean isFinalState() { return this == STOPPED || this == CANCELLED; } - } - protected long id; protected String name; protected long dbId; protected long tableId; @@ -125,14 +132,15 @@ public boolean isFinalState() { protected long authCode; protected RoutineLoadDesc routineLoadDesc; // 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 // if current error rate is more then max error rate, the job will be paused protected int maxErrorNum = DEFAULT_MAX_ERROR_NUM; // optional - // thrift object - protected TResourceInfo resourceInfo; + protected int maxBatchIntervalS = DEFAULT_MAX_INTERVAL_SECOND; + protected int maxBatchRows = DEFAULT_MAX_BATCH_ROWS; + protected int maxBatchSizeBytes = DEFAULT_MAX_BATCH_SIZE; protected RoutineLoadProgress progress; protected String pausedReason; @@ -141,32 +149,43 @@ public boolean isFinalState() { // 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 long currentErrorNum; + protected long currentTotalNum; // The tasks belong to this job - protected List routineLoadTaskInfoList; + protected List routineLoadTaskInfoList = Lists.newArrayList(); // plan fragment which will be initialized during job scheduler protected TExecPlanFragmentParams tExecPlanFragmentParams; - protected ReentrantReadWriteLock lock; + // 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 = new ReentrantReadWriteLock(true); // TODO(ml): error sample - public RoutineLoadJob(String name, long dbId, long tableId, LoadDataSourceType dataSourceType) { - this.id = Catalog.getInstance().getNextId(); + protected boolean isTypeRead = false; + + public void setTypeRead(boolean isTypeRead) { + this.isTypeRead = isTypeRead; + } + + public RoutineLoadJob(long id, LoadDataSourceType type) { + super(id); + this.dataSourceType = type; + } + + public RoutineLoadJob(Long id, String name, long dbId, long tableId, LoadDataSourceType dataSourceType) { + super(id); this.name = name; this.dbId = dbId; this.tableId = tableId; - this.state = JobState.NEED_SCHEDULE; this.dataSourceType = dataSourceType; - this.resourceInfo = ConnectContext.get().toResourceCtx(); this.endTimestamp = -1; this.authCode = new StringBuilder().append(ConnectContext.get().getQualifiedUser()) .append(ConnectContext.get().getRemoteIP()) .append(id).append(System.currentTimeMillis()).toString().hashCode(); - this.routineLoadTaskInfoList = new ArrayList<>(); - lock = new ReentrantReadWriteLock(true); } // TODO(ml): I will change it after ut. @@ -175,19 +194,34 @@ public RoutineLoadJob(long id, String name, long dbId, long tableId, RoutineLoadDesc routineLoadDesc, int desireTaskConcurrentNum, LoadDataSourceType dataSourceType, int maxErrorNum) { - this.id = id; + super(id); this.name = name; 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.endTimestamp = -1; - this.routineLoadTaskInfoList = new ArrayList<>(); - lock = new ReentrantReadWriteLock(true); + } + + protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException { + this.routineLoadDesc = 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(); + } } public void readLock() { @@ -206,10 +240,6 @@ public void writeUnlock() { lock.writeLock().unlock(); } - public long getId() { - return id; - } - public String getName() { return name; } @@ -273,10 +303,6 @@ public RoutineLoadDesc getRoutineLoadDesc() { return routineLoadDesc; } - public TResourceInfo getResourceInfo() { - return resourceInfo; - } - public RoutineLoadProgress getProgress() { return progress; } @@ -304,33 +330,24 @@ public String getClusterName() throws MetaNotFoundException { } } - 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 int getDesiredConcurrentNumber() { + return desireTaskConcurrentNum; } - public String getDesiredConcurrentNumber() { - if (desireTaskConcurrentNum == 0) { - return ""; - } else { - return String.valueOf(desireTaskConcurrentNum); - } + public int getMaxErrorNum() { + return maxErrorNum; } - protected void setMaxErrorNum(int maxErrorNum) throws LoadException { - writeLock(); - try { - this.maxErrorNum = maxErrorNum; - } finally { - writeUnlock(); - } + public int getMaxBatchIntervalS() { + return maxBatchIntervalS; + } + + public int getMaxBatchRows() { + return maxBatchRows; + } + + public int getMaxBatchSizeBytes() { + return maxBatchSizeBytes; } public int getSizeOfRoutineLoadTaskInfoList() { @@ -340,7 +357,6 @@ public int getSizeOfRoutineLoadTaskInfoList() { } finally { readUnlock(); } - } public TExecPlanFragmentParams gettExecPlanFragmentParams() { @@ -409,19 +425,10 @@ public Map getBeIdToConcurrentTaskNum() { } } - @Override - public void write(DataOutput out) throws IOException { - // TODO(ml) - } - - @Override - public void readFields(DataInput in) throws IOException { - // TODO(ml) - } - // if rate of error data is more then max_filter_ratio, pause job - protected void updateProgress(RLTaskTxnCommitAttachment attachment) { - updateNumOfData(attachment.getFilteredRows(), attachment.getLoadedRows() + attachment.getFilteredRows()); + protected void updateProgress(RLTaskTxnCommitAttachment attachment, boolean isReplay) { + updateNumOfData(attachment.getFilteredRows(), attachment.getLoadedRows() + attachment.getFilteredRows(), + isReplay); } public boolean containsTask(UUID taskId) { @@ -456,7 +463,7 @@ private void loadTxnCommit(TLoadTxnCommitRequest request) throws TException { frontendService.loadTxnCommit(request); } - private void updateNumOfData(long numOfErrorData, long numOfTotalData) { + private void updateNumOfData(long numOfErrorData, long numOfTotalData, boolean isReplay) { currentErrorNum += numOfErrorData; currentTotalNum += numOfTotalData; if (currentTotalNum > BASE_OF_ERROR_RATE) { @@ -468,7 +475,7 @@ private void updateNumOfData(long numOfErrorData, long numOfTotalData) { .add("msg", "current error num is more then max error num, begin to pause job") .build()); // remove all of task in jobs and change job state to paused - updateState(JobState.PAUSED, "current error num of job is more then max error num"); + updateState(JobState.PAUSED, "current error num of job is more then max error num", isReplay); } LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) @@ -488,7 +495,7 @@ private void updateNumOfData(long numOfErrorData, long numOfTotalData) { .add("msg", "current error num is more then max error num, begin to pause job") .build()); // remove all of task in jobs and change job state to paused - updateState(JobState.PAUSED, "current error num is more then max error num"); + updateState(JobState.PAUSED, "current error num is more then max error num", isReplay); // reset currentTotalNum and currentErrorNum currentErrorNum = 0; currentTotalNum = 0; @@ -528,8 +535,7 @@ public void beforeAborted(TransactionState txnState, String txnStatusChangeReaso .build()); readLock(); try { - String taskId = txnState.getLabel(); - if (routineLoadTaskInfoList.parallelStream().anyMatch(entity -> DebugUtil.printId(entity.getId()).equals(taskId))) { + if (routineLoadTaskInfoList.parallelStream().anyMatch(entity -> entity.getTxnId() == txnState.getTransactionId())) { LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) .add("txn_id", txnState.getTransactionId()) .add("msg", "task will be aborted") @@ -551,10 +557,10 @@ public void beforeCommitted(TransactionState txnState) throws TransactionExcepti // check if task has been aborted Optional routineLoadTaskInfoOptional = routineLoadTaskInfoList.parallelStream() - .filter(entity -> DebugUtil.printId(entity.getId()).equals(txnState.getLabel())).findFirst(); + .filter(entity -> entity.getTxnId() == txnState.getTransactionId()).findFirst(); if (!routineLoadTaskInfoOptional.isPresent()) { throw new TransactionException("txn " + txnState.getTransactionId() + " could not be committed" - + " while task " + txnState.getLabel() + "has been aborted "); + + " while task " + txnState.getLabel() + "has been aborted "); } } finally { readUnlock(); @@ -563,23 +569,27 @@ public void beforeCommitted(TransactionState txnState) throws TransactionExcepti // the task is committed when the correct number of rows is more then 0 @Override - public void onCommitted(TransactionState txnState) throws TransactionException { + public void onCommitted(TransactionState txnState, boolean isReplay) throws TransactionException { writeLock(); try { - // step0: find task in job - Optional routineLoadTaskInfoOptional = - routineLoadTaskInfoList.parallelStream() - .filter(entity -> DebugUtil.printId(entity.getId()).equals(txnState.getLabel())).findFirst(); - if (routineLoadTaskInfoOptional.isPresent()) { - executeCommitTask(routineLoadTaskInfoOptional.get(), txnState); + if (isReplay) { + // only update progress + if (txnState.getTxnCommitAttachment() != null) { + updateProgress((RLTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(), isReplay); + } } else { - LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) - .add("txn_id", txnState.getTransactionId()) - .add("msg", "The task is not in task info list. " - + "Maybe task has been renew or job state has changed. Transaction will not be committed.") - .build()); - throw new TransactionException("txn " + txnState.getTransactionId() + " could not be committed" - + " while task " + txnState.getLabel() + "has been aborted "); + // find task in job + Optional routineLoadTaskInfoOptional = routineLoadTaskInfoList.parallelStream().filter( + entity -> entity.getTxnId() == txnState.getTransactionId()).findFirst(); + if (routineLoadTaskInfoOptional.isPresent()) { + executeCommitTask(routineLoadTaskInfoOptional.get(), txnState); + } else { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()).add("txn_id", + txnState.getTransactionId()).add("msg", "The task is not in task info list. " + + "Maybe task has been renew or job state has changed. Transaction will not be committed.").build()); + throw new TransactionException("txn " + txnState.getTransactionId() + " could not be committed" + + " while task " + txnState.getLabel() + "has been aborted "); + } } } catch (TransactionException e) { LOG.warn(e.getMessage(), e); @@ -587,7 +597,7 @@ public void onCommitted(TransactionState txnState) throws TransactionException { } catch (Throwable e) { LOG.warn(e.getMessage(), e); updateState(JobState.PAUSED, "failed to update offset when transaction " - + txnState.getTransactionId() + " has been committed"); + + txnState.getTransactionId() + " has been committed", isReplay); } finally { writeUnlock(); } @@ -598,49 +608,47 @@ public void onCommitted(TransactionState txnState) throws TransactionException { // txn will be aborted but progress will be update // progress will be update otherwise the progress will be hung @Override - public void onAborted(TransactionState txnState, String txnStatusChangeReasonString) { + public void onAborted(TransactionState txnState, String txnStatusChangeReasonString, boolean isReplay) { writeLock(); try { - // step0: find task in job - Optional routineLoadTaskInfoOptional = - routineLoadTaskInfoList.parallelStream() - .filter(entity -> DebugUtil.printId(entity.getId()).equals(txnState.getLabel())).findFirst(); - if (routineLoadTaskInfoOptional.isPresent()) { - // step1: job state will be changed depending on txnStatusChangeReasonString - if (txnStatusChangeReasonString != null) { - LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) - .add("txn_id", txnState.getTransactionId()) - .add("msg", "txn abort with reason " + txnStatusChangeReasonString) - .build()); - TransactionState.TxnStatusChangeReason txnStatusChangeReason = - TransactionState.TxnStatusChangeReason.fromString(txnStatusChangeReasonString); - if (txnStatusChangeReason != null) { - switch (txnStatusChangeReason) { - case OFFSET_OUT_OF_RANGE: - updateState(JobState.CANCELLED, txnStatusChangeReason.toString()); - return; - default: - break; + if (isReplay) { + // only update progress + if (txnState.getTxnCommitAttachment() != null) { + updateProgress((RLTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(), isReplay); + } + } else { + // step0: find task in job + Optional routineLoadTaskInfoOptional = routineLoadTaskInfoList.parallelStream().filter( + entity -> entity.getTxnId() == txnState.getTransactionId()).findFirst(); + if (routineLoadTaskInfoOptional.isPresent()) { + // step1: job state will be changed depending on txnStatusChangeReasonString + if (txnStatusChangeReasonString != null) { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()).add("txn_id", + txnState.getTransactionId()).add("msg", + "txn abort with reason " + txnStatusChangeReasonString).build()); + TransactionState.TxnStatusChangeReason txnStatusChangeReason = TransactionState.TxnStatusChangeReason.fromString( + txnStatusChangeReasonString); + if (txnStatusChangeReason != null) { + switch (txnStatusChangeReason) { + case OFFSET_OUT_OF_RANGE: + updateState(JobState.CANCELLED, txnStatusChangeReason.toString(), isReplay); + return; + default: + break; + } } + // todo(ml): use previous be id depend on change reason + } else { + LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()).add("txn_id", + txnState.getTransactionId()).add("msg", "txn abort").build()); } - // todo(ml): use previous be id depend on change reason - } else { - LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) - .add("txn_id", txnState.getTransactionId()) - .add("msg", "txn abort").build()); + // step2: commit task , update progress, maybe create a new task + executeCommitTask(routineLoadTaskInfoOptional.get(), txnState); } - // step2: commit task , update progress, maybe create a new task - executeCommitTask(routineLoadTaskInfoOptional.get(), txnState); - } else { - LOG.debug(new LogBuilder(LogKey.ROUINTE_LOAD_TASK, txnState.getLabel()) - .add("txn_id", txnState.getTransactionId()) - .add("msg", "The task is not in task info list. " - + "Maybe task has been renew or job state has changed. Transaction will not be aborted successfully.") - .build()); } } catch (Exception e) { updateState(JobState.PAUSED, - "failed to renew task when txn has been aborted with error " + e.getMessage()); + "failed to renew task when txn has been aborted with error " + e.getMessage(), isReplay); // TODO(ml): edit log LOG.warn("failed to renew a routine load task in job {} with error message {}", id, e.getMessage(), e); } finally { @@ -662,7 +670,7 @@ private void executeCommitTask(RoutineLoadTaskInfo routineLoadTaskInfo, Transact .build()); } else if (checkCommitInfo(rlTaskTxnCommitAttachment)) { // step2: update job progress - updateProgress(rlTaskTxnCommitAttachment); + updateProgress(rlTaskTxnCommitAttachment, false /* not replay */); } if (state == JobState.RUNNING) { @@ -672,8 +680,36 @@ private void executeCommitTask(RoutineLoadTaskInfo routineLoadTaskInfo, Transact } } - // check the correctness of commit info - abstract boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment); + 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"); + } + } + + // columns will be checked when planing + } protected static void unprotectCheckCreate(CreateRoutineLoadStmt stmt) throws AnalysisException { // check table belong to db, partitions belong to table @@ -712,20 +748,16 @@ private static void checkDBSemantics(TableName dbTableName, List partiti } } - public void updateState(JobState jobState) { - updateState(jobState, null); - } - - public void updateState(JobState jobState, String reason) { + public void updateState(JobState jobState, String reason, boolean isReplay) { writeLock(); try { - unprotectUpdateState(jobState, reason); + unprotectUpdateState(jobState, reason, isReplay); } finally { writeUnlock(); } } - protected void unprotectUpdateState(JobState jobState, String reason) { + protected void unprotectUpdateState(JobState jobState, String reason, boolean isReplay) { LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) .add("current_job_state", getState()) .add("desire_job_state", jobState) @@ -748,6 +780,9 @@ protected void unprotectUpdateState(JobState jobState, String reason) { default: break; } + if (!isReplay) { + 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") @@ -755,7 +790,6 @@ protected void unprotectUpdateState(JobState jobState, String reason) { } private void executePause(String reason) { - // TODO(ml): edit log // remove all of task in jobs and change job state to paused pausedReason = reason; state = JobState.PAUSED; @@ -763,13 +797,11 @@ private void executePause(String reason) { } private void executeNeedSchedule() { - // TODO(ml): edit log state = JobState.NEED_SCHEDULE; routineLoadTaskInfoList.clear(); } private void executeStop() { - // TODO(ml): edit log state = JobState.STOPPED; routineLoadTaskInfoList.clear(); endTimestamp = System.currentTimeMillis(); @@ -789,7 +821,7 @@ public void update() { LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) .add("db_id", dbId) .add("msg", "The database has been deleted. Change job state to stopped").build()); - updateState(JobState.STOPPED); + updateState(JobState.STOPPED, "db not exist", false /* not replay */); } database.readLock(); try { @@ -799,7 +831,7 @@ public void update() { 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 stopped").build()); - updateState(JobState.STOPPED); + updateState(JobState.STOPPED, "table not exist", false /* not replay */); } } finally { database.readUnlock(); @@ -826,4 +858,92 @@ protected void unprotectUpdateProgress() { protected boolean unprotectNeedReschedule() { return false; } + + public void setOrigStmt(String origStmt) { + this.origStmt = origStmt; + } + + public String getOrigStmt() { + return origStmt; + } + + // check the correctness of commit info + abstract boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment); + + 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); + out.writeLong(dbId); + out.writeLong(tableId); + out.writeInt(desireTaskConcurrentNum); + out.writeInt(maxErrorNum); + out.writeInt(maxBatchIntervalS); + out.writeInt(maxBatchRows); + out.writeInt(maxBatchSizeBytes); + progress.write(out); + out.writeLong(currentErrorNum); + out.writeLong(currentTotalNum); + 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); + dbId = in.readLong(); + tableId = in.readLong(); + desireTaskConcurrentNum = in.readInt(); + maxErrorNum = in.readInt(); + maxBatchIntervalS = in.readInt(); + maxBatchRows = in.readInt(); + maxBatchSizeBytes = in.readInt(); + + switch (dataSourceType) { + case KAFKA: { + progress = new KafkaProgress(); + progress.readFields(in); + break; + } + default: + throw new IOException("unknown data source type: " + dataSourceType); + } + + currentErrorNum = in.readLong(); + currentTotalNum = 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); + routineLoadDesc = stmt.getRoutineLoadDesc(); + } catch (Throwable 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 5304df7658cf17..6e03fa49d7c7d2 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,11 +17,6 @@ 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 com.google.common.collect.Queues; -import com.sleepycat.je.tree.IN; import org.apache.doris.analysis.CreateRoutineLoadStmt; import org.apache.doris.analysis.PauseRoutineLoadStmt; import org.apache.doris.analysis.ResumeRoutineLoadStmt; @@ -35,38 +30,44 @@ 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.base.Joiner; +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.util.ArrayList; -import java.util.Collection; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; 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; // Long is beId, integer is the size of tasks in be - private Map beIdToMaxConcurrentTasks; + private Map beIdToMaxConcurrentTasks = Maps.newHashMap(); // stream load job meta - private Map idToRoutineLoadJob; - private Map>> dbToNameToRoutineLoadJob; + 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(); @@ -85,10 +86,6 @@ private void writeUnlock() { } public RoutineLoadManager() { - idToRoutineLoadJob = Maps.newConcurrentMap(); - dbToNameToRoutineLoadJob = Maps.newConcurrentMap(); - beIdToMaxConcurrentTasks = Maps.newConcurrentMap(); - lock = new ReentrantReadWriteLock(true); } private void updateBeIdToMaxConcurrentTasks() { @@ -144,8 +141,8 @@ private Map getBeIdConcurrentTaskMaps() { } - 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(), @@ -163,11 +160,11 @@ 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); } public void addRoutineLoadJob(RoutineLoadJob routineLoadJob) throws DdlException { @@ -178,13 +175,37 @@ public void addRoutineLoadJob(RoutineLoadJob routineLoadJob) throws DdlException throw new DdlException("Name " + routineLoadJob.getName() + " already used in db " + routineLoadJob.getDbId()); } - 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); + 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); + } + // register txn state listener + Catalog.getCurrentGlobalTransactionMgr().getListenerRegistry().register(routineLoadJob); } // TODO(ml): Idempotency @@ -204,26 +225,6 @@ 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()); if (routineLoadJob == null) { @@ -249,7 +250,8 @@ public void pauseRoutineLoadJob(PauseRoutineLoadStmt pauseRoutineLoadStmt) throw } routineLoadJob.updateState(RoutineLoadJob.JobState.PAUSED, - "User " + ConnectContext.get().getQualifiedUser() + "pauses routine load job"); + "User " + ConnectContext.get().getQualifiedUser() + "pauses routine load job", + false /* not replay */); } public void resumeRoutineLoadJob(ResumeRoutineLoadStmt resumeRoutineLoadStmt) throws DdlException, @@ -276,7 +278,8 @@ public void resumeRoutineLoadJob(ResumeRoutineLoadStmt resumeRoutineLoadStmt) th ConnectContext.get().getRemoteIP(), tableName); } - routineLoadJob.updateState(RoutineLoadJob.JobState.NEED_SCHEDULE); + routineLoadJob.updateState(RoutineLoadJob.JobState.NEED_SCHEDULE, "user operation", + false /* not replay */); } public void stopRoutineLoadJob(StopRoutineLoadStmt stopRoutineLoadStmt) throws DdlException, AnalysisException { @@ -302,7 +305,7 @@ public void stopRoutineLoadJob(StopRoutineLoadStmt stopRoutineLoadStmt) throws D ConnectContext.get().getRemoteIP(), tableName); } - routineLoadJob.updateState(RoutineLoadJob.JobState.STOPPED); + routineLoadJob.updateState(RoutineLoadJob.JobState.STOPPED, "user operation", false /* not replay */); } public int getSizeOfIdToRoutineLoadTask() { @@ -483,4 +486,43 @@ public void updateRoutineLoadJob() { } } + public void replayCreateRoutineLoadJob(RoutineLoadJob routineLoadJob) { + unprotectedAddJob(routineLoadJob); + LOG.info("replay add routine load job: {}", routineLoadJob.getId()); + } + + public void replayChangeRoutineLoadJob(RoutineLoadOperation operation) { + RoutineLoadJob job = getJob(operation.getId()); + job.updateState(operation.getJobState(), "replay", true /* is replay */); + LOG.info("replay change routine load job: {}, state: {}", operation.getId(), operation.getJobState()); + } + + @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); + } + } } 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 cc79e1ec52c319..abdf911a34a33a 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 @@ -17,14 +17,15 @@ package org.apache.doris.load.routineload; -import com.google.common.annotations.VisibleForTesting; import org.apache.doris.catalog.Catalog; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; 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; @@ -90,15 +91,10 @@ private void process() { // check state and divide job into tasks routineLoadJob.divideRoutineLoadJob(desiredConcurrentTaskNum); } catch (MetaNotFoundException e) { - LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) - .add("error_msg", "failed to get metadata, change job state to cancelled") - .build(), e); - routineLoadJob.updateState(RoutineLoadJob.JobState.CANCELLED, e.getMessage()); + routineLoadJob.updateState(RoutineLoadJob.JobState.CANCELLED, e.getMessage(), false /* not replay */); } catch (Throwable e) { - LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, routineLoadJob.getId()) - .add("error_msg", "failed to scheduler job, change job state to paused") - .build(), e); - routineLoadJob.updateState(RoutineLoadJob.JobState.PAUSED, e.getMessage()); + LOG.warn("failed to scheduler job, change job state to paused", e); + routineLoadJob.updateState(RoutineLoadJob.JobState.PAUSED, e.getMessage(), false /* not replay */); continue; } } 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 c55bcabdfe94f6..fec8fb731e0ee4 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 @@ -102,7 +102,7 @@ public void beginTxn() throws LabelAlreadyUsedException, BeginTransactionExcepti RoutineLoadJob routineLoadJob = routineLoadManager.getJob(jobId); txnId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction( routineLoadJob.getDbId(), DebugUtil.printId(id), -1, "streamLoad", - TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, routineLoadJob); + TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, routineLoadJob.getId()); } @Override 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 a8f520c6c27cef..83a40f2c563f0a 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 @@ -17,11 +17,6 @@ package org.apache.doris.load.routineload; -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.common.collect.Queues; import org.apache.doris.catalog.Catalog; import org.apache.doris.common.ClientPool; import org.apache.doris.common.LoadException; @@ -31,17 +26,21 @@ import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; import org.apache.doris.system.Backend; -import org.apache.doris.task.RoutineLoadTask; 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.base.Joiner; +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.Queue; import java.util.concurrent.LinkedBlockingQueue; /** @@ -179,7 +178,6 @@ private void submitBatchTask(Map> beIdToRoutineLoad ClientPool.backendPool.invalidateObject(address, client); } } - } } 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..b4a30f902a918b 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,16 @@ 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; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); @@ -1166,4 +1176,12 @@ 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); + } } 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..dc7c8026e6309f 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; + } 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..a88f02fab7ca95 100644 --- a/fe/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -511,6 +511,7 @@ private void getScanRangeLocations(Analyzer analyzer) throws UserException, Anal partitionIds.add(partition.getId()); } } + selectedPartitionNum = partitionIds.size(); LOG.debug("partition prune cost: {} ms, partitions: {}", (System.currentTimeMillis() - start), partitionIds); @@ -576,7 +577,7 @@ private void getScanRangeLocations(Analyzer analyzer) throws UserException, Anal int j = 0; for (Long partitionId : partitionIds) { Partition partition = olapTable.getPartition(partitionId); - LOG.debug("selected partition: " + partition.getName()); + LOG.debug("selected partition: {}", partition.getName()); selectedTable = tables.get(j++).get(partitionPos); List tablets = new ArrayList(); Collection tabletIds = distributionPrune(selectedTable, partition.getDistributionInfo()); 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/ShowExecutor.java b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java index a91a44bf38edd9..69914661aa8360 100644 --- a/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -829,7 +829,7 @@ private void handleShowRoutineLoad() throws AnalysisException { row.add(String.valueOf(routineLoadJob.getTableId())); row.add(routineLoadJob.getPartitions()); row.add(routineLoadJob.getState().name()); - row.add(routineLoadJob.getDesiredConcurrentNumber()); + row.add(String.valueOf(routineLoadJob.getDesiredConcurrentNumber())); row.add(routineLoadJob.getProgress().toString()); rows.add(row); 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..1da9457fbc3c06 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; @@ -68,9 +64,13 @@ import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TQueryOptions; -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; @@ -750,7 +750,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 7c52d52f2f5b33..60dde9ca014067 100644 --- a/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -593,7 +593,7 @@ private long loadTxnBeginImpl(TLoadTxnBeginRequest request) throws UserException long timestamp = request.isSetTimestamp() ? request.getTimestamp() : -1; return Catalog.getCurrentGlobalTransactionMgr().beginTransaction( db.getId(), request.getLabel(), timestamp, "streamLoad", - TransactionState.LoadJobSourceType.BACKEND_STREAMING, null); + TransactionState.LoadJobSourceType.BACKEND_STREAMING, -1); } @Override @@ -729,6 +729,7 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) } throw new UserException("unknown database, database=" + dbName); } + db.readLock(); try { Table table = db.getTable(request.getTbl()); diff --git a/fe/src/main/java/org/apache/doris/task/KafkaRoutineLoadTask.java b/fe/src/main/java/org/apache/doris/task/KafkaRoutineLoadTask.java deleted file mode 100644 index 922127387374ed..00000000000000 --- a/fe/src/main/java/org/apache/doris/task/KafkaRoutineLoadTask.java +++ /dev/null @@ -1,40 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.task; - -import org.apache.doris.load.routineload.LoadDataSourceType; -import org.apache.doris.thrift.TResourceInfo; - -import java.util.Map; - - -public class KafkaRoutineLoadTask extends RoutineLoadTask { - - private Map partitionIdToOffset; - - 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; - } - - public Map getPartitionIdToOffset() { - return partitionIdToOffset; - } -} 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 index b37d16b2ae19e2..040d492ad92398 100644 --- a/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java +++ b/fe/src/main/java/org/apache/doris/task/StreamLoadTask.java @@ -20,8 +20,6 @@ package org.apache.doris.task; -import com.google.common.base.Joiner; -import com.google.common.collect.Maps; import org.apache.doris.analysis.ColumnSeparator; import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.ImportColumnDesc; @@ -29,17 +27,18 @@ import org.apache.doris.analysis.ImportWhereStmt; import org.apache.doris.analysis.SqlParser; import org.apache.doris.analysis.SqlScanner; -import org.apache.doris.catalog.Catalog; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.load.RoutineLoadDesc; import org.apache.doris.load.routineload.RoutineLoadJob; -import org.apache.doris.load.routineload.RoutineLoadManager; -import org.apache.doris.load.routineload.RoutineLoadTaskInfo; 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 com.google.common.collect.Maps; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -147,10 +146,16 @@ private void setOptionalFromRoutineLoadJob(RoutineLoadJob routineLoadJob) { if (routineLoadJob.getRoutineLoadDesc() != null) { RoutineLoadDesc routineLoadDesc = routineLoadJob.getRoutineLoadDesc(); if (routineLoadDesc.getColumnsInfo() != null) { - columnToColumnExpr = routineLoadDesc.getColumnsInfo().getParsedExprMap(); + ImportColumnsStmt columnsStmt = routineLoadDesc.getColumnsInfo(); + if (columnsStmt.getColumns() != null || columnsStmt.getColumns().size() != 0) { + columnToColumnExpr = Maps.newHashMap(); + for (ImportColumnDesc columnDesc : columnsStmt.getColumns()) { + columnToColumnExpr.put(columnDesc.getColumn(), columnDesc.getExpr()); + } + } } if (routineLoadDesc.getWherePredicate() != null) { - whereExpr = routineLoadDesc.getWherePredicate(); + whereExpr = routineLoadDesc.getWherePredicate().getExpr(); } if (routineLoadDesc.getColumnSeparator() != null) { columnSeparator = routineLoadDesc.getColumnSeparator(); @@ -162,7 +167,7 @@ private void setOptionalFromRoutineLoadJob(RoutineLoadJob routineLoadJob) { } private void setColumnToColumnExpr(String columns) throws UserException { - String columnsSQL = new String("COLUMNS " + columns); + String columnsSQL = new String("COLUMNS (" + columns + ")"); SqlParser parser = new SqlParser(new SqlScanner(new StringReader(columnsSQL))); ImportColumnsStmt columnsStmt; try { 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 42678c55748fa3..4b0d1c416067fe 100644 --- a/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -39,7 +39,6 @@ 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.persist.EditLog; import org.apache.doris.task.AgentTaskQueue; import org.apache.doris.task.PublishVersionTask; @@ -88,9 +87,10 @@ public class GlobalTransactionMgr { private com.google.common.collect.Table dbIdToTxnLabels; private Map runningTxnNums; private TransactionIdGenerator idGenerator; + private TxnStateListenerRegistry listenerRegistry = new TxnStateListenerRegistry(); private Catalog catalog; - + public GlobalTransactionMgr(Catalog catalog) { idToTransactionState = new HashMap<>(); dbIdToTxnLabels = HashBasedTable.create(); @@ -99,16 +99,20 @@ public GlobalTransactionMgr(Catalog catalog) { this.idGenerator = new TransactionIdGenerator(); } + public TxnStateListenerRegistry getListenerRegistry() { + return listenerRegistry; + } + public long beginTransaction(long dbId, String label, String coordinator, LoadJobSourceType sourceType) throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException { - return beginTransaction(dbId, label, -1, coordinator, sourceType, null); + return beginTransaction(dbId, label, -1, coordinator, sourceType, -1); } /** * 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,8 +121,7 @@ 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) throws AnalysisException, LabelAlreadyUsedException, BeginTransactionException { if (Config.disable_load_job) { @@ -151,7 +154,7 @@ public long beginTransaction(long dbId, String label, long timestamp, 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); transactionState.setPrepareTime(System.currentTimeMillis()); unprotectUpsertTransactionState(transactionState); return tid; @@ -361,7 +364,7 @@ public void commitTransaction(long dbId, long transactionId, List getReadyToPublishTransactions() { long dbId = transactionState.getDbId(); Database db = catalog.getDb(dbId); if (null == db) { - transactionState.setTransactionStatus(TransactionStatus.ABORTED); + transactionState.setTransactionStatus(TransactionStatus.ABORTED, false /* not replay */); unprotectUpsertTransactionState(transactionState); continue; } @@ -570,7 +572,7 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) { if (db == null) { writeLock(); try { - transactionState.setTransactionStatus(TransactionStatus.ABORTED); + transactionState.setTransactionStatus(TransactionStatus.ABORTED, false /* not replay */); transactionState.setReason("db is dropped"); LOG.warn("db is dropped during transaction, abort transaction {}", transactionState); unprotectUpsertTransactionState(transactionState); @@ -694,7 +696,7 @@ public void finishTransaction(long transactionId, Set errorReplicaIds) { try { transactionState.setErrorReplicas(errorReplicaIds); transactionState.setFinishTime(System.currentTimeMillis()); - transactionState.setTransactionStatus(TransactionStatus.VISIBLE); + transactionState.setTransactionStatus(TransactionStatus.VISIBLE, false /* not replay */); unprotectUpsertTransactionState(transactionState); } catch (TransactionException e) { LOG.warn("failed to change transaction {} status to visible", transactionState.getTransactionId()); @@ -793,7 +795,7 @@ public void removeOldTransactions() { || !checkTxnHasRelatedJob(transactionState, dbIdToTxnIds)) { try { transactionState.setTransactionStatus(TransactionStatus.ABORTED, - TransactionState.TxnStatusChangeReason.TIMEOUT.name()); + TransactionState.TxnStatusChangeReason.TIMEOUT.name(), false /* not replay */); } catch (TransactionException e) { LOG.warn("txn {} could not be aborted with error message {}", transactionState.getTransactionId(), e.getMessage()); @@ -861,7 +863,11 @@ private void writeUnlock() { // for add/update/delete TransactionState private void unprotectUpsertTransactionState(TransactionState transactionState) { - editLog.logInsertTransactionState(transactionState); + if (transactionState.getTransactionStatus() != TransactionStatus.PREPARE) { + // no need to persist prepare txn. if prepare txn lost, the following commit will just be failed. + // user only need to retry this txn. + editLog.logInsertTransactionState(transactionState); + } idToTransactionState.put(transactionState.getTransactionId(), transactionState); updateTxnLabels(transactionState); updateDBRunningTxnNum(transactionState.getPreStatus(), transactionState); @@ -890,7 +896,7 @@ private void unprotectAbortTransaction(long transactionId, String reason, TxnCom } transactionState.setFinishTime(System.currentTimeMillis()); transactionState.setReason(reason); - transactionState.setTransactionStatus(TransactionStatus.ABORTED, reason); + transactionState.setTransactionStatus(TransactionStatus.ABORTED, reason, false /* not replay */); unprotectUpsertTransactionState(transactionState); for (PublishVersionTask task : transactionState.getPublishVersionTasks().values()) { AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.PUBLISH_VERSION, task.getSignature()); @@ -902,6 +908,8 @@ private void unprotectAbortTransaction(long transactionId, String reason, TxnCom public void replayUpsertTransactionState(TransactionState transactionState) { writeLock(); try { + // set transaction status will call txn state change listener + transactionState.setTransactionStatus(transactionState.getTransactionStatus(), true /* is replay */); Database db = catalog.getDb(transactionState.getDbId()); if (transactionState.getTransactionStatus() == TransactionStatus.COMMITTED) { LOG.debug("replay a committed transaction {}", transactionState); @@ -915,6 +923,9 @@ public void replayUpsertTransactionState(TransactionState transactionState) { updateTxnLabels(transactionState); updateDBRunningTxnNum(preTxnState == null ? null : preTxnState.getTransactionStatus(), transactionState); + } catch (TransactionException e) { + // should not happen + throw new RuntimeException(e); } finally { writeUnlock(); } 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 69b360a11a75e9..0292cde4fbe006 100644 --- a/fe/src/main/java/org/apache/doris/transaction/TransactionState.java +++ b/fe/src/main/java/org/apache/doris/transaction/TransactionState.java @@ -17,10 +17,10 @@ package org.apache.doris.transaction; +import org.apache.doris.catalog.Catalog; import org.apache.doris.common.Config; 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; @@ -133,9 +133,10 @@ public String toString() { private long publishVersionTime; private TransactionStatus preStatus = null; + private long listenerId; + // optional private TxnCommitAttachment txnCommitAttachment; - private TxnStateChangeListener txnStateChangeListener; public TransactionState() { this.dbId = -1; @@ -177,11 +178,9 @@ public TransactionState(long dbId, long transactionId, String label, long timest } public TransactionState(long dbId, long transactionId, String label, long timestamp, - LoadJobSourceType sourceType, String coordinator, TxnStateChangeListener txnStateChangeListener) { + LoadJobSourceType sourceType, String coordinator, long listenerId) { this(dbId, transactionId, label, timestamp, sourceType, coordinator); - if (txnStateChangeListener != null) { - this.txnStateChangeListener = txnStateChangeListener; - } + this.listenerId = listenerId; } public void setErrorReplicas(Set newErrorReplicas) { @@ -221,68 +220,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; @@ -320,49 +257,57 @@ public TransactionStatus getPreStatus() { return this.preStatus; } - public TxnCommitAttachment getTxnCommitAttachment() { return txnCommitAttachment; } - public void setTransactionStatus(TransactionStatus transactionStatus) throws TransactionException { - setTransactionStatus(transactionStatus, null); + public void setTransactionStatus(TransactionStatus transactionStatus, boolean isReplay) + throws TransactionException { + setTransactionStatus(transactionStatus, null, isReplay); } - public void setTransactionStatus(TransactionStatus transactionStatus, String txnStatusChangeReason) - throws TransactionException { + public void setTransactionStatus(TransactionStatus transactionStatus, String txnStatusChangeReason, + boolean isReplay) throws TransactionException { // before state changed - if (txnStateChangeListener != null) { - switch (transactionStatus) { - case ABORTED: - txnStateChangeListener.beforeAborted(this, txnStatusChangeReason); - break; - case COMMITTED: - txnStateChangeListener.beforeCommitted(this); - default: - break; + TxnStateChangeListener listener = Catalog.getCurrentGlobalTransactionMgr().getListenerRegistry().getListener(listenerId); + if (!isReplay) { + if (listener != null) { + switch (transactionStatus) { + case ABORTED: + listener.beforeAborted(this, txnStatusChangeReason); + break; + case COMMITTED: + listener.beforeCommitted(this); + default: + break; + } } } - // state changed - this.preStatus = this.transactionStatus; - this.transactionStatus = transactionStatus; + // if is replay, the status is already be set + if (!isReplay) { + // state changed + this.preStatus = this.transactionStatus; + this.transactionStatus = transactionStatus; + } // after state changed if (transactionStatus == TransactionStatus.VISIBLE) { - this.latch.countDown(); - if (MetricRepo.isInit.get()) { - MetricRepo.COUNTER_TXN_SUCCESS.increase(1L); + if (!isReplay) { + this.latch.countDown(); + if (MetricRepo.isInit.get()) { + MetricRepo.COUNTER_TXN_SUCCESS.increase(1L); + } } } else if (transactionStatus == TransactionStatus.ABORTED) { if (MetricRepo.isInit.get()) { MetricRepo.COUNTER_TXN_FAILED.increase(1L); } - if (txnStateChangeListener != null) { - txnStateChangeListener.onAborted(this, txnStatusChangeReason); + if (listener != null) { + listener.onAborted(this, txnStatusChangeReason, isReplay); } - } else if (transactionStatus == TransactionStatus.COMMITTED && txnStateChangeListener != null) { - txnStateChangeListener.onCommitted(this); + } else if (transactionStatus == TransactionStatus.COMMITTED && listener != null) { + listener.onCommitted(this, isReplay); } } @@ -450,8 +395,66 @@ public boolean isPublishTimeout() { return System.currentTimeMillis() - publishVersionTime > timeoutMillis; } - public TxnStateChangeListener getTxnStateChangeListener() { - return 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); + } + // 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 + } } 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 dc17df77deee1c..70eef69c0e4cb4 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); @@ -131,7 +133,6 @@ public void analyze(Analyzer analyzer1) { Assert.assertEquals(0, createRoutineLoadStmt.getMaxErrorNum()); 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/load/routineload/KafkaRoutineLoadJobTest.java b/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java index 731adcdd582ea5..3c166aad31ef39 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,31 +17,22 @@ 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.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.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; @@ -49,6 +40,10 @@ 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; import org.apache.logging.log4j.LogManager; @@ -64,6 +59,14 @@ 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; +import mockit.Verifications; + public class KafkaRoutineLoadJobTest { private static final Logger LOG = LogManager.getLogger(KafkaRoutineLoadJobTest.class); @@ -157,7 +160,7 @@ public void testDivideRoutineLoadJob(@Mocked KafkaConsumer kafkaConsumer, new Expectations() { { globalTransactionMgr.beginTransaction(anyLong, anyString, anyLong, anyString, - TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, (KafkaRoutineLoadJob) any); + TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, anyLong); result = 0L; catalog.getRoutineLoadManager(); result = routineLoadManager; @@ -208,7 +211,7 @@ public void testProcessTimeOutTasks(@Mocked KafkaConsumer kafkaConsumer, new Expectations() { { globalTransactionMgr.beginTransaction(anyLong, anyString, anyLong, anyString, - TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, routineLoadJob); + TransactionState.LoadJobSourceType.ROUTINE_LOAD_TASK, routineLoadJob.getId()); result = 0L; catalog.getRoutineLoadManager(); result = routineLoadManager; @@ -279,7 +282,7 @@ public void testFromCreateStmtWithErrorPartition(@Mocked Catalog catalog, try { KafkaRoutineLoadJob kafkaRoutineLoadJob = KafkaRoutineLoadJob.fromCreateStmt(createRoutineLoadStmt); Assert.fail(); - } catch (AnalysisException e) { + } catch (UserException e) { LOG.info(e.getMessage()); } } @@ -303,7 +306,7 @@ public void testFromCreateStmtWithErrorTable(@Mocked Catalog catalog, try { KafkaRoutineLoadJob kafkaRoutineLoadJob = KafkaRoutineLoadJob.fromCreateStmt(createRoutineLoadStmt); Assert.fail(); - } catch (AnalysisException e) { + } catch (UserException e) { LOG.info(e.getMessage()); } } @@ -312,7 +315,7 @@ public void testFromCreateStmtWithErrorTable(@Mocked Catalog catalog, 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); 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 3ac1c088ead059..1711d1cdc2ecd2 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,14 +17,6 @@ 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 org.apache.doris.analysis.ColumnSeparator; import org.apache.doris.analysis.CreateRoutineLoadStmt; import org.apache.doris.analysis.ParseNode; @@ -33,11 +25,16 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.common.LoadException; +import org.apache.doris.common.UserException; import org.apache.doris.mysql.privilege.PaloAuth; import org.apache.doris.mysql.privilege.PrivPredicate; 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,6 +45,13 @@ 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); @@ -59,9 +63,9 @@ public class RoutineLoadManagerTest { @Test public void testAddJobByStmt(@Injectable PaloAuth paloAuth, - @Injectable TResourceInfo tResourceInfo, - @Mocked ConnectContext connectContext, - @Mocked Catalog catalog) throws DdlException, LoadException, AnalysisException { + @Injectable TResourceInfo tResourceInfo, + @Mocked ConnectContext connectContext, + @Mocked Catalog catalog) throws UserException { String jobName = "job1"; String dbName = "db1"; String tableNameString = "table1"; @@ -81,7 +85,8 @@ public void testAddJobByStmt(@Injectable PaloAuth paloAuth, loadPropertyList, properties, typeName, customProperties); - KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(jobName, 1L, 1L, serverAddress, topicName); + KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(1L, jobName, 1L, 1L, serverAddress, + topicName); new MockUp() { @@ -100,7 +105,7 @@ public KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) { } }; RoutineLoadManager routineLoadManager = new RoutineLoadManager(); - routineLoadManager.addRoutineLoadJob(createRoutineLoadStmt); + routineLoadManager.createRoutineLoadJob(createRoutineLoadStmt, "dummy"); Map idToRoutineLoadJob = Deencapsulation.getField(routineLoadManager, "idToRoutineLoadJob"); @@ -120,8 +125,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 @@ -159,12 +162,14 @@ public void testCreateJobAuthDeny(@Injectable PaloAuth paloAuth, }; 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(); } } @@ -173,14 +178,16 @@ 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, 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, 1L, 1L, + serverAddress, topicName); routineLoadJobList.add(kafkaRoutineLoadJobWithSameName); nameToRoutineLoadJob.put(jobName, routineLoadJobList); dbToNameToRoutineLoadJob.put(1L, nameToRoutineLoadJob); @@ -199,14 +206,16 @@ public void testCreateWithSameNameOfStoppedJob(@Mocked ConnectContext connectCon 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, 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, 1L, 1L, + serverAddress, topicName); Deencapsulation.setField(kafkaRoutineLoadJobWithSameName, "state", RoutineLoadJob.JobState.STOPPED); routineLoadJobList.add(kafkaRoutineLoadJobWithSameName); nameToRoutineLoadJob.put(jobName, routineLoadJobList); 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 e26ad28f2558cf..975b091b58e3c9 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,23 +17,19 @@ 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.common.DdlException; import org.apache.doris.common.LoadException; import org.apache.doris.common.MetaNotFoundException; +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 com.google.common.collect.Lists; + import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; @@ -41,6 +37,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 @@ -119,7 +120,6 @@ public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, // } } - public void functionTest(@Mocked Catalog catalog, @Mocked SystemInfoService systemInfoService, @Injectable Database database) throws DdlException, InterruptedException { @@ -130,7 +130,8 @@ 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", 1L, 1L, "10.74.167.16:8092", + "test"); RoutineLoadManager routineLoadManager = new RoutineLoadManager(); routineLoadManager.addRoutineLoadJob(kafkaRoutineLoadJob); @@ -157,9 +158,8 @@ 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", 1L, 1L, + "10.74.167.16:8092", "test_1"); List customKafkaPartitions = new ArrayList<>(); customKafkaPartitions.add(2); Deencapsulation.setField(kafkaRoutineLoadJob1, "customKafkaPartitions", customKafkaPartitions); 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 45aabb1aed9a5a..3fdb5bcc32be8d 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 @@ -24,17 +24,12 @@ 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.transaction.BeginTransactionException; import com.google.common.collect.Maps; import com.google.common.collect.Queues; -import org.junit.Assert; import org.junit.Test; import java.util.Map; @@ -45,7 +40,6 @@ import mockit.Expectations; import mockit.Injectable; import mockit.Mocked; -import mockit.Verifications; public class RoutineLoadTaskSchedulerTest { @@ -119,35 +113,8 @@ public void testRunOneCycle(@Injectable KafkaRoutineLoadJob kafkaRoutineLoadJob1 } }; - KafkaRoutineLoadTask kafkaRoutineLoadTask = new KafkaRoutineLoadTask(kafkaRoutineLoadJob1.getResourceInfo(), - beId, kafkaRoutineLoadJob1.getDbId(), kafkaRoutineLoadJob1.getTableId(), - "", 0L, partitionIdToOffset); -// -// new Expectations() { -// { -// routineLoadTaskInfo1.createRoutineLoadTask(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.increaseNumOfConcurrentTasksByBeId(beId); -// times = 1; - } - }; } } 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..c7e7609dc78650 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); @@ -535,7 +535,7 @@ 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.expectNew(SqlParser.class, EasyMock.isA(SqlScanner.class), EasyMock.anyString()).andReturn(parser); PowerMock.replay(SqlParser.class); StmtExecutor executor = new StmtExecutor(ctx, ""); 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 3a484a73092e54..30b37cfbcd1881 100644 --- a/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -316,11 +316,11 @@ public void testCommitRoutineLoadTransaction(@Injectable TabletCommitInfo tablet List tabletCommitInfoList = new ArrayList<>(); tabletCommitInfoList.add(tabletCommitInfo); - KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob("test", 1L, 1L, "host:port", "topic"); + KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(1L, "test", 1L, 1L, "host:port", "topic"); List routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); routineLoadTaskInfoList.add(routineLoadTaskInfo); TransactionState transactionState = new TransactionState(1L, 1L, "label", 1L, LoadJobSourceType.ROUTINE_LOAD_TASK, "be1"); - transactionState.setTransactionStatus(TransactionStatus.PREPARE); + transactionState.setTransactionStatus(TransactionStatus.PREPARE, false); Deencapsulation.setField(transactionState, "txnStateChangeListener", routineLoadJob); Map idToTransactionState = Maps.newHashMap(); idToTransactionState.put(1L, transactionState); @@ -389,11 +389,11 @@ public void testCommitRoutineLoadTransactionWithErrorMax(@Injectable TabletCommi List tabletCommitInfoList = new ArrayList<>(); tabletCommitInfoList.add(tabletCommitInfo); - KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob("test", 1L, 1L, "host:port", "topic"); + KafkaRoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(1L, "test", 1L, 1L, "host:port", "topic"); List routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); routineLoadTaskInfoList.add(routineLoadTaskInfo); TransactionState transactionState = new TransactionState(1L, 1L, "label", 1L, LoadJobSourceType.ROUTINE_LOAD_TASK, "be1"); - transactionState.setTransactionStatus(TransactionStatus.PREPARE); + transactionState.setTransactionStatus(TransactionStatus.PREPARE, false); Deencapsulation.setField(transactionState, "txnStateChangeListener", routineLoadJob); Map idToTransactionState = Maps.newHashMap(); idToTransactionState.put(1L, transactionState); diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index 96f688512331c3..5a197f9d9ae900 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -67,9 +67,6 @@ struct TKafkaLoadInfo { 1: required string brokers; 2: required string topic; 3: required map partition_begin_offset; - 4: optional i64 max_interval_s; - 5: optional i64 max_batch_rows; - 6: optional i64 max_batch_size; } struct TRoutineLoadTask { @@ -81,8 +78,11 @@ struct TRoutineLoadTask { 6: optional string db 7: optional string tbl 8: optional string label - 9: optional TKafkaLoadInfo kafka_load_info - 10: optional PaloInternalService.TExecPlanFragmentParams params + 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 { @@ -124,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) From 27b42c1a559a4ff5e54b9d291b23a6da3044bb0f Mon Sep 17 00:00:00 2001 From: morningman Date: Thu, 14 Mar 2019 19:42:05 +0800 Subject: [PATCH 2/2] fix bug --- .../runtime/stream_load/stream_load_context.h | 12 ---- .../doris/persist/RoutineLoadOperation.java | 70 +++++++++++++++++++ .../transaction/TxnStateChangeListener.java | 59 ++++++++++++++++ .../transaction/TxnStateListenerRegistry.java | 48 +++++++++++++ 4 files changed, 177 insertions(+), 12 deletions(-) create mode 100644 fe/src/main/java/org/apache/doris/persist/RoutineLoadOperation.java create mode 100644 fe/src/main/java/org/apache/doris/transaction/TxnStateChangeListener.java create mode 100644 fe/src/main/java/org/apache/doris/transaction/TxnStateListenerRegistry.java diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index b11bd6412dbbf6..5c320e84cd090d 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -118,18 +118,6 @@ class StreamLoadContext { 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 = 1024; - int64_t max_batch_size = 100 * 1024 * 1024; // 100MB - - // 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 - // 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; 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..395e2b655cf8e8 --- /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 { + in.readLong(); + jobState = JobState.valueOf(Text.readString(in)); + } +} diff --git a/fe/src/main/java/org/apache/doris/transaction/TxnStateChangeListener.java b/fe/src/main/java/org/apache/doris/transaction/TxnStateChangeListener.java new file mode 100644 index 00000000000000..3c80ddd7d263e2 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/transaction/TxnStateChangeListener.java @@ -0,0 +1,59 @@ +// 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; + +public abstract class TxnStateChangeListener { + + protected long id; + + public long getId() { + return id; + } + + public TxnStateChangeListener(long id) { + this.id = id; + } + + public abstract void beforeCommitted(TransactionState txnState) throws TransactionException; + + /** + * update catalog of job which has related txn after transaction has been committed + * + * @param txnState + */ + public abstract void onCommitted(TransactionState txnState, boolean isReplay) throws TransactionException; + + /** + * 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 + */ + public abstract void beforeAborted(TransactionState txnState, String txnStatusChangeReason) + throws AbortTransactionException; + + /** + * this interface is executed when transaction has been aborted + * + * @param txnState + * @param txnStatusChangeReason maybe null + */ + public abstract void onAborted(TransactionState txnState, String txnStatusChangeReason, boolean isReplay); +} diff --git a/fe/src/main/java/org/apache/doris/transaction/TxnStateListenerRegistry.java b/fe/src/main/java/org/apache/doris/transaction/TxnStateListenerRegistry.java new file mode 100644 index 00000000000000..d2529e76c03a82 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/transaction/TxnStateListenerRegistry.java @@ -0,0 +1,48 @@ +// 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 java.util.Map; + +/* + * Author: Chenmingyu + * Date: Mar 14, 2019 + */ + +// saves all TxnStateChangeListeners +public class TxnStateListenerRegistry { + private Map listeners = Maps.newHashMap(); + + public synchronized boolean register(TxnStateChangeListener listener) { + if (listeners.containsKey(listener.getId())) { + return false; + } + listeners.put(listener.getId(), listener); + return true; + } + + public synchronized void unregister(long id) { + listeners.remove(id); + } + + public synchronized TxnStateChangeListener getListener(long id) { + return listeners.get(id); + } +}