This is an automated email from the ASF dual-hosted git repository. dataroaring pushed a commit to branch branch-2.0 in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-2.0 by this push: new d45e2194e3b [opt](routine load) optimize routine load timeout logic (#40818) (#41137) d45e2194e3b is described below commit d45e2194e3b7118fbcde0c6825183ae265c7b5ba Author: hui lai <1353307...@qq.com> AuthorDate: Thu Sep 26 11:47:54 2024 +0800 [opt](routine load) optimize routine load timeout logic (#40818) (#41137) pick (#40818) If IO/CPU resources are tight, routine load task is likely to timeout. The current method is self-adaption backoff https://github.com/apache/doris/pull/32227, but the problem is it will do some ineffective work to match proper timeout. For one routine load task, a better way to handle task is finishing executing instead of retry when resources are tight. Therefore, this pr increase timeout to make "task always finish even if it is slow when resources are tight". --- .../main/java/org/apache/doris/common/Config.java | 6 ++++ .../load/routineload/KafkaRoutineLoadJob.java | 3 +- .../doris/load/routineload/KafkaTaskInfo.java | 8 ++--- .../doris/load/routineload/RoutineLoadJob.java | 12 ------- .../load/routineload/RoutineLoadTaskInfo.java | 42 +++------------------- .../load/routineload/KafkaRoutineLoadJobTest.java | 2 +- .../routineload/RoutineLoadTaskSchedulerTest.java | 2 +- .../transaction/GlobalTransactionMgrTest.java | 4 +-- 8 files changed, 19 insertions(+), 60 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index ffc23900a0f..2af297c2a59 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1145,6 +1145,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static int max_routine_load_task_num_per_be = 1024; + /** + * routine load timeout is equal to maxBatchIntervalS * routine_load_task_timeout_multiplier. + */ + @ConfField(mutable = true, masterOnly = true) + public static int routine_load_task_timeout_multiplier = 10; + /** * the max timeout of get kafka meta. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java index 8fedd791b1e..488ae5b429a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java @@ -227,7 +227,8 @@ public class KafkaRoutineLoadJob extends RoutineLoadJob { ((KafkaProgress) progress).getOffsetByPartition(kafkaPartition)); } KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), id, clusterName, - maxBatchIntervalS * 2 * 1000, 0, taskKafkaProgress, isMultiTable()); + maxBatchIntervalS * Config.routine_load_task_timeout_multiplier * 1000, + taskKafkaProgress, isMultiTable()); routineLoadTaskInfoList.add(kafkaTaskInfo); result.add(kafkaTaskInfo); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java index dbddc695a68..476a74836f4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java @@ -47,16 +47,14 @@ public class KafkaTaskInfo extends RoutineLoadTaskInfo { private Map<Integer, Long> partitionIdToOffset; public KafkaTaskInfo(UUID id, long jobId, String clusterName, - long timeoutMs, int timeoutBackOffCount, - Map<Integer, Long> partitionIdToOffset, boolean isMultiTable) { - super(id, jobId, clusterName, timeoutMs, timeoutBackOffCount, isMultiTable); + long timeoutMs, Map<Integer, Long> partitionIdToOffset, boolean isMultiTable) { + super(id, jobId, clusterName, timeoutMs, isMultiTable); this.partitionIdToOffset = partitionIdToOffset; } public KafkaTaskInfo(KafkaTaskInfo kafkaTaskInfo, Map<Integer, Long> partitionIdToOffset, boolean isMultiTable) { super(UUID.randomUUID(), kafkaTaskInfo.getJobId(), kafkaTaskInfo.getClusterName(), - kafkaTaskInfo.getTimeoutMs(), kafkaTaskInfo.getTimeoutBackOffCount(), - kafkaTaskInfo.getBeId(), isMultiTable); + kafkaTaskInfo.getTimeoutMs(), kafkaTaskInfo.getBeId(), isMultiTable); this.partitionIdToOffset = partitionIdToOffset; this.isEof = kafkaTaskInfo.getIsEof(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index 8892ac79b02..b4ef59206ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -715,18 +715,6 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl // and after renew, the previous task is removed from routineLoadTaskInfoList, // so task can no longer be committed successfully. // the already committed task will not be handled here. - int timeoutBackOffCount = routineLoadTaskInfo.getTimeoutBackOffCount(); - if (timeoutBackOffCount > RoutineLoadTaskInfo.MAX_TIMEOUT_BACK_OFF_COUNT) { - try { - updateState(JobState.PAUSED, new ErrorReason(InternalErrorCode.TIMEOUT_TOO_MUCH, - "task " + routineLoadTaskInfo.getId() + " timeout too much"), false); - } catch (UserException e) { - LOG.warn("update job state to pause failed", e); - } - return; - } - routineLoadTaskInfo.setTimeoutBackOffCount(timeoutBackOffCount + 1); - routineLoadTaskInfo.setTimeoutMs((routineLoadTaskInfo.getTimeoutMs() << 1)); RoutineLoadTaskInfo newTask = unprotectRenewTask(routineLoadTaskInfo); Env.getCurrentEnv().getRoutineLoadTaskScheduler().addTaskInQueue(newTask); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java index 69c07507487..d8ecfe1abe0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java @@ -74,30 +74,24 @@ public abstract class RoutineLoadTaskInfo { protected boolean isMultiTable = false; - protected static final int MAX_TIMEOUT_BACK_OFF_COUNT = 3; - protected int timeoutBackOffCount = 0; - protected boolean isEof = false; // this status will be set when corresponding transaction's status is changed. // so that user or other logic can know the status of the corresponding txn. protected TransactionStatus txnStatus = TransactionStatus.UNKNOWN; - - public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, - int timeoutBackOffCount, boolean isMultiTable) { + public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, boolean isMultiTable) { this.id = id; this.jobId = jobId; this.clusterName = clusterName; this.createTimeMs = System.currentTimeMillis(); this.timeoutMs = timeoutMs; - this.timeoutBackOffCount = timeoutBackOffCount; this.isMultiTable = isMultiTable; } - public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, - int timeoutBackOffCount, long previousBeId, boolean isMultiTable) { - this(id, jobId, clusterName, timeoutMs, timeoutBackOffCount, isMultiTable); + public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName, long timeoutMs, long previousBeId, + boolean isMultiTable) { + this(id, jobId, clusterName, timeoutMs, isMultiTable); this.previousBeId = previousBeId; } @@ -145,10 +139,6 @@ public abstract class RoutineLoadTaskInfo { this.lastScheduledTime = lastScheduledTime; } - public void setTimeoutMs(long timeoutMs) { - this.timeoutMs = timeoutMs; - } - public long getTimeoutMs() { return timeoutMs; } @@ -161,14 +151,6 @@ public abstract class RoutineLoadTaskInfo { return txnStatus; } - public void setTimeoutBackOffCount(int timeoutBackOffCount) { - this.timeoutBackOffCount = timeoutBackOffCount; - } - - public int getTimeoutBackOffCount() { - return timeoutBackOffCount; - } - public boolean getIsEof() { return isEof; } @@ -188,25 +170,9 @@ public abstract class RoutineLoadTaskInfo { } public void handleTaskByTxnCommitAttachment(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { - selfAdaptTimeout(rlTaskTxnCommitAttachment); judgeEof(rlTaskTxnCommitAttachment); } - private void selfAdaptTimeout(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { - long taskExecutionTime = rlTaskTxnCommitAttachment.getTaskExecutionTimeMs(); - long timeoutMs = this.timeoutMs; - - while (this.timeoutBackOffCount > 0) { - timeoutMs = timeoutMs >> 1; - if (timeoutMs <= taskExecutionTime) { - this.timeoutMs = timeoutMs << 1; - return; - } - this.timeoutBackOffCount--; - } - this.timeoutMs = timeoutMs; - } - private void judgeEof(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { RoutineLoadJob routineLoadJob = routineLoadManager.getJob(jobId); if (rlTaskTxnCommitAttachment.getTotalRows() < routineLoadJob.getMaxBatchRows() diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java index 842d4d4003f..2df0afd0d36 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java @@ -227,7 +227,7 @@ public class KafkaRoutineLoadJobTest { Map<Integer, Long> partitionIdsToOffset = Maps.newHashMap(); partitionIdsToOffset.put(100, 0L); KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(new UUID(1, 1), 1L, "default_cluster", - maxBatchIntervalS * 2 * 1000, 0, partitionIdsToOffset, false); + maxBatchIntervalS * 2 * 1000, partitionIdsToOffset, false); kafkaTaskInfo.setExecuteStartTimeMs(System.currentTimeMillis() - maxBatchIntervalS * 2 * 1000 - 1); routineLoadTaskInfoList.add(kafkaTaskInfo); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java index 0ce694bfb11..5a0ee0e5dfa 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java @@ -68,7 +68,7 @@ public class RoutineLoadTaskSchedulerTest { Deencapsulation.setField(kafkaProgress, "partitionIdToOffset", partitionIdToOffset); LinkedBlockingDeque<RoutineLoadTaskInfo> routineLoadTaskInfoQueue = new LinkedBlockingDeque<>(); - KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo(new UUID(1, 1), 1L, "default_cluster", 20000, 0, + KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo(new UUID(1, 1), 1L, "default_cluster", 20000, partitionIdToOffset, false); routineLoadTaskInfoQueue.addFirst(routineLoadTaskInfo1); diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index 09421d0849c..7337ef64532 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -322,7 +322,7 @@ public class GlobalTransactionMgrTest { List<RoutineLoadTaskInfo> routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); Map<Integer, Long> partitionIdToOffset = Maps.newHashMap(); partitionIdToOffset.put(1, 0L); - KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "default_cluster", 20000, 0, + KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "default_cluster", 20000, partitionIdToOffset, false); Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L); routineLoadTaskInfoList.add(routineLoadTaskInfo); @@ -396,7 +396,7 @@ public class GlobalTransactionMgrTest { List<RoutineLoadTaskInfo> routineLoadTaskInfoList = Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); Map<Integer, Long> partitionIdToOffset = Maps.newHashMap(); partitionIdToOffset.put(1, 0L); - KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "defualt_cluster", 20000, 0, + KafkaTaskInfo routineLoadTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), 1L, "defualt_cluster", 20000, partitionIdToOffset, false); Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L); routineLoadTaskInfoList.add(routineLoadTaskInfo); --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org