This is an automated email from the ASF dual-hosted git repository.
dataroaring pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new 026b3fe5c73 [opt](routine load) optimize routine load timeout logic
(#40818)
026b3fe5c73 is described below
commit 026b3fe5c7331ca1a1d1c56765d9442819dfee2a
Author: hui lai <[email protected]>
AuthorDate: Thu Sep 19 14:08:43 2024 +0800
[opt](routine load) optimize routine load timeout logic (#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 +++
.../org/apache/doris/common/InternalErrorCode.java | 1 -
.../load/routineload/KafkaRoutineLoadJob.java | 3 +-
.../doris/load/routineload/KafkaTaskInfo.java | 13 ++-----
.../doris/load/routineload/RoutineLoadJob.java | 12 ------
.../load/routineload/RoutineLoadTaskInfo.java | 45 +++-------------------
.../load/routineload/KafkaRoutineLoadJobTest.java | 2 +-
.../routineload/RoutineLoadTaskSchedulerTest.java | 2 +-
.../transaction/GlobalTransactionMgrTest.java | 4 +-
9 files changed, 21 insertions(+), 67 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 dc7cea8bb85..71bb0e396f1 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
@@ -1204,6 +1204,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/common/InternalErrorCode.java
b/fe/fe-core/src/main/java/org/apache/doris/common/InternalErrorCode.java
index 4382a7ea06e..73808e2354e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/InternalErrorCode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/InternalErrorCode.java
@@ -36,7 +36,6 @@ public enum InternalErrorCode {
CREATE_TASKS_ERR(103),
TASKS_ABORT_ERR(104),
CANNOT_RESUME_ERR(105),
- TIMEOUT_TOO_MUCH(106),
// for external catalog
GET_REMOTE_DATA_ERROR(202),
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 1762f8d1122..6bdef3301a6 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
@@ -234,7 +234,8 @@ public class KafkaRoutineLoadJob extends RoutineLoadJob {
((KafkaProgress)
progress).getOffsetByPartition(kafkaPartition));
}
KafkaTaskInfo kafkaTaskInfo = new
KafkaTaskInfo(UUID.randomUUID(), id,
- 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 7aa9ebda09f..f1578269529 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
@@ -49,16 +49,14 @@ public class KafkaTaskInfo extends RoutineLoadTaskInfo {
private Map<Integer, Long> partitionIdToOffset;
public KafkaTaskInfo(UUID id, long jobId,
- long timeoutMs, int timeoutBackOffCount,
- Map<Integer, Long> partitionIdToOffset, boolean
isMultiTable) {
- super(id, jobId, timeoutMs, timeoutBackOffCount, isMultiTable);
+ long timeoutMs, Map<Integer, Long>
partitionIdToOffset, boolean isMultiTable) {
+ super(id, jobId, timeoutMs, isMultiTable);
this.partitionIdToOffset = partitionIdToOffset;
}
public KafkaTaskInfo(KafkaTaskInfo kafkaTaskInfo, Map<Integer, Long>
partitionIdToOffset, boolean isMultiTable) {
super(UUID.randomUUID(), kafkaTaskInfo.getJobId(),
- kafkaTaskInfo.getTimeoutMs(),
kafkaTaskInfo.getTimeoutBackOffCount(),
- kafkaTaskInfo.getBeId(), isMultiTable);
+ kafkaTaskInfo.getTimeoutMs(), kafkaTaskInfo.getBeId(),
isMultiTable);
this.partitionIdToOffset = partitionIdToOffset;
this.isEof = kafkaTaskInfo.getIsEof();
}
@@ -137,11 +135,6 @@ public class KafkaTaskInfo extends RoutineLoadTaskInfo {
TPipelineFragmentParams tExecPlanFragmentParams =
routineLoadJob.plan(planner, loadId, txnId);
TPlanFragment tPlanFragment = tExecPlanFragmentParams.getFragment();
tPlanFragment.getOutputSink().getOlapTableSink().setTxnId(txnId);
- // it needs update timeout to make task timeout backoff work
- long timeoutS = this.getTimeoutMs() / 1000;
-
tPlanFragment.getOutputSink().getOlapTableSink().setLoadChannelTimeoutS(timeoutS);
- tExecPlanFragmentParams.getQueryOptions().setQueryTimeout((int)
timeoutS);
- tExecPlanFragmentParams.getQueryOptions().setExecutionTimeout((int)
timeoutS);
if (Config.enable_workload_group) {
long wgId = routineLoadJob.getWorkloadId();
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 ac4a548c62f..42032ebfc96 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
@@ -781,18 +781,6 @@ public abstract class RoutineLoadJob
// 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 301efe4d9c9..1ff825d97b9 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
@@ -73,28 +73,23 @@ 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, long timeoutMs,
- int timeoutBackOffCount, boolean isMultiTable) {
+ public RoutineLoadTaskInfo(UUID id, long jobId, long timeoutMs, boolean
isMultiTable) {
this.id = id;
this.jobId = jobId;
this.createTimeMs = System.currentTimeMillis();
this.timeoutMs = timeoutMs;
- this.timeoutBackOffCount = timeoutBackOffCount;
this.isMultiTable = isMultiTable;
}
- public RoutineLoadTaskInfo(UUID id, long jobId, long timeoutMs, int
timeoutBackOffCount,
- long previousBeId, boolean isMultiTable) {
- this(id, jobId, timeoutMs, timeoutBackOffCount, isMultiTable);
+ public RoutineLoadTaskInfo(UUID id, long jobId, long timeoutMs, long
previousBeId,
+ boolean isMultiTable) {
+ this(id, jobId, timeoutMs, isMultiTable);
this.previousBeId = previousBeId;
}
@@ -138,10 +133,6 @@ public abstract class RoutineLoadTaskInfo {
this.lastScheduledTime = lastScheduledTime;
}
- public void setTimeoutMs(long timeoutMs) {
- this.timeoutMs = timeoutMs;
- }
-
public long getTimeoutMs() {
return timeoutMs;
}
@@ -154,14 +145,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;
}
@@ -173,33 +156,17 @@ public abstract class RoutineLoadTaskInfo {
}
if (isRunning() && System.currentTimeMillis() - executeStartTimeMs >
timeoutMs) {
- LOG.info("task {} is timeout. start: {}, timeout: {},
timeoutBackOffCount: {}", DebugUtil.printId(id),
- executeStartTimeMs, timeoutMs, timeoutBackOffCount);
+ LOG.info("task {} is timeout. start: {}, timeout: {}",
DebugUtil.printId(id),
+ executeStartTimeMs, timeoutMs);
return true;
}
return false;
}
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 c18682bdc3c..20cb626ff37 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
@@ -225,7 +225,7 @@ public class KafkaRoutineLoadJobTest {
Map<Integer, Long> partitionIdsToOffset = Maps.newHashMap();
partitionIdsToOffset.put(100, 0L);
KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(new UUID(1, 1), 1L,
- 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 1548017b661..95c2423de71 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
@@ -69,7 +69,7 @@ public class RoutineLoadTaskSchedulerTest {
Deencapsulation.setField(kafkaProgress, "partitionIdToOffset",
partitionIdToOffset);
LinkedBlockingDeque<RoutineLoadTaskInfo> routineLoadTaskInfoQueue =
new LinkedBlockingDeque<>();
- KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo(new UUID(1, 1),
1L, 20000, 0,
+ KafkaTaskInfo routineLoadTaskInfo1 = new KafkaTaskInfo(new UUID(1, 1),
1L, 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 03a29fb453f..0c3acc9ceee 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
@@ -301,7 +301,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, 20000, 0,
+ KafkaTaskInfo routineLoadTaskInfo = new
KafkaTaskInfo(UUID.randomUUID(), 1L, 20000,
partitionIdToOffset, false);
Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L);
routineLoadTaskInfoList.add(routineLoadTaskInfo);
@@ -367,7 +367,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, 20000, 0,
+ KafkaTaskInfo routineLoadTaskInfo = new
KafkaTaskInfo(UUID.randomUUID(), 1L, 20000,
partitionIdToOffset, false);
Deencapsulation.setField(routineLoadTaskInfo, "txnId", 1L);
routineLoadTaskInfoList.add(routineLoadTaskInfo);
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]