From eb6aa5291f4e168fe90988294c41044e9060ac22 Mon Sep 17 00:00:00 2001 From: hui lai Date: Tue, 9 Sep 2025 16:07:45 +0800 Subject: [PATCH 01/25] [feat](streaming job) introduce streaming job schedule task (#55790) ### What problem does this PR solve? introduce streaming job schedule task --- .../apache/doris/job/base/AbstractJob.java | 4 + .../apache/doris/job/common/JobStatus.java | 5 +- .../apache/doris/job/common/PauseReason.java | 67 +++++++++++ .../job/executor/DispatchTaskHandler.java | 10 +- .../insert/streaming/StreamingInsertJob.java | 106 ++++++++++++++++++ .../streaming/StreamingJobSchedulerTask.java | 93 +++++++++++++++ 6 files changed, 283 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/common/PauseReason.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index d5ebcf26341951..4db4292c04763e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -477,4 +477,8 @@ public void onReplayEnd(AbstractJob replayJob) throws JobException { public boolean needPersist() { return true; } + + public boolean isFinalStatus() { + return jobStatus.equals(JobStatus.STOPPED) || jobStatus.equals(JobStatus.FINISHED); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java index 94cb06db6200fe..25d207110d9cb3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java @@ -19,7 +19,10 @@ public enum JobStatus { - + /** + * For streaming job, if the task has not been created, the job will be in pending state. + */ + PENDING, /** * When the task is not started, the initial state will be triggered. * The initial state can be started diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/PauseReason.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/PauseReason.java new file mode 100644 index 00000000000000..49a46327b32756 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/PauseReason.java @@ -0,0 +1,67 @@ +// 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.job.common; + +import org.apache.doris.common.InternalErrorCode; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; + +import java.io.DataOutput; +import java.io.IOException; + +public class PauseReason implements Writable { + @SerializedName(value = "code") + private InternalErrorCode code; + @SerializedName(value = "msg") + private String msg; + + public PauseReason(InternalErrorCode errCode, String msg) { + this.code = errCode; + this.msg = msg; + } + + public InternalErrorCode getCode() { + return code; + } + + public void setCode(InternalErrorCode code) { + this.code = code; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } + + @Override + public String toString() { + return "ErrorReason{" + "code=" + code + ", msg='" + msg + '\'' + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java index 56222fd3e1fe60..77078896735a9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java @@ -18,6 +18,7 @@ package org.apache.doris.job.executor; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.TaskType; @@ -55,7 +56,7 @@ public void onEvent(TimerJobEvent event) { log.info("job is null,may be job is deleted, ignore"); return; } - if (event.getJob().isReadyForScheduling(null) && event.getJob().getJobStatus() == JobStatus.RUNNING) { + if (event.getJob().isReadyForScheduling(null) && checkStatus(event)) { List tasks = event.getJob().commonCreateTasks(TaskType.SCHEDULED, null); if (CollectionUtils.isEmpty(tasks)) { log.warn("job is ready for scheduling, but create task is empty, skip scheduler," @@ -77,4 +78,11 @@ public void onEvent(TimerJobEvent event) { log.warn("dispatch timer job error, task id is {}", event.getJob().getJobId(), e); } } + + private boolean checkStatus(TimerJobEvent event) { + if (event.getJob().getJobConfig().getExecuteType() == JobExecuteType.STREAMING) { + return !event.getJob().isFinalStatus(); + } + return event.getJob().getJobStatus() == JobStatus.RUNNING; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java new file mode 100644 index 00000000000000..9e5baccd13450c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -0,0 +1,106 @@ +// 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.job.extensions.insert.streaming; + +import org.apache.doris.common.io.Text; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.common.PauseReason; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import lombok.Getter; +import lombok.Setter; + +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class StreamingInsertJob extends AbstractJob> { + + @Getter + @SerializedName("st") + protected JobStatus status; + + @Getter + protected PauseReason pauseReason; + + @Getter + @Setter + protected long latestAutoResumeTimestamp; + + @Getter + @Setter + protected long autoResumeCount; + + @Override + public void updateJobStatus(JobStatus status) throws JobException { + super.updateJobStatus(status); + } + + protected void createStreamingInsertTask() { + } + + protected void fetchMeta() { + } + + @Override + public JobType getJobType() { + return JobType.INSERT; + } + + @Override + protected void checkJobParamsInternal() { + } + + @Override + public boolean isReadyForScheduling(Map taskContext) { + return true; + } + + @Override + public java.util.List createTasks(org.apache.doris.job.common.TaskType taskType, + Map taskContext) { + return java.util.Collections.emptyList(); + } + + @Override + public org.apache.doris.qe.ShowResultSetMetaData getTaskMetaData() { + return org.apache.doris.qe.ShowResultSetMetaData.builder().build(); + } + + @Override + public String formatMsgWhenExecuteQueueFull(Long taskId) { + return commonFormatMsgWhenExecuteQueueFull(taskId, "streaming_task_queue_size", + "job_streaming_task_consumer_thread_num"); + } + + @Override + public List queryTasks() { + return new ArrayList<>(); + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java new file mode 100644 index 00000000000000..51ab96b3b178a7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -0,0 +1,93 @@ +// 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.job.extensions.insert.streaming; + +import org.apache.doris.common.InternalErrorCode; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.common.PauseReason; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.thrift.TRow; + +public class StreamingJobSchedulerTask extends AbstractTask { + + private static final long BACK_OFF_BASIC_TIME_SEC = 10L; + private static final long MAX_BACK_OFF_TIME_SEC = 60 * 5; + + private StreamingInsertJob streamingInsertJob; + + public StreamingJobSchedulerTask(StreamingInsertJob streamingInsertJob) { + this.streamingInsertJob = streamingInsertJob; + } + + @Override + public void run() throws JobException { + switch (streamingInsertJob.getStatus()) { + case PENDING: + streamingInsertJob.createStreamingInsertTask(); + streamingInsertJob.updateJobStatus(JobStatus.RUNNING); + streamingInsertJob.setAutoResumeCount(0); + break; + case RUNNING: + streamingInsertJob.fetchMeta(); + break; + case PAUSED: + autoResumeHandler(); + break; + default: + break; + } + } + + private void autoResumeHandler() throws JobException { + final PauseReason pauseReason = streamingInsertJob.getPauseReason(); + final long latestAutoResumeTimestamp = streamingInsertJob.getLatestAutoResumeTimestamp(); + final long autoResumeCount = streamingInsertJob.getAutoResumeCount(); + final long current = System.currentTimeMillis(); + + if (pauseReason != null + && pauseReason.getCode() != InternalErrorCode.MANUAL_PAUSE_ERR + && pauseReason.getCode() != InternalErrorCode.TOO_MANY_FAILURE_ROWS_ERR + && pauseReason.getCode() != InternalErrorCode.CANNOT_RESUME_ERR) { + long autoResumeIntervalTimeSec = autoResumeCount < 5 + ? Math.min((long) Math.pow(2, autoResumeCount) * BACK_OFF_BASIC_TIME_SEC, + MAX_BACK_OFF_TIME_SEC) : MAX_BACK_OFF_TIME_SEC; + if (current - latestAutoResumeTimestamp > autoResumeIntervalTimeSec * 1000L) { + streamingInsertJob.setLatestAutoResumeTimestamp(current); + if (autoResumeCount < Long.MAX_VALUE) { + streamingInsertJob.setAutoResumeCount(autoResumeCount + 1); + } + streamingInsertJob.updateJobStatus(JobStatus.RUNNING); + return; + } + } + } + + @Override + protected void closeOrReleaseResources() { + } + + @Override + protected void executeCancelLogic(boolean needWaitCancelComplete) throws Exception { + } + + @Override + public TRow getTvfInfo(String jobName) { + return null; + } +} From a69c8fdcc2aad853ab3313a1e2fad50b95321e69 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 9 Sep 2025 17:16:09 +0800 Subject: [PATCH 02/25] [Feature](WIP) Add Create StreamingJob and Alter Job (#55814) ### What problem does this PR solve? 1. Add Create StreamingJob and Alter Job 2. add job and task tvf schema 3. add offset api --- .../org/apache/doris/nereids/DorisParser.g4 | 13 +- .../apache/doris/job/base/JobProperties.java | 25 +++ .../job/extensions/insert/InsertJob.java | 31 ++++ .../job/extensions/insert/InsertTask.java | 5 +- .../insert/streaming/StreamingInsertJob.java | 25 +++ .../streaming/StreamingJobProperties.java | 64 ++++++++ .../apache/doris/job/manager/JobManager.java | 23 +++ .../org/apache/doris/job/offset/Offset.java | 22 +++ .../job/offset/SourceOffsetProvider.java | 62 +++++++ .../offset/SourceOffsetProviderFactory.java | 37 +++++ .../apache/doris/job/offset/s3/S3Offset.java | 36 +++++ .../job/offset/s3/S3SourceOffsetProvider.java | 53 ++++++ .../nereids/parser/LogicalPlanBuilder.java | 14 +- .../doris/nereids/trees/plans/PlanType.java | 1 + .../trees/plans/commands/AlterJobCommand.java | 151 ++++++++++++++++++ .../plans/commands/info/CreateJobInfo.java | 64 +++++++- .../trees/plans/visitor/CommandVisitor.java | 5 + 17 files changed, 619 insertions(+), 12 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/base/JobProperties.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d8f5f810eb046c..a0ae7c6916e636 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -99,16 +99,19 @@ materializedViewStatement | SHOW CREATE MATERIALIZED VIEW mvName=multipartIdentifier #showCreateMTMV ; supportedJobStatement - : CREATE JOB label=multipartIdentifier ON SCHEDULE - ( + : CREATE JOB label=multipartIdentifier propertyClause? + ON (STREAMING | SCHEDULE( (EVERY timeInterval=INTEGER_VALUE timeUnit=identifier (STARTS (startTime=STRING_LITERAL | CURRENT_TIMESTAMP))? (ENDS endsTime=STRING_LITERAL)?) | - (AT (atTime=STRING_LITERAL | CURRENT_TIMESTAMP))) - commentSpec? - DO supportedDmlStatement #createScheduledJob + (AT (atTime=STRING_LITERAL | CURRENT_TIMESTAMP)) + ) + ) + commentSpec? + DO supportedDmlStatement #createScheduledJob | PAUSE JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #pauseJob + | ALTER JOB FOR (jobNameKey=identifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob | DROP JOB (IF EXISTS)? WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #dropJob | RESUME JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #resumeJob | CANCEL TASK WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) AND (taskIdKey=identifier) EQ (taskIdValue=INTEGER_VALUE) #cancelJobTask diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobProperties.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobProperties.java new file mode 100644 index 00000000000000..3985b59bf16379 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobProperties.java @@ -0,0 +1,25 @@ +// 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.job.base; + +import org.apache.doris.common.AnalysisException; + +public interface JobProperties { + default void validate() throws AnalysisException { + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index dd0e0d3a228420..5b386886b19ce8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -51,6 +51,8 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TRow; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.ErrorTabletInfo; import org.apache.doris.transaction.TabletCommitInfo; @@ -96,6 +98,11 @@ public class InsertJob extends AbstractJob> impl .add(new Column("CreateTime", ScalarType.createStringType())) .addAll(COMMON_SCHEMA) .add(new Column("Comment", ScalarType.createStringType())) + // only execute type = streaming need record + .add(new Column("Progress", ScalarType.createStringType())) + .add(new Column("RemoteOffset", ScalarType.createStringType())) + .add(new Column("LoadStatistic", ScalarType.createStringType())) + .add(new Column("ErrorMsg", ScalarType.createStringType())) .build(); private static final ShowResultSetMetaData TASK_META_DATA = @@ -112,6 +119,8 @@ public class InsertJob extends AbstractJob> impl .addColumn(new Column("TrackingUrl", ScalarType.createVarchar(200))) .addColumn(new Column("LoadStatistic", ScalarType.createVarchar(200))) .addColumn(new Column("User", ScalarType.createVarchar(50))) + // only execute type = streaming need record + .addColumn(new Column("Offset", ScalarType.createStringType())) .build(); public static final ImmutableMap COLUMN_TO_INDEX; @@ -523,6 +532,28 @@ public List getShowInfo() { } } + @Override + public TRow getTvfInfo() { + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getJobId()))); + trow.addToColumnValue(new TCell().setStringVal(getJobName())); + trow.addToColumnValue(new TCell().setStringVal(getCreateUser().getQualifiedUser())); + trow.addToColumnValue(new TCell().setStringVal(getJobConfig().getExecuteType().name())); + trow.addToColumnValue(new TCell().setStringVal(getJobConfig().convertRecurringStrategyToString())); + trow.addToColumnValue(new TCell().setStringVal(getJobStatus().name())); + trow.addToColumnValue(new TCell().setStringVal(getExecuteSql())); + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getSucceedTaskCount().get()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getFailedTaskCount().get()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getCanceledTaskCount().get()))); + trow.addToColumnValue(new TCell().setStringVal(getComment())); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(loadStatistic.toJson())); + trow.addToColumnValue(new TCell().setStringVal(failMsg == null ? FeConstants.null_string : failMsg.getMsg())); + return trow; + } + @Override public String formatMsgWhenExecuteQueueFull(Long taskId) { return commonFormatMsgWhenExecuteQueueFull(taskId, "insert_task_queue_size", diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index f6633d6c5e8e6e..34cfdf6edea874 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -67,7 +67,8 @@ public class InsertTask extends AbstractTask { new Column("FinishTime", ScalarType.createStringType()), new Column("TrackingUrl", ScalarType.createStringType()), new Column("LoadStatistic", ScalarType.createStringType()), - new Column("User", ScalarType.createStringType())); + new Column("User", ScalarType.createStringType()), + new Column("Offset", ScalarType.createStringType())); public static final ImmutableMap COLUMN_TO_INDEX; @@ -272,6 +273,7 @@ public TRow getTvfInfo(String jobName) { } else { trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); } + trow.addToColumnValue(new TCell().setStringVal("")); return trow; } @@ -292,6 +294,7 @@ private TRow getPendingTaskTVFInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); + trow.addToColumnValue(new TCell().setStringVal("")); return trow; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 9e5baccd13450c..a2016c52cb9371 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -17,13 +17,16 @@ package org.apache.doris.job.extensions.insert.streaming; +import org.apache.doris.analysis.UserIdentity; import org.apache.doris.common.io.Text; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.PauseReason; import org.apache.doris.job.exception.JobException; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.ConnectContext; import com.google.gson.annotations.SerializedName; import lombok.Getter; @@ -37,6 +40,8 @@ public class StreamingInsertJob extends AbstractJob> { + @SerializedName("did") + private final long dbId; @Getter @SerializedName("st") protected JobStatus status; @@ -52,6 +57,26 @@ public class StreamingInsertJob extends AbstractJob properties; + private long maxIntervalSecond; + private long s3BatchFiles; + private long s3BatchSize; + + public StreamingJobProperties(Map jobProperties) { + this.properties = jobProperties; + } + + @Override + public void validate() throws AnalysisException { + this.maxIntervalSecond = Util.getLongPropertyOrDefault( + properties.get(StreamingJobProperties.MAX_INTERVAL_SECOND_PROPERTY), + StreamingJobProperties.DEFAULT_MAX_INTERVAL_SECOND, (v) -> v >= 1, + StreamingJobProperties.MAX_INTERVAL_SECOND_PROPERTY + " should > 1"); + + this.s3BatchFiles = Util.getLongPropertyOrDefault( + properties.get(StreamingJobProperties.S3_BATCH_FILES_PROPERTY), + StreamingJobProperties.DEFAULT_S3_BATCH_FILES, (v) -> v >= 1, + StreamingJobProperties.S3_BATCH_FILES_PROPERTY + " should >=1 "); + + this.s3BatchSize = Util.getLongPropertyOrDefault(properties.get(StreamingJobProperties.S3_BATCH_SIZE_PROPERTY), + StreamingJobProperties.DEFAULT_S3_BATCH_SIZE, (v) -> v >= 100 * 1024 * 1024 + && v <= (long) (1024 * 1024 * 1024) * 10, + StreamingJobProperties.S3_BATCH_SIZE_PROPERTY + " should between 100MB and 10GB"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 3d705b945e7dbd..e763f8de590153 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -32,6 +32,7 @@ import org.apache.doris.common.util.LogKey; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.TaskType; @@ -219,6 +220,17 @@ public void alterJobStatus(Long jobId, JobStatus status) throws JobException { jobMap.get(jobId).logUpdateOperation(); } + public void alterJob(T job) { + writeLock(); + try { + jobMap.put(job.getJobId(), job); + job.logUpdateOperation(); + } finally { + writeUnlock(); + } + log.info("update job success, jobId: {}", job.getJobId()); + } + public void alterJobStatus(String jobName, JobStatus jobStatus) throws JobException { for (T a : jobMap.values()) { if (a.getJobName().equals(jobName)) { @@ -349,6 +361,9 @@ public void replayDeleteJob(T replayJob) throws JobException { */ public void cancelTaskById(String jobName, Long taskId) throws JobException { for (T job : jobMap.values()) { + if (job.getJobConfig().getExecuteType().equals(JobExecuteType.STREAMING)) { + throw new JobException("streaming job not support cancel task by id"); + } if (job.getJobName().equals(jobName)) { job.cancelTaskById(taskId); job.logUpdateOperation(); @@ -392,6 +407,14 @@ public T getJob(Long jobId) { return jobMap.get(jobId); } + public T getJobByName(String jobName) throws JobException { + for (T a : jobMap.values()) { + if (a.getJobName().equals(jobName)) { + return a; + } + } + throw new JobException("job not exist, jobName:" + jobName); + } /** * get load info by db diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java new file mode 100644 index 00000000000000..095f0a5e6bf2bc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java @@ -0,0 +1,22 @@ +// 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.job.offset; + +public interface Offset { + String toJson(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java new file mode 100644 index 00000000000000..f88079617de532 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -0,0 +1,62 @@ +// 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.job.offset; + +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; + +/** + * Interface for managing offsets and metadata of a data source. + */ +public interface SourceOffsetProvider { + /** + * Get source type, e.g. s3, kafka + * @return + */ + String getSourceType(); + + /** + * Get next offset to consume + * @return + */ + Offset getNextOffset(); + + /** + * Rewrite the TVF parameters in the InsertIntoTableCommand based on the current offset. + * @param command + * @return + */ + InsertIntoTableCommand rewriteTvfParamsInCommand(InsertIntoTableCommand command); + + /** + * Update the progress of the source. + * @param offset + */ + void updateProgress(Offset offset); + + /** + * Fetch remote meta information, such as listing files in S3 or getting latest offsets in Kafka. + */ + void fetchRemoteMeta(); + + /** + * Whether there is more data to consume + * @return + */ + boolean hasMoreData(); +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java new file mode 100644 index 00000000000000..5ba1d903d78135 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java @@ -0,0 +1,37 @@ +// 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.job.offset; + +import org.apache.doris.job.offset.s3.S3SourceOffsetProvider; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class SourceOffsetProviderFactory { + private static final Map> map = new ConcurrentHashMap<>(); + + static { + map.put("s3", S3SourceOffsetProvider.class); + } + + public static SourceOffsetProvider createSourceOffsetProvider(String sourceType) throws InstantiationException, + IllegalAccessException { + Class cla = map.get(sourceType.toUpperCase()); + return cla.newInstance(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java new file mode 100644 index 00000000000000..86ff467796af8d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -0,0 +1,36 @@ +// 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.job.offset.s3; + +import org.apache.doris.job.offset.Offset; +import org.apache.doris.persist.gson.GsonUtils; + +import java.util.List; + +public class S3Offset implements Offset { + String startFile; + + String endFile; + + List fileLists; + + @Override + public String toJson() { + return GsonUtils.GSON.toJson(this); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java new file mode 100644 index 00000000000000..087d9c2beb7685 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -0,0 +1,53 @@ +// 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.job.offset.s3; + +import org.apache.doris.job.offset.Offset; +import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; + +public class S3SourceOffsetProvider implements SourceOffsetProvider { + + @Override + public String getSourceType() { + return null; + } + + @Override + public Offset getNextOffset() { + return null; + } + + @Override + public InsertIntoTableCommand rewriteTvfParamsInCommand(InsertIntoTableCommand command) { + return null; + } + + @Override + public void updateProgress(Offset offset) { + } + + @Override + public void fetchRemoteMeta() { + } + + @Override + public boolean hasMoreData() { + return false; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index abb468a19ccb30..80a0886017ea6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -608,6 +608,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterColocateGroupCommand; import org.apache.doris.nereids.trees.plans.commands.AlterColumnStatsCommand; import org.apache.doris.nereids.trees.plans.commands.AlterDatabasePropertiesCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterJobCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterResourceCommand; import org.apache.doris.nereids.trees.plans.commands.AlterRoleCommand; @@ -1144,11 +1145,13 @@ public LogicalPlan visitCreateScheduledJob(DorisParser.CreateScheduledJobContext Optional interval = ctx.timeInterval == null ? Optional.empty() : Optional.of(Long.valueOf(ctx.timeInterval.getText())); Optional intervalUnit = ctx.timeUnit == null ? Optional.empty() : Optional.of(ctx.timeUnit.getText()); + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); String comment = visitCommentSpec(ctx.commentSpec()); String executeSql = getOriginSql(ctx.supportedDmlStatement()); CreateJobInfo createJobInfo = new CreateJobInfo(label, atTime, interval, intervalUnit, startTime, - endsTime, immediateStartOptional, comment, executeSql); + endsTime, immediateStartOptional, comment, executeSql, ctx.STREAMING() != null, properties); return new CreateJobCommand(createJobInfo); } @@ -1158,6 +1161,15 @@ private void checkJobNameKey(String key, String keyFormat, DorisParser.Supported } } + @Override + public LogicalPlan visitAlterJob(DorisParser.AlterJobContext ctx) { + checkJobNameKey(stripQuotes(ctx.jobNameKey.getText()), JOB_NAME, ctx); + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + String executeSql = getOriginSql(ctx.supportedDmlStatement()); + return new AlterJobCommand(stripQuotes(ctx.jobNameKey.getText()), properties, executeSql); + } + @Override public LogicalPlan visitPauseJob(DorisParser.PauseJobContext ctx) { checkJobNameKey(stripQuotes(ctx.jobNameKey.getText()), JOB_NAME, ctx); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index e150899f95cb79..367e6d641e04ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -162,6 +162,7 @@ public enum PlanType { CREATE_MTMV_COMMAND, CREATE_MATERIALIZED_VIEW_COMMAND, CREATE_JOB_COMMAND, + ALTER_JOB_COMMAND, PAUSE_JOB_COMMAND, CANCEL_JOB_COMMAND, DROP_CATALOG_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java new file mode 100644 index 00000000000000..ef8c9bb8b7a120 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecuteType; +import org.apache.doris.job.base.JobExecutionConfiguration; +import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.common.IntervalUnit; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * alter job command. + */ +public class AlterJobCommand extends AlterCommand implements ForwardWithSync { + // exclude job name prefix, which is used by inner job + private static final String excludeJobNamePrefix = "inner_"; + private String jobName; + private Map properties; + private String sql; + + public AlterJobCommand(String jobName, Map properties, String sql) { + super(PlanType.ALTER_JOB_COMMAND); + this.jobName = jobName; + this.properties = properties; + this.sql = sql; + } + + public String getJobName() { + return jobName; + } + + @Override + public StmtType stmtType() { + return StmtType.ALTER; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + + validate(); + AbstractJob job = analyzeAndBuildJobInfo(ctx); + ctx.getEnv().getJobManager().alterJob(job); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAlterJobCommand(this, context); + } + + private AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws JobException { + AbstractJob job = Env.getCurrentEnv().getJobManager().getJobByName(jobName); + if (job instanceof StreamingInsertJob) { + StreamingInsertJob originJob = (StreamingInsertJob) job; + String updateSQL = StringUtils.isEmpty(sql) ? originJob.getExecuteSql() : sql; + Map updateProps = properties == null || properties.isEmpty() ? originJob.getJobProperties() + .getProperties() : properties; + StreamingJobProperties streamJobProps = new StreamingJobProperties(updateProps); + // rebuild time definition + JobExecutionConfiguration execConfig = originJob.getJobConfig(); + TimerDefinition timerDefinition = new TimerDefinition(); + timerDefinition.setInterval(streamJobProps.getMaxIntervalSecond()); + timerDefinition.setIntervalUnit(IntervalUnit.SECOND); + timerDefinition.setStartTimeMs(execConfig.getTimerDefinition().getStartTimeMs()); + execConfig.setTimerDefinition(timerDefinition); + return new StreamingInsertJob(jobName, + job.getJobStatus(), + job.getCurrentDbName(), + job.getComment(), + ConnectContext.get().getCurrentUserIdentity(), + execConfig, + System.currentTimeMillis(), + updateSQL, + streamJobProps); + } else { + throw new JobException("Unsupported job type for ALTER:" + job.getJobType()); + } + } + + private void validate() throws Exception { + if (jobName.startsWith(excludeJobNamePrefix)) { + throw new AnalysisException("Can't alter inner job"); + } + AbstractJob job = Env.getCurrentEnv().getJobManager().getJobByName(jobName); + if (!JobStatus.PAUSED.equals(job.getJobStatus())) { + throw new AnalysisException("Only PAUSED job can be altered"); + } + + if (job.getJobConfig().getExecuteType().equals(JobExecuteType.STREAMING) + && job instanceof StreamingInsertJob) { + StreamingInsertJob streamingJob = (StreamingInsertJob) job; + boolean proCheck = checkProperties(streamingJob.getJobProperties().getProperties()); + boolean sqlCheck = checkSql(streamingJob.getExecuteSql()); + if (!proCheck && !sqlCheck) { + throw new AnalysisException("No properties or sql changed in ALTER JOB"); + } + } else { + throw new AnalysisException("Unsupported job type for ALTER:" + job.getJobType()); + } + } + + private boolean checkProperties(Map originProps) { + if (originProps.isEmpty()) { + return false; + } + if (!originProps.equals(properties)) { + return true; + } + return false; + } + + private boolean checkSql(String sql) { + if (sql == null || sql.isEmpty()) { + return false; + } + if (!sql.equals(sql)) { + return true; + } + return false; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 6cef7ee89ec960..0d52e23ece56ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -27,10 +27,13 @@ import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.base.JobExecutionConfiguration; +import org.apache.doris.job.base.JobProperties; import org.apache.doris.job.base.TimerDefinition; import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; @@ -39,6 +42,7 @@ import com.google.common.base.Strings; +import java.util.Map; import java.util.Optional; /** @@ -66,6 +70,8 @@ public class CreateJobInfo { private final String comment; private final String executeSql; + private final boolean streamingJob; + private final Map jobProperties; /** * Constructor for CreateJobInfo. @@ -83,7 +89,8 @@ public class CreateJobInfo { public CreateJobInfo(Optional labelNameOptional, Optional onceJobStartTimestampOptional, Optional intervalOptional, Optional intervalTimeUnitOptional, Optional startsTimeStampOptional, Optional endsTimeStampOptional, - Optional immediateStartOptional, String comment, String executeSql) { + Optional immediateStartOptional, String comment, String executeSql, + boolean streamingJob, Map jobProperties) { this.labelNameOptional = labelNameOptional; this.onceJobStartTimestampOptional = onceJobStartTimestampOptional; this.intervalOptional = intervalOptional; @@ -93,7 +100,8 @@ public CreateJobInfo(Optional labelNameOptional, Optional onceJo this.immediateStartOptional = immediateStartOptional; this.comment = comment; this.executeSql = executeSql; - + this.streamingJob = streamingJob; + this.jobProperties = jobProperties; } /** @@ -117,16 +125,32 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti // check its insert stmt,currently only support insert stmt JobExecutionConfiguration jobExecutionConfiguration = new JobExecutionConfiguration(); JobExecuteType executeType = intervalOptional.isPresent() ? JobExecuteType.RECURRING : JobExecuteType.ONE_TIME; + JobProperties properties = null; + if (streamingJob) { + executeType = JobExecuteType.STREAMING; + properties = new StreamingJobProperties(jobProperties); + } jobExecutionConfiguration.setExecuteType(executeType); - TimerDefinition timerDefinition = new TimerDefinition(); + TimerDefinition timerDefinition = new TimerDefinition(); if (executeType.equals(JobExecuteType.ONE_TIME)) { buildOnceJob(timerDefinition, jobExecutionConfiguration); + } else if (executeType.equals(JobExecuteType.STREAMING)) { + buildStreamingJob(timerDefinition, properties); } else { buildRecurringJob(timerDefinition, jobExecutionConfiguration); } jobExecutionConfiguration.setTimerDefinition(timerDefinition); - return analyzeAndCreateJob(executeSql, dbName, jobExecutionConfiguration); + return analyzeAndCreateJob(executeSql, dbName, jobExecutionConfiguration, properties); + } + + private void buildStreamingJob(TimerDefinition timerDefinition, JobProperties props) + throws AnalysisException { + StreamingJobProperties properties = (StreamingJobProperties) props; + timerDefinition.setInterval(properties.getMaxIntervalSecond()); + timerDefinition.setIntervalUnit(IntervalUnit.SECOND); + timerDefinition.setStartTimeMs(System.currentTimeMillis()); + properties.validate(); } /** @@ -210,7 +234,17 @@ protected static void checkAuth() throws AnalysisException { * @throws UserException if there is an error during SQL analysis or job creation */ private AbstractJob analyzeAndCreateJob(String sql, String currentDbName, - JobExecutionConfiguration jobExecutionConfiguration) throws UserException { + JobExecutionConfiguration jobExecutionConfiguration, + JobProperties properties) throws UserException { + if (jobExecutionConfiguration.getExecuteType().equals(JobExecuteType.STREAMING)) { + return analyzeAndCreateStreamingInsertJob(sql, currentDbName, jobExecutionConfiguration, properties); + } else { + return analyzeAndCreateInsertJob(sql, currentDbName, jobExecutionConfiguration); + } + } + + private AbstractJob analyzeAndCreateInsertJob(String sql, String currentDbName, + JobExecutionConfiguration jobExecutionConfiguration) throws UserException { NereidsParser parser = new NereidsParser(); LogicalPlan logicalPlan = parser.parseSingle(sql); if (logicalPlan instanceof InsertIntoTableCommand) { @@ -234,6 +268,26 @@ private AbstractJob analyzeAndCreateJob(String sql, String currentDbName, } } + private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String currentDbName, + JobExecutionConfiguration jobExecutionConfiguration, JobProperties properties) throws UserException { + NereidsParser parser = new NereidsParser(); + LogicalPlan logicalPlan = parser.parseSingle(sql); + if (logicalPlan instanceof InsertIntoTableCommand) { + return new StreamingInsertJob(labelNameOptional.get(), + JobStatus.RUNNING, + currentDbName, + comment, + ConnectContext.get().getCurrentUserIdentity(), + jobExecutionConfiguration, + System.currentTimeMillis(), + sql, + (StreamingJobProperties) properties); + } else { + throw new AnalysisException("Not support this sql : " + sql + " Command class is " + + logicalPlan.getClass().getName() + "."); + } + } + private void checkJobName(String jobName) throws AnalysisException { if (Strings.isNullOrEmpty(jobName)) { throw new AnalysisException("job name can not be null"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index a5f9a00be1d934..45abfa0c25064c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -37,6 +37,7 @@ import org.apache.doris.nereids.trees.plans.commands.AlterColocateGroupCommand; import org.apache.doris.nereids.trees.plans.commands.AlterColumnStatsCommand; import org.apache.doris.nereids.trees.plans.commands.AlterDatabasePropertiesCommand; +import org.apache.doris.nereids.trees.plans.commands.AlterJobCommand; import org.apache.doris.nereids.trees.plans.commands.AlterJobStatusCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; import org.apache.doris.nereids.trees.plans.commands.AlterResourceCommand; @@ -387,6 +388,10 @@ default R visitCreateJobCommand(CreateJobCommand createJobCommand, C context) { return visitCommand(createJobCommand, context); } + default R visitAlterJobCommand(AlterJobCommand alterJobCommand, C context) { + return visitCommand(alterJobCommand, context); + } + default R visitCreateFileCommand(CreateFileCommand createFileCommand, C context) { return visitCommand(createFileCommand, context); } From 47f2e1e7486552334c570f20069a706ac774544a Mon Sep 17 00:00:00 2001 From: hui lai Date: Wed, 10 Sep 2025 16:16:54 +0800 Subject: [PATCH 03/25] [feat](streaming job) introduce streaming task scheduler (#55857) ### What problem does this PR solve? Introduce streaming task scheduler to schedule all streaming tasks. --- .../java/org/apache/doris/common/Config.java | 5 + .../insert/streaming/StreamingInsertJob.java | 20 +++- .../insert/streaming/StreamingInsertTask.java | 36 ++++++ .../streaming/StreamingJobSchedulerTask.java | 2 +- .../apache/doris/job/manager/JobManager.java | 6 +- .../job/scheduler/StreamingTaskScheduler.java | 110 ++++++++++++++++++ 6 files changed, 173 insertions(+), 6 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java 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 e2c5b9eb8aec9f..7770318a5e928b 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 @@ -1955,6 +1955,11 @@ public class Config extends ConfigBase { + " greater than 0, otherwise it defaults to 3." }) public static int job_dictionary_task_consumer_thread_num = 3; + @ConfField(masterOnly = true, description = {"最大的 Streaming 作业数量,值应该大于0,否则默认为1024", + "The maximum number of Streaming jobs, " + + "the value should be greater than 0, if it is <=0, default is 1024."}) + public static int max_streaming_job_num = 1024; + /* job test config */ /** * If set to true, we will allow the interval unit to be set to second, when creating a recurring job. diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index a2016c52cb9371..a4bbf51b8a40d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -42,9 +42,6 @@ public class StreamingInsertJob extends AbstractJob queryTasks() { public void write(DataOutput out) throws IOException { Text.writeString(out, GsonUtils.GSON.toJson(this)); } + + public boolean needScheduleTask() { + return (getJobStatus().equals(JobStatus.RUNNING) || getJobStatus().equals(JobStatus.PENDING)); + } + + // When consumer to EOF, delay schedule task appropriately can avoid too many small transactions. + public boolean needDelayScheduleTask() { + return System.currentTimeMillis() - lastScheduleTaskTimestamp > jobProperties.getMaxIntervalSecond() * 1000; + } + + public boolean hasMoreDataToConsume() { + // TODO: implement this + return true; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java new file mode 100644 index 00000000000000..4e07ee01e305cf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -0,0 +1,36 @@ +// 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.job.extensions.insert.streaming; + +import lombok.Getter; + +public class StreamingInsertTask { + @Getter + private long jobId; + + @Getter + private long taskId; + + public StreamingInsertTask(long jobId, long taskId) { + this.jobId = jobId; + this.taskId = taskId; + } + + public void execute() { + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index 51ab96b3b178a7..888669f428af90 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -37,7 +37,7 @@ public StreamingJobSchedulerTask(StreamingInsertJob streamingInsertJob) { @Override public void run() throws JobException { - switch (streamingInsertJob.getStatus()) { + switch (streamingInsertJob.getJobStatus()) { case PENDING: streamingInsertJob.createStreamingInsertTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index e763f8de590153..ae6bad07066d4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -40,6 +40,7 @@ import org.apache.doris.job.extensions.insert.InsertJob; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.job.scheduler.JobScheduler; +import org.apache.doris.job.scheduler.StreamingTaskScheduler; import org.apache.doris.load.loadv2.JobState; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.expressions.And; @@ -72,6 +73,8 @@ public class JobManager, C> implements Writable { private JobScheduler jobScheduler; + private StreamingTaskScheduler streamingTaskScheduler; + // lock for job // lock is private and must use after db lock private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); @@ -95,9 +98,10 @@ private void writeUnlock() { public void start() { jobScheduler = new JobScheduler(jobMap); jobScheduler.start(); + streamingTaskScheduler = new StreamingTaskScheduler(); + streamingTaskScheduler.start(); } - /** * get running job * diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java new file mode 100644 index 00000000000000..d87a2b7833d402 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -0,0 +1,110 @@ +// 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.job.scheduler; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.common.CustomThreadFactory; +import org.apache.doris.common.util.MasterDaemon; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; + +import lombok.extern.log4j.Log4j2; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +@Log4j2 +public class StreamingTaskScheduler extends MasterDaemon { + private final LinkedBlockingDeque needScheduleTasksQueue = new LinkedBlockingDeque<>(); + private final ThreadPoolExecutor threadPool = new ThreadPoolExecutor( + 0, + Config.max_streaming_job_num, + 60, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>(Config.max_streaming_job_num), + new CustomThreadFactory("streaming-task-execute"), + new ThreadPoolExecutor.AbortPolicy() + ); + private final ScheduledThreadPoolExecutor delayScheduler + = new ScheduledThreadPoolExecutor(1, new CustomThreadFactory("streaming-task-delay-scheduler")); + + public StreamingTaskScheduler() { + super("Streaming-task-scheduler", 1); + } + + @Override + protected void runAfterCatalogReady() { + try { + process(); + } catch (Throwable e) { + log.warn("Failed to process one round of StreamingTaskScheduler", e); + } + } + + public void registerTask(StreamingInsertTask task) { + needScheduleTasksQueue.add(task); + } + + private void process() throws InterruptedException { + List tasks = new ArrayList<>(); + tasks.add(needScheduleTasksQueue.take()); + needScheduleTasksQueue.drainTo(tasks); + scheduleTasks(tasks); + } + + private void scheduleTasks(List tasks) { + for (StreamingInsertTask task : tasks) { + threadPool.execute(() -> scheduleOneTask(task)); + } + } + + private void scheduleOneTask(StreamingInsertTask task) { + StreamingInsertJob job = (StreamingInsertJob) Env.getCurrentEnv().getJobManager().getJob(task.getJobId()); + if (job == null) { + log.warn("Job not found, job id: {}", task.getJobId()); + return; + } + if (!job.needScheduleTask()) { + log.info("do not need to schedule invalid task, task id: {}, job id: {}", + task.getTaskId(), task.getJobId()); + return; + } + if (job.hasMoreDataToConsume()) { + scheduleTaskWithDelay(task, 500); + return; + } + if (job.needDelayScheduleTask()) { + scheduleTaskWithDelay(task, 500); + return; + } + log.info("prepare to schedule task, task id: {}, job id: {}", task.getTaskId(), task.getJobId()); + task.execute(); + } + + private void scheduleTaskWithDelay(StreamingInsertTask task, long delayMs) { + delayScheduler.schedule(() -> { + needScheduleTasksQueue.add(task); + }, delayMs, TimeUnit.MILLISECONDS); + } +} From cc4c616df092be1928f9a93657b590641faf8cd9 Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 11 Sep 2025 09:45:50 +0800 Subject: [PATCH 04/25] [Feature](WIP) add StreamingInsertTask and improve StreamInsertJob (#55862) ### What problem does this PR solve? 1. add StreamingInsertTask For StreamJob 2. Improve StreamInsertJob 3. add insertcommand rewrite tvf params --- .../job/extensions/insert/InsertJob.java | 2 +- .../insert/streaming/StreamingInsertJob.java | 195 ++++++++++++++--- .../insert/streaming/StreamingInsertTask.java | 196 +++++++++++++++++- .../streaming/StreamingJobProperties.java | 28 ++- .../streaming/StreamingJobSchedulerTask.java | 33 ++- .../apache/doris/job/manager/JobManager.java | 17 ++ .../job/offset/SourceOffsetProvider.java | 14 +- .../offset/SourceOffsetProviderFactory.java | 19 +- .../apache/doris/job/offset/s3/S3Offset.java | 5 +- .../job/offset/s3/S3SourceOffsetProvider.java | 35 +++- .../job/scheduler/StreamingTaskScheduler.java | 11 +- .../apache/doris/job/task/AbstractTask.java | 2 +- .../trees/plans/commands/AlterJobCommand.java | 1 - .../plans/commands/CreateJobCommand.java | 12 ++ .../plans/commands/info/CreateJobInfo.java | 6 +- .../insert/InsertIntoTableCommand.java | 47 +++++ 16 files changed, 568 insertions(+), 55 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index 5b386886b19ce8..e855aa4f83676f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -105,7 +105,7 @@ public class InsertJob extends AbstractJob> impl .add(new Column("ErrorMsg", ScalarType.createStringType())) .build(); - private static final ShowResultSetMetaData TASK_META_DATA = + public static final ShowResultSetMetaData TASK_META_DATA = ShowResultSetMetaData.builder() .addColumn(new Column("TaskId", ScalarType.createVarchar(80))) .addColumn(new Column("Label", ScalarType.createVarchar(80))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index a4bbf51b8a40d8..c88fd019c46b98 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -18,46 +18,70 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; +import org.apache.doris.common.util.TimeUtils; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.PauseReason; +import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.offset.SourceOffsetProviderFactory; +import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.load.FailMsg; +import org.apache.doris.load.loadv2.LoadStatistic; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TRow; +import org.apache.doris.transaction.TransactionException; +import org.apache.doris.transaction.TransactionState; +import org.apache.doris.transaction.TxnStateChangeCallback; import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; +import org.apache.commons.collections.CollectionUtils; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; -public class StreamingInsertJob extends AbstractJob> { +public class StreamingInsertJob extends AbstractJob> implements + TxnStateChangeCallback { @SerializedName("did") private final long dbId; - + private LoadStatistic loadStatistic = new LoadStatistic(); + @SerializedName("fm") + private FailMsg failMsg; @Getter protected PauseReason pauseReason; - @Getter @Setter protected long latestAutoResumeTimestamp; - @Getter @Setter protected long autoResumeCount; - @Getter @SerializedName("jp") private StreamingJobProperties jobProperties; - + @Getter + StreamingInsertTask runningStreamTask; + SourceOffsetProvider offsetProvider; private long lastScheduleTaskTimestamp = -1L; public StreamingInsertJob(String jobName, @@ -73,6 +97,14 @@ public StreamingInsertJob(String jobName, jobConfig, createTimeMs, executeSql); this.dbId = ConnectContext.get().getCurrentDbId(); this.jobProperties = jobProperties; + String tvfType = parseTvfType(); + this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); + } + + private String parseTvfType() { + NereidsParser parser = new NereidsParser(); + InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(getExecuteSql()); + return command.getFirstTvfName(); } @Override @@ -80,35 +112,109 @@ public void updateJobStatus(JobStatus status) throws JobException { super.updateJobStatus(status); } - protected void createStreamingInsertTask() { + @Override + public JobType getJobType() { + return JobType.INSERT; + } + + @Override + protected void checkJobParamsInternal() { + } + + @Override + public boolean isReadyForScheduling(Map taskContext) { + return CollectionUtils.isEmpty(getRunningTasks()); + } + + @Override + public List createTasks(TaskType taskType, Map taskContext) { + List newTasks = new ArrayList<>(); + StreamingJobSchedulerTask streamingJobSchedulerTask = new StreamingJobSchedulerTask(this); + newTasks.add(streamingJobSchedulerTask); + super.initTasks(newTasks, taskType); + return newTasks; + } + + protected StreamingInsertTask createStreamingInsertTask() { + InsertIntoTableCommand command = offsetProvider.rewriteTvfParams(getExecuteSql()); + this.runningStreamTask = new StreamingInsertTask(getJobId(), AbstractTask.getNextTaskId(), command, + loadStatistic, getCurrentDbName(), offsetProvider.getCurrentOffset(), jobProperties); + return this.runningStreamTask; } protected void fetchMeta() { + offsetProvider.fetchRemoteMeta(); + } + + public boolean needScheduleTask() { + return (getJobStatus().equals(JobStatus.RUNNING) || getJobStatus().equals(JobStatus.PENDING)); + } + + // When consumer to EOF, delay schedule task appropriately can avoid too many small transactions. + public boolean needDelayScheduleTask() { + return System.currentTimeMillis() - lastScheduleTaskTimestamp > jobProperties.getMaxIntervalSecond() * 1000; + } + + public boolean hasMoreDataToConsume() { + return offsetProvider.hasMoreDataToConsume(); } @Override - public JobType getJobType() { - return JobType.INSERT; + public void onTaskFail(StreamingJobSchedulerTask task) throws JobException { + // Here is the failure of StreamingJobSchedulerTask, no processing is required + getRunningTasks().remove(task); } @Override - protected void checkJobParamsInternal() { + public void onTaskSuccess(StreamingJobSchedulerTask task) throws JobException { + // Here is the success of StreamingJobSchedulerTask, no processing is required + getRunningTasks().remove(task); + } + + public void onStreamTaskFail(StreamingInsertTask task) throws JobException { + if (getJobConfig().getExecuteType().equals(JobExecuteType.INSTANT)) { + this.failMsg = new FailMsg(FailMsg.CancelType.LOAD_RUN_FAIL, task.getErrMsg()); + } + updateJobStatus(JobStatus.PAUSED); } + public void onStreamTaskSuccess(StreamingInsertTask task) { + StreamingInsertTask nextTask = createStreamingInsertTask(); + this.runningStreamTask = nextTask; + Env.getCurrentEnv().getJobManager().getStreamingTaskScheduler().registerTask(runningStreamTask); + } + + @Override - public boolean isReadyForScheduling(Map taskContext) { - return true; + public ShowResultSetMetaData getTaskMetaData() { + return InsertJob.TASK_META_DATA; } @Override - public java.util.List createTasks(org.apache.doris.job.common.TaskType taskType, - Map taskContext) { - return java.util.Collections.emptyList(); + public List getShowInfo() { + return getCommonShowInfo(); } @Override - public org.apache.doris.qe.ShowResultSetMetaData getTaskMetaData() { - return org.apache.doris.qe.ShowResultSetMetaData.builder().build(); + public TRow getTvfInfo() { + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getJobId()))); + trow.addToColumnValue(new TCell().setStringVal(getJobName())); + trow.addToColumnValue(new TCell().setStringVal(getCreateUser().getQualifiedUser())); + trow.addToColumnValue(new TCell().setStringVal(getJobConfig().getExecuteType().name())); + trow.addToColumnValue(new TCell().setStringVal(getJobConfig().convertRecurringStrategyToString())); + trow.addToColumnValue(new TCell().setStringVal(getJobStatus().name())); + trow.addToColumnValue(new TCell().setStringVal(getExecuteSql())); + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getSucceedTaskCount().get()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getFailedTaskCount().get()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getCanceledTaskCount().get()))); + trow.addToColumnValue(new TCell().setStringVal(getComment())); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(loadStatistic.toJson())); + trow.addToColumnValue(new TCell().setStringVal(failMsg == null ? FeConstants.null_string : failMsg.getMsg())); + return trow; } @Override @@ -119,7 +225,11 @@ public String formatMsgWhenExecuteQueueFull(Long taskId) { @Override public List queryTasks() { - return new ArrayList<>(); + if (!getRunningTasks().isEmpty()) { + return getRunningTasks(); + } else { + return Arrays.asList(new StreamingJobSchedulerTask(this)); + } } @Override @@ -127,17 +237,50 @@ public void write(DataOutput out) throws IOException { Text.writeString(out, GsonUtils.GSON.toJson(this)); } - public boolean needScheduleTask() { - return (getJobStatus().equals(JobStatus.RUNNING) || getJobStatus().equals(JobStatus.PENDING)); + @Override + public long getId() { + return getJobId(); } - // When consumer to EOF, delay schedule task appropriately can avoid too many small transactions. - public boolean needDelayScheduleTask() { - return System.currentTimeMillis() - lastScheduleTaskTimestamp > jobProperties.getMaxIntervalSecond() * 1000; + @Override + public void beforeCommitted(TransactionState txnState) throws TransactionException { + } - public boolean hasMoreDataToConsume() { - // TODO: implement this - return true; + @Override + public void beforeAborted(TransactionState txnState) throws TransactionException { + + } + + @Override + public void afterCommitted(TransactionState txnState, boolean txnOperated) throws UserException { + + } + + @Override + public void replayOnCommitted(TransactionState txnState) { + + } + + @Override + public void afterAborted(TransactionState txnState, boolean txnOperated, String txnStatusChangeReason) + throws UserException { + + } + + @Override + public void replayOnAborted(TransactionState txnState) { + + } + + @Override + public void afterVisible(TransactionState txnState, boolean txnOperated) { + + } + + @Override + public void replayOnVisible(TransactionState txnState) { + + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index 4e07ee01e305cf..56a59d3a7f6a7b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -17,20 +17,206 @@ package org.apache.doris.job.extensions.insert.streaming; +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Status; +import org.apache.doris.common.util.Util; +import org.apache.doris.job.base.Job; +import org.apache.doris.job.common.TaskStatus; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.InsertTask; +import org.apache.doris.job.offset.Offset; +import org.apache.doris.load.loadv2.LoadStatistic; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.glue.LogicalPlanAdapter; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.QueryState; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.TStatusCode; + import lombok.Getter; +import lombok.extern.log4j.Log4j2; + +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +@Log4j2 +@Getter public class StreamingInsertTask { - @Getter + private static final String LABEL_SPLITTER = "_"; + private static final int MAX_RETRY = 3; private long jobId; - - @Getter private long taskId; + private String labelName; + private TaskStatus status; + private String errMsg; + private Long createTimeMs; + private Long startTimeMs; + private Long finishTimeMs; + private InsertIntoTableCommand command; + private StmtExecutor stmtExecutor; + private String currentDb; + private UserIdentity userIdentity; + private ConnectContext ctx; + private LoadStatistic loadStatistic; + private Offset offset; + private AtomicBoolean isCanceled = new AtomicBoolean(false); + private StreamingJobProperties jobProperties; - public StreamingInsertTask(long jobId, long taskId) { + public StreamingInsertTask(long jobId, + long taskId, + InsertIntoTableCommand command, + LoadStatistic loadStatistic, + String currentDb, + Offset offset, + StreamingJobProperties jobProperties) { this.jobId = jobId; this.taskId = taskId; + this.command = command; + this.loadStatistic = loadStatistic; + this.userIdentity = ctx.getCurrentUserIdentity(); + this.currentDb = currentDb; + this.offset = offset; + this.jobProperties = jobProperties; + this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); + this.createTimeMs = System.currentTimeMillis(); + } + + public void execute() throws JobException { + try { + before(); + run(); + onSuccess(); + } catch (Exception e) { + if (TaskStatus.CANCELED.equals(status)) { + return; + } + onFail(e.getMessage()); + log.warn("execute task error, job id is {}, task id is {}", jobId, taskId, e); + } finally { + // The cancel logic will call the closeOrReleased Resources method by itself. + // If it is also called here, + // it may result in the inability to obtain relevant information when canceling the task + if (!TaskStatus.CANCELED.equals(status)) { + closeOrReleaseResources(); + } + } + } + + private void before() throws JobException { + this.startTimeMs = System.currentTimeMillis(); + if (isCanceled.get()) { + throw new JobException("Export executor has been canceled, task id: {}", getTaskId()); + } + ctx = InsertTask.makeConnectContext(userIdentity, currentDb); + ctx.setSessionVariable(jobProperties.getSessionVariable()); + StatementContext statementContext = new StatementContext(); + ctx.setStatementContext(statementContext); + this.command.setLabelName(Optional.of(this.labelName)); + this.command.setJobId(getTaskId()); + stmtExecutor = new StmtExecutor(ctx, new LogicalPlanAdapter(command, ctx.getStatementContext())); + } + + private void run() throws JobException { + String errMsg = null; + int retry = 0; + while (retry <= MAX_RETRY) { + try { + if (isCanceled.get()) { + log.info("task has been canceled, task id is {}", getTaskId()); + return; + } + command.runWithUpdateInfo(ctx, stmtExecutor, loadStatistic); + if (ctx.getState().getStateType() == QueryState.MysqlStateType.OK) { + return; + } else { + errMsg = ctx.getState().getErrorMessage(); + } + log.error( + "streaming insert failed with {}, reason {}, to retry", + command.getLabelName(), + errMsg); + if (retry == MAX_RETRY) { + errMsg = "reached max retry times, failed with" + errMsg; + } + } catch (Exception e) { + log.warn("execute insert task error, label is {},offset is {}", command.getLabelName(), + offset.toJson(), e); + errMsg = Util.getRootCauseMessage(e); + } + retry++; + } + log.error("streaming insert task failed, job id is {}, task id is {}, offset is {}, errMsg is {}", + getJobId(), getTaskId(), offset.toJson(), errMsg); + throw new JobException(errMsg); + } + + public boolean onSuccess() throws JobException { + if (TaskStatus.CANCELED.equals(status)) { + return false; + } + this.status = TaskStatus.SUCCESS; + this.finishTimeMs = System.currentTimeMillis(); + if (!isCallable()) { + return false; + } + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + if (null == job) { + log.info("job is null, job id is {}", jobId); + return false; + } + + StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; + streamingInsertJob.onStreamTaskSuccess(this); + return true; + } + + public void onFail(String errMsg) throws JobException { + this.errMsg = errMsg; + if (TaskStatus.CANCELED.equals(status)) { + return; + } + this.status = TaskStatus.FAILED; + this.finishTimeMs = System.currentTimeMillis(); + if (!isCallable()) { + return; + } + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; + streamingInsertJob.onStreamTaskFail(this); + } + + public void cancel(boolean needWaitCancelComplete) throws Exception { + if (isCanceled.get()) { + return; + } + isCanceled.getAndSet(true); + if (null != stmtExecutor) { + stmtExecutor.cancel(new Status(TStatusCode.CANCELLED, "streaming insert task cancelled"), + needWaitCancelComplete); + } + } + + public void closeOrReleaseResources() { + if (null != stmtExecutor) { + stmtExecutor = null; + } + if (null != command) { + command = null; + } + if (null != ctx) { + ctx = null; + } } - public void execute() { + private boolean isCallable() { + if (status.equals(TaskStatus.CANCELED)) { + return false; + } + if (null != Env.getCurrentEnv().getJobManager().getJob(jobId)) { + return true; + } + return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java index 25d256b127b791..4f463a090d5225 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java @@ -20,9 +20,13 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.Util; import org.apache.doris.job.base.JobProperties; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.SessionVariable; import lombok.Data; +import java.util.HashMap; import java.util.Map; @Data @@ -30,13 +34,16 @@ public class StreamingJobProperties implements JobProperties { public static final String MAX_INTERVAL_SECOND_PROPERTY = "max_interval"; public static final String S3_BATCH_FILES_PROPERTY = "s3.batch_files"; public static final String S3_BATCH_SIZE_PROPERTY = "s3.batch_size"; + public static final String SESSION_VAR_PREFIX = "session."; + public static final long DEFAULT_MAX_INTERVAL_SECOND = 10; public static final long DEFAULT_S3_BATCH_FILES = 256; public static final long DEFAULT_S3_BATCH_SIZE = 10 * 1024 * 1024 * 1024L; // 10GB - public static final long DEFAULT_INSERT_TIMEOUT = 30 * 60; // 30min + public static final int DEFAULT_INSERT_TIMEOUT = 30 * 60; // 30min private final Map properties; private long maxIntervalSecond; + private int maxRetry; private long s3BatchFiles; private long s3BatchSize; @@ -61,4 +68,23 @@ public void validate() throws AnalysisException { && v <= (long) (1024 * 1024 * 1024) * 10, StreamingJobProperties.S3_BATCH_SIZE_PROPERTY + " should between 100MB and 10GB"); } + + public SessionVariable getSessionVariable() throws JobException { + final Map sessionVarMap = new HashMap<>(); + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey().startsWith(SESSION_VAR_PREFIX)) { + String subKey = entry.getKey().substring(SESSION_VAR_PREFIX.length()); + sessionVarMap.put(subKey, entry.getValue()); + } + } + + SessionVariable sessionVariable = new SessionVariable(); + try { + sessionVariable.setInsertTimeoutS(DEFAULT_INSERT_TIMEOUT); + sessionVariable.readFromJson(GsonUtils.GSON.toJson(sessionVarMap)); + } catch (Exception e) { + throw new JobException("Invalid session variable, " + e.getMessage()); + } + return sessionVariable; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index 888669f428af90..d724b09fbf0046 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -18,17 +18,17 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.common.InternalErrorCode; +import org.apache.doris.common.util.TimeUtils; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.PauseReason; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; public class StreamingJobSchedulerTask extends AbstractTask { - private static final long BACK_OFF_BASIC_TIME_SEC = 10L; private static final long MAX_BACK_OFF_TIME_SEC = 60 * 5; - private StreamingInsertJob streamingInsertJob; public StreamingJobSchedulerTask(StreamingInsertJob streamingInsertJob) { @@ -80,14 +80,43 @@ private void autoResumeHandler() throws JobException { @Override protected void closeOrReleaseResources() { + if (streamingInsertJob.getRunningStreamTask() != null) { + streamingInsertJob.getRunningStreamTask().closeOrReleaseResources(); + } } @Override protected void executeCancelLogic(boolean needWaitCancelComplete) throws Exception { + if (streamingInsertJob.getRunningStreamTask() != null) { + streamingInsertJob.getRunningStreamTask().cancel(needWaitCancelComplete); + } } @Override public TRow getTvfInfo(String jobName) { + StreamingInsertTask runningTask = streamingInsertJob.getRunningStreamTask(); + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(runningTask.getTaskId()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(runningTask.getJobId()))); + trow.addToColumnValue(new TCell().setStringVal(jobName)); + trow.addToColumnValue(new TCell().setStringVal(runningTask.getLabelName())); + trow.addToColumnValue(new TCell().setStringVal(runningTask.getStatus().name())); + trow.addToColumnValue(new TCell().setStringVal(runningTask.getErrMsg())); + // create time + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(runningTask.getCreateTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" + : TimeUtils.longToTimeString(runningTask.getStartTimeMs()))); + // load end time + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(runningTask.getFinishTimeMs()))); + // tracking url + trow.addToColumnValue(new TCell().setStringVal("trackingUrl")); + trow.addToColumnValue(new TCell().setStringVal("statistic")); + if (runningTask.getUserIdentity() == null) { + trow.addToColumnValue(new TCell().setStringVal("")); + } else { + trow.addToColumnValue(new TCell().setStringVal(runningTask.getUserIdentity().getQualifiedUser())); + } + trow.addToColumnValue(new TCell().setStringVal(runningTask.getOffset().toJson())); return null; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index ae6bad07066d4e..9954d26b6f1dde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -48,6 +48,7 @@ import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; +import lombok.Getter; import lombok.extern.log4j.Log4j2; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -73,6 +74,7 @@ public class JobManager, C> implements Writable { private JobScheduler jobScheduler; + @Getter private StreamingTaskScheduler streamingTaskScheduler; // lock for job @@ -112,6 +114,21 @@ public T getJob(long jobId) { return jobMap.get(jobId); } + /** + * get streaming running job + * + * @return running job + */ + public int getStreamingJobCnt() { + int count = 0; + for (T job : jobMap.values()) { + if (job.getJobConfig().getExecuteType().equals(JobExecuteType.STREAMING)) { + count++; + } + } + return count; + } + public void registerJob(T job) throws JobException { job.initParams(); createJobInternal(job, false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index f88079617de532..3d62073929068b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -36,11 +36,17 @@ public interface SourceOffsetProvider { Offset getNextOffset(); /** - * Rewrite the TVF parameters in the InsertIntoTableCommand based on the current offset. - * @param command + * Get current offset * @return */ - InsertIntoTableCommand rewriteTvfParamsInCommand(InsertIntoTableCommand command); + Offset getCurrentOffset(); + + /** + * Rewrite the TVF parameters in the SQL based on the current offset. + * @param sql + * @return rewritten InsertIntoTableCommand + */ + InsertIntoTableCommand rewriteTvfParams(String sql); /** * Update the progress of the source. @@ -57,6 +63,6 @@ public interface SourceOffsetProvider { * Whether there is more data to consume * @return */ - boolean hasMoreData(); + boolean hasMoreDataToConsume(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java index 5ba1d903d78135..9cefa4e9d42314 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java @@ -17,11 +17,15 @@ package org.apache.doris.job.offset; +import org.apache.doris.job.exception.JobException; import org.apache.doris.job.offset.s3.S3SourceOffsetProvider; +import lombok.extern.log4j.Log4j2; + import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +@Log4j2 public class SourceOffsetProviderFactory { private static final Map> map = new ConcurrentHashMap<>(); @@ -29,9 +33,16 @@ public class SourceOffsetProviderFactory { map.put("s3", S3SourceOffsetProvider.class); } - public static SourceOffsetProvider createSourceOffsetProvider(String sourceType) throws InstantiationException, - IllegalAccessException { - Class cla = map.get(sourceType.toUpperCase()); - return cla.newInstance(); + public static SourceOffsetProvider createSourceOffsetProvider(String sourceType) { + try { + Class cla = map.get(sourceType.toUpperCase()); + if (cla == null) { + throw new JobException("Unsupported source type: " + sourceType); + } + return cla.newInstance(); + } catch (Exception e) { + log.error("Failed to create source provider for type: " + sourceType, e); + throw new RuntimeException("Failed to create source provider for type: " + sourceType); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index 86ff467796af8d..a175575757f080 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -20,13 +20,14 @@ import org.apache.doris.job.offset.Offset; import org.apache.doris.persist.gson.GsonUtils; +import lombok.Getter; + import java.util.List; public class S3Offset implements Offset { String startFile; - String endFile; - + @Getter List fileLists; @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index 087d9c2beb7685..771736a9559f93 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -19,35 +19,60 @@ import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import java.util.HashMap; +import java.util.Map; + public class S3SourceOffsetProvider implements SourceOffsetProvider { + S3Offset currentOffset; + String maxRemoteEndFile; @Override public String getSourceType() { - return null; + return "s3"; } @Override - public Offset getNextOffset() { + public S3Offset getNextOffset() { + //todo: listObjects from end file return null; } @Override - public InsertIntoTableCommand rewriteTvfParamsInCommand(InsertIntoTableCommand command) { - return null; + public Offset getCurrentOffset() { + return currentOffset; + } + + @Override + public InsertIntoTableCommand rewriteTvfParams(String sql) { + S3Offset nextOffset = getNextOffset(); + Map props = new HashMap<>(); + //todo: need to change file list to glob string + props.put("uri", nextOffset.getFileLists().toString()); + + NereidsParser parser = new NereidsParser(); + InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(sql); + command.rewriteTvfProperties(getSourceType(), props); + return command; } @Override public void updateProgress(Offset offset) { + this.currentOffset = (S3Offset) offset; } @Override public void fetchRemoteMeta() { + // list object } @Override - public boolean hasMoreData() { + public boolean hasMoreDataToConsume() { + if (currentOffset.endFile.compareTo(maxRemoteEndFile) < 0) { + return true; + } return false; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java index d87a2b7833d402..bdc27e983e17d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -21,6 +21,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.CustomThreadFactory; import org.apache.doris.common.util.MasterDaemon; +import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; @@ -75,11 +76,17 @@ private void process() throws InterruptedException { private void scheduleTasks(List tasks) { for (StreamingInsertTask task : tasks) { - threadPool.execute(() -> scheduleOneTask(task)); + threadPool.execute(() -> { + try { + scheduleOneTask(task); + } catch (Exception e) { + log.error("Failed to schedule task, task id: {}, job id: {}", task.getTaskId(), task.getJobId(), e); + } + }); } } - private void scheduleOneTask(StreamingInsertTask task) { + private void scheduleOneTask(StreamingInsertTask task) throws JobException { StreamingInsertJob job = (StreamingInsertJob) Env.getCurrentEnv().getJobManager().getJob(task.getJobId()); if (job == null) { log.warn("Job not found, job id: {}", task.getJobId()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java index 4e2ac653cf700f..18c5f525295d0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java @@ -60,7 +60,7 @@ public AbstractTask() { taskId = getNextTaskId(); } - private static long getNextTaskId() { + public static long getNextTaskId() { // do not use Env.getNextId(), just generate id without logging return System.nanoTime() + RandomUtils.nextInt(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index ef8c9bb8b7a120..53a90893b81444 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -66,7 +66,6 @@ public StmtType stmtType() { @Override public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { - validate(); AbstractJob job = analyzeAndBuildJobInfo(ctx); ctx.getEnv().getJobManager().alterJob(job); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java index fecd457ada56eb..fe81921211fe9b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java @@ -19,7 +19,9 @@ import org.apache.doris.analysis.StmtType; import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.exception.JobException; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.commands.info.CreateJobInfo; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -56,10 +58,20 @@ public CreateJobCommand(CreateJobInfo jobInfo) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(); AbstractJob job = createJobInfo.analyzeAndBuildJobInfo(ctx); Env.getCurrentEnv().getJobManager().registerJob(job); } + private void validate() throws JobException { + if (createJobInfo.streamingJob()) { + int streamingJobCnt = Env.getCurrentEnv().getJobManager().getStreamingJobCnt(); + if (streamingJobCnt >= Config.max_streaming_job_num) { + throw new JobException("Exceed max streaming job num limit " + Config.max_streaming_job_num); + } + } + } + @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitCreateJobCommand(this, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 0d52e23ece56ff..4334526630ee8b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -274,7 +274,7 @@ private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String curren LogicalPlan logicalPlan = parser.parseSingle(sql); if (logicalPlan instanceof InsertIntoTableCommand) { return new StreamingInsertJob(labelNameOptional.get(), - JobStatus.RUNNING, + JobStatus.PENDING, currentDbName, comment, ConnectContext.get().getCurrentUserIdentity(), @@ -314,4 +314,8 @@ public static Long stripQuotesAndParseTimestamp(String str) { } return TimeUtils.timeStringToLong(str.trim()); } + + public boolean streamingJob() { + return streamingJob; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 8ffa8884dd995c..a5a428c78158c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -38,6 +38,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; @@ -79,6 +80,7 @@ import org.apache.logging.log4j.Logger; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; @@ -536,6 +538,51 @@ public List getTargetColumns() { } } + // todo: add ut + public String getFirstTvfName() { + return getFirstTvfInPlan(getLogicalQuery()); + } + + private String getFirstTvfInPlan(LogicalPlan plan) { + if (plan instanceof UnboundTVFRelation) { + UnboundTVFRelation tvfRelation = (UnboundTVFRelation) plan; + return tvfRelation.getFunctionName(); + } + + for (Plan child : plan.children()) { + if (child instanceof LogicalPlan) { + String result = getFirstTvfInPlan((LogicalPlan) child); + if (!result.isEmpty()) { + return result; + } + } + } + return ""; + } + + // todo: add ut + public void rewriteTvfProperties(String functionName, Map props) { + rewriteTvfInPlan(originLogicalQuery, functionName, props); + if (logicalQuery.isPresent()) { + rewriteTvfInPlan(logicalQuery.get(), functionName, props); + } + } + + private void rewriteTvfInPlan(LogicalPlan plan, String functionName, Map props) { + if (plan instanceof UnboundTVFRelation) { + UnboundTVFRelation tvfRelation = (UnboundTVFRelation) plan; + if (functionName.equalsIgnoreCase(tvfRelation.getFunctionName())) { + tvfRelation.getProperties().getMap().putAll(props); + } + } + + for (Plan child : plan.children()) { + if (child instanceof LogicalPlan) { + rewriteTvfInPlan((LogicalPlan) child, functionName, props); + } + } + } + @Override public Plan getExplainPlan(ConnectContext ctx) { Optional analyzeContext = Optional.of( From e2a492cf1704c0c28f7e5f1744482a366df4a870 Mon Sep 17 00:00:00 2001 From: hui lai Date: Thu, 11 Sep 2025 21:14:04 +0800 Subject: [PATCH 05/25] [feat](streaming job) implement offset persistence and replay (#55918) ### What problem does this PR solve? Implement offset persistence and replay logic(shared noting mode). --- .../insert/streaming/StreamingInsertJob.java | 40 ++++++++++-- .../insert/streaming/StreamingInsertTask.java | 6 +- .../streaming/StreamingJobStatistic.java | 42 ++++++++++++ .../StreamingTaskTxnCommitAttachment.java | 65 +++++++++++++++++++ .../job/offset/SourceOffsetProvider.java | 4 +- .../apache/doris/job/offset/s3/S3Offset.java | 9 +++ .../job/offset/s3/S3SourceOffsetProvider.java | 2 +- .../doris/load/loadv2/LoadStatistic.java | 8 +++ 8 files changed, 162 insertions(+), 14 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index c88fd019c46b98..1b4cac14bc8b6b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -36,6 +36,7 @@ import org.apache.doris.job.offset.SourceOffsetProviderFactory; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.load.FailMsg; +import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; @@ -48,6 +49,7 @@ import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnStateChangeCallback; +import com.google.common.base.Preconditions; import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; @@ -65,7 +67,7 @@ public class StreamingInsertJob extends AbstractJob createTasks(TaskType taskType, Map taskIds = new ArrayList<>(); + taskIds.add(runningStreamTask.getTaskId()); + List loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIds); + if (loadJobs.size() != 1) { + throw new TransactionException("load job not found, insert job id is " + runningStreamTask.getTaskId()); + } + LoadJob loadJob = loadJobs.get(0); + LoadStatistic loadStatistic = loadJob.getLoadStatistic(); + txnState.setTxnCommitAttachment(new StreamingTaskTxnCommitAttachment( + loadStatistic.getScannedRows(), + loadStatistic.getLoadBytes(), + loadStatistic.getFileNumber(), + loadStatistic.getTotalFileSizeB(), + runningStreamTask.getOffset())); } @Override @@ -254,12 +276,18 @@ public void beforeAborted(TransactionState txnState) throws TransactionException @Override public void afterCommitted(TransactionState txnState, boolean txnOperated) throws UserException { - + Preconditions.checkNotNull(txnState.getTxnCommitAttachment(), txnState); + StreamingTaskTxnCommitAttachment attachment = + (StreamingTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(); + updateJobStatisticAndOffset(attachment); } @Override public void replayOnCommitted(TransactionState txnState) { - + Preconditions.checkNotNull(txnState.getTxnCommitAttachment(), txnState); + StreamingTaskTxnCommitAttachment attachment = + (StreamingTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(); + updateJobStatisticAndOffset(attachment); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index 56a59d3a7f6a7b..cb880c8ac40b9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -26,7 +26,6 @@ import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertTask; import org.apache.doris.job.offset.Offset; -import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; @@ -59,7 +58,6 @@ public class StreamingInsertTask { private String currentDb; private UserIdentity userIdentity; private ConnectContext ctx; - private LoadStatistic loadStatistic; private Offset offset; private AtomicBoolean isCanceled = new AtomicBoolean(false); private StreamingJobProperties jobProperties; @@ -67,14 +65,12 @@ public class StreamingInsertTask { public StreamingInsertTask(long jobId, long taskId, InsertIntoTableCommand command, - LoadStatistic loadStatistic, String currentDb, Offset offset, StreamingJobProperties jobProperties) { this.jobId = jobId; this.taskId = taskId; this.command = command; - this.loadStatistic = loadStatistic; this.userIdentity = ctx.getCurrentUserIdentity(); this.currentDb = currentDb; this.offset = offset; @@ -127,7 +123,7 @@ private void run() throws JobException { log.info("task has been canceled, task id is {}", getTaskId()); return; } - command.runWithUpdateInfo(ctx, stmtExecutor, loadStatistic); + command.runWithUpdateInfo(ctx, stmtExecutor, null); if (ctx.getState().getStateType() == QueryState.MysqlStateType.OK) { return; } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java new file mode 100644 index 00000000000000..70e5c9f6d0f3cf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java @@ -0,0 +1,42 @@ +// 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.job.extensions.insert.streaming; + +import org.apache.doris.persist.gson.GsonUtils; + +import lombok.Getter; +import lombok.Setter; + +public class StreamingJobStatistic { + @Getter + @Setter + private long scannedRows; + @Getter + @Setter + private long loadBytes; + @Getter + @Setter + private long fileNumber; + @Getter + @Setter + private long fileSize; + + public String toJson() { + return GsonUtils.GSON.toJson(this); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java new file mode 100644 index 00000000000000..8a8768dee34b22 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java @@ -0,0 +1,65 @@ +// 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.job.extensions.insert.streaming; + +import org.apache.doris.job.offset.Offset; +import org.apache.doris.transaction.TransactionState; +import org.apache.doris.transaction.TxnCommitAttachment; + +import com.google.gson.annotations.SerializedName; +import lombok.Getter; + +public class StreamingTaskTxnCommitAttachment extends TxnCommitAttachment { + + public StreamingTaskTxnCommitAttachment(long scannedRows, long loadBytes, + long fileNumber, long fileSize, Offset offset) { + super(TransactionState.LoadJobSourceType.BACKEND_STREAMING); + this.scannedRows = scannedRows; + this.loadBytes = loadBytes; + this.fileNumber = fileNumber; + this.fileSize = fileSize; + this.offset = offset; + } + + @SerializedName(value = "sr") + @Getter + private long scannedRows; + @SerializedName(value = "lb") + @Getter + private long loadBytes; + @SerializedName(value = "fn") + @Getter + private long fileNumber; + @SerializedName(value = "fs") + @Getter + private long fileSize; + @SerializedName(value = "of") + @Getter + private Offset offset; + + @Override + public String toString() { + return "StreamingTaskTxnCommitAttachment: [" + + "scannedRows=" + scannedRows + + ", loadBytes=" + loadBytes + + ", fileNumber=" + fileNumber + + ", fileSize=" + fileSize + + ", offset=" + offset.toString() + + "]"; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index 3d62073929068b..564327ed0cc1cf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -49,10 +49,10 @@ public interface SourceOffsetProvider { InsertIntoTableCommand rewriteTvfParams(String sql); /** - * Update the progress of the source. + * Update the offset of the source. * @param offset */ - void updateProgress(Offset offset); + void updateOffset(Offset offset); /** * Fetch remote meta information, such as listing files in S3 or getting latest offsets in Kafka. diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index a175575757f080..57f89b9950590d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -34,4 +34,13 @@ public class S3Offset implements Offset { public String toJson() { return GsonUtils.GSON.toJson(this); } + + @Override + public String toString() { + return "S3Offset: [" + + "startFile=" + startFile + + ", endFile=" + endFile + + ", fileLists=" + fileLists + + "]"; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index 771736a9559f93..e52d9995051e11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -59,7 +59,7 @@ public InsertIntoTableCommand rewriteTvfParams(String sql) { } @Override - public void updateProgress(Offset offset) { + public void updateOffset(Offset offset) { this.currentOffset = (S3Offset) offset; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadStatistic.java index 43c67098bfd1f0..58137d34886ff1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadStatistic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadStatistic.java @@ -123,6 +123,14 @@ public Map getCounters() { return counters; } + public int getFileNumber() { + return fileNum; + } + + public long getTotalFileSizeB() { + return totalFileSizeB; + } + public synchronized String toJson() { long total = 0; for (long rows : counterTbl.values()) { From 4c6f13cb646bceb15b4093e62ffeb47ff6793f85 Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 12 Sep 2025 10:38:03 +0800 Subject: [PATCH 06/25] [Feature](WIP) add S3 Stream job split offset (#55927) ### What problem does this PR solve? 1. add S3 Stream job split offset 2. fix stream job create bug --- .../java/org/apache/doris/fs/FileSystem.java | 14 ++ .../org/apache/doris/fs/obj/S3ObjStorage.java | 139 ++++++++++++++++++ .../apache/doris/fs/remote/S3FileSystem.java | 7 + .../job/base/JobExecutionConfiguration.java | 2 +- .../job/extensions/insert/InsertJob.java | 3 +- .../insert/streaming/StreamingInsertJob.java | 16 +- .../streaming/StreamingJobSchedulerTask.java | 5 +- .../job/offset/SourceOffsetProvider.java | 12 +- .../offset/SourceOffsetProviderFactory.java | 2 +- .../apache/doris/job/offset/s3/S3Offset.java | 4 +- .../job/offset/s3/S3SourceOffsetProvider.java | 60 +++++++- .../plans/commands/info/CreateJobInfo.java | 25 ++-- .../insert/InsertIntoTableCommand.java | 33 +++-- .../apache/doris/persist/gson/GsonUtils.java | 4 +- 14 files changed, 283 insertions(+), 43 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java index 5ea7a3b67c1d17..cfbb3e560f39db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java @@ -115,6 +115,20 @@ default Status globList(String remotePath, List result) { */ Status globList(String remotePath, List result, boolean fileNameOnly); + /** + * List files in remotePath
+ * @param remotePath remote path + * @param result All eligible files under the path + * @param startFile start file name + * @param fileSizeLimit limit the total size of files to be listed. + * @param fileNumLimit limit the total number of files to be listed. + * @return + */ + default String globListWithLimit(String remotePath, List result, + String startFile, long fileSizeLimit, long fileNumLimit) { + throw new UnsupportedOperationException("Unsupported operation glob list with limit on current file system."); + } + default Status listDirectories(String remotePath, Set result) { throw new UnsupportedOperationException("Unsupported operation list directories on current file system."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java index 238072df3c063e..0a4c9159881d70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java @@ -54,6 +54,7 @@ import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request.Builder; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.services.s3.model.NoSuchKeyException; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; @@ -630,6 +631,144 @@ public Status globList(String remotePath, List result, boolean fileN } } + + /** + * List all files under the given path with glob pattern. + * For example, if the path is "s3://bucket/path/to/*.csv", + * it will list all files under "s3://bucket/path/to/" with ".csv" suffix. + *

+ * Limit: Starting from startFile, until the total file size is greater than fileSizeLimit, + * or the number of files is greater than fileNumLimit. + * + * @return The largest file name after listObject this time + */ + public String globListWithLimit(String remotePath, List result, String startFile, + long fileSizeLimit, long fileNumLimit) { + long roundCnt = 0; + long elementCnt = 0; + long matchCnt = 0; + long matchFileSize = 0L; + long startTime = System.nanoTime(); + try { + S3URI uri = S3URI.create(remotePath, isUsePathStyle, forceParsingByStandardUri); + if (uri.useS3DirectoryBucket()) { + throw new RuntimeException("Not support glob with limit for directory bucket"); + } + + String bucket = uri.getBucket(); + String globPath = uri.getKey(); // eg: path/to/*.csv + + if (LOG.isDebugEnabled()) { + LOG.debug("globList globPath:{}, remotePath:{}", globPath, remotePath); + } + java.nio.file.Path pathPattern = Paths.get(globPath); + PathMatcher matcher = FileSystems.getDefault().getPathMatcher("glob:" + pathPattern); + HashSet directorySet = new HashSet<>(); + + String listPrefix = S3Util.getLongestPrefix(globPath); // similar to Azure + if (LOG.isDebugEnabled()) { + LOG.debug("globList listPrefix: '{}' (from globPath: '{}')", listPrefix, globPath); + } + + Builder builder = ListObjectsV2Request.builder(); + builder.bucket(bucket) + .prefix(listPrefix); + + if (startFile != null) { + builder.startAfter(startFile); + } + + ListObjectsV2Request request = builder.build(); + + String currentMaxFile = ""; + boolean isTruncated = false; + do { + roundCnt++; + ListObjectsV2Response response = listObjectsV2(request); + for (S3Object obj : response.contents()) { + elementCnt++; + java.nio.file.Path objPath = Paths.get(obj.key()); + + boolean isPrefix = false; + while (objPath != null && objPath.normalize().toString().startsWith(listPrefix)) { + if (!matcher.matches(objPath)) { + isPrefix = true; + objPath = objPath.getParent(); + continue; + } + if (directorySet.contains(objPath.normalize().toString())) { + break; + } + if (isPrefix) { + directorySet.add(objPath.normalize().toString()); + } + + matchCnt++; + matchFileSize += obj.size(); + String remoteFileName = "s3://" + bucket + "/" + objPath; + result.add(remoteFileName); + + if (reachLimit(result.size(), matchFileSize, fileSizeLimit, fileNumLimit)) { + break; + } + + objPath = objPath.getParent(); + isPrefix = true; + } + } + //record current last object file name + S3Object lastS3Object = response.contents().get(response.contents().size() - 1); + java.nio.file.Path lastObjPath = Paths.get(lastS3Object.key()); + currentMaxFile = "s3://" + bucket + "/" + lastObjPath; + + isTruncated = response.isTruncated(); + if (isTruncated) { + request = request.toBuilder() + .continuationToken(response.nextContinuationToken()) + .build(); + } + } while (isTruncated); + + if (LOG.isDebugEnabled()) { + LOG.debug("remotePath:{}, result:{}", remotePath, result); + } + return currentMaxFile; + } catch (Exception e) { + LOG.warn("Errors while getting file status", e); + throw new RuntimeException(e); + } finally { + long endTime = System.nanoTime(); + long duration = endTime - startTime; + if (LOG.isDebugEnabled()) { + LOG.debug("process {} elements under prefix {} for {} round, match {} elements, take {} ms", + elementCnt, remotePath, roundCnt, matchCnt, + duration / 1000 / 1000); + } + } + } + + private static boolean reachLimit(int matchFileCnt, long matchFileSize, long sizeLimit, long fileNum) { + if (matchFileCnt < 0 || sizeLimit < 0 || fileNum < 0) { + return false; + } + if (fileNum > 0 && matchFileCnt >= fileNum) { + LOG.info( + "reach file num limit fileNum:{} objectFiles count:{}", + fileNum, + matchFileCnt); + return true; + } + + if (sizeLimit > 0 && matchFileSize >= sizeLimit) { + LOG.info( + "reach size limit sizeLimit:{}, objectFilesSize:{}", + sizeLimit, + matchFileSize); + return true; + } + return false; + } + @Override public synchronized void close() throws Exception { if (client != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java index c3608b7b35f00c..9c409a66a2942c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java @@ -68,6 +68,13 @@ public Status globList(String remotePath, List result, boolean fileN return objStorage.globList(remotePath, result, fileNameOnly); } + @Override + public String globListWithLimit(String remotePath, List result, String startFile, + long fileSizeLimit, long fileNumLimit) { + S3ObjStorage objStorage = (S3ObjStorage) this.objStorage; + return objStorage.globListWithLimit(remotePath, result, startFile, fileSizeLimit, fileNumLimit); + } + @Override public Status listDirectories(String remotePath, Set result) { S3ObjStorage objStorage = (S3ObjStorage) this.objStorage; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java index f26828c5a453a3..3d8c9afa36b290 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java @@ -100,7 +100,7 @@ private void validateStartTimeMs() { if (timerDefinition.getStartTimeMs() == null) { throw new IllegalArgumentException("startTimeMs cannot be null"); } - if (isImmediate()) { + if (isImmediate() || JobExecuteType.STREAMING.equals(executeType)) { return; } if (timerDefinition.getStartTimeMs() < System.currentTimeMillis()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index e855aa4f83676f..67b56dbc13df20 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -549,7 +549,8 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal(getComment())); trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - trow.addToColumnValue(new TCell().setStringVal(loadStatistic.toJson())); + trow.addToColumnValue(new TCell().setStringVal( + loadStatistic == null ? FeConstants.null_string : loadStatistic.toJson())); trow.addToColumnValue(new TCell().setStringVal(failMsg == null ? FeConstants.null_string : failMsg.getMsg())); return trow; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 1b4cac14bc8b6b..84833604f84683 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -32,12 +32,14 @@ import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.SourceOffsetProviderFactory; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.load.FailMsg; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; +import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.persist.gson.GsonUtils; @@ -64,8 +66,6 @@ public class StreamingInsertJob extends AbstractJob> implements TxnStateChangeCallback { - - @SerializedName("did") private final long dbId; private StreamingJobStatistic jobStatistic = new StreamingJobStatistic(); @SerializedName("fm") @@ -79,7 +79,6 @@ public class StreamingInsertJob extends AbstractJob createTasks(TaskType taskType, Map cla = map.get(sourceType.toUpperCase()); + Class cla = map.get(sourceType.toLowerCase()); if (cla == null) { throw new JobException("Unsupported source type: " + sourceType); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index 57f89b9950590d..95271ba60ab1e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -21,13 +21,15 @@ import org.apache.doris.persist.gson.GsonUtils; import lombok.Getter; +import lombok.Setter; import java.util.List; +@Getter +@Setter public class S3Offset implements Offset { String startFile; String endFile; - @Getter List fileLists; @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index e52d9995051e11..1c7b3839e9d2d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -17,17 +17,50 @@ package org.apache.doris.job.offset.s3; +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.StorageProperties; +import org.apache.doris.fs.FileSystemFactory; +import org.apache.doris.fs.remote.RemoteFileSystem; +import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import lombok.extern.log4j.Log4j2; + +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; +@Log4j2 public class S3SourceOffsetProvider implements SourceOffsetProvider { + String executeSql; S3Offset currentOffset; String maxRemoteEndFile; + StreamingJobProperties jobProperties; + NereidsParser parser; + String filePath; + StorageProperties storageProperties; + + @Override + public void init(String executeSql, StreamingJobProperties jobProperties) { + this.executeSql = executeSql; + this.jobProperties = jobProperties; + this.parser = new NereidsParser(); + InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(executeSql); + UnboundTVFRelation firstTVF = command.getFirstTVF(); + Map properties = firstTVF.getProperties().getMap(); + try { + this.storageProperties = StorageProperties.createPrimary(properties); + String uri = storageProperties.validateAndGetUri(properties); + this.filePath = storageProperties.validateAndNormalizeUri(uri); + } catch (UserException e) { + throw new RuntimeException("Failed check storage props, " + e.getMessage(), e); + } + } @Override public String getSourceType() { @@ -36,8 +69,19 @@ public String getSourceType() { @Override public S3Offset getNextOffset() { - //todo: listObjects from end file - return null; + S3Offset offset = new S3Offset(); + List rfiles = new ArrayList<>(); + try (RemoteFileSystem fileSystem = FileSystemFactory.get(storageProperties)) { + maxRemoteEndFile = fileSystem.globListWithLimit(filePath, rfiles, currentOffset.endFile, + jobProperties.getS3BatchFiles(), jobProperties.getS3BatchSize()); + offset.setStartFile(currentOffset.endFile); + offset.setEndFile(rfiles.get(rfiles.size() - 1)); + offset.setFileLists(rfiles); + } catch (Exception e) { + log.warn("list path exception, path={}", filePath, e); + throw new RuntimeException(e); + } + return offset; } @Override @@ -46,15 +90,15 @@ public Offset getCurrentOffset() { } @Override - public InsertIntoTableCommand rewriteTvfParams(String sql) { - S3Offset nextOffset = getNextOffset(); + public InsertIntoTableCommand rewriteTvfParams(Offset nextOffset) { + S3Offset offset = (S3Offset) nextOffset; Map props = new HashMap<>(); //todo: need to change file list to glob string - props.put("uri", nextOffset.getFileLists().toString()); + props.put("uri", offset.getFileLists().toString()); + + InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(executeSql); - NereidsParser parser = new NereidsParser(); - InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(sql); - command.rewriteTvfProperties(getSourceType(), props); + command.rewriteFirstTvfProperties(getSourceType(), props); return command; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 4334526630ee8b..61a8eb0a5c06ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -129,6 +129,7 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti if (streamingJob) { executeType = JobExecuteType.STREAMING; properties = new StreamingJobProperties(jobProperties); + properties.validate(); } jobExecutionConfiguration.setExecuteType(executeType); @@ -273,15 +274,21 @@ private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String curren NereidsParser parser = new NereidsParser(); LogicalPlan logicalPlan = parser.parseSingle(sql); if (logicalPlan instanceof InsertIntoTableCommand) { - return new StreamingInsertJob(labelNameOptional.get(), - JobStatus.PENDING, - currentDbName, - comment, - ConnectContext.get().getCurrentUserIdentity(), - jobExecutionConfiguration, - System.currentTimeMillis(), - sql, - (StreamingJobProperties) properties); + // InsertIntoTableCommand insertIntoTableCommand = (InsertIntoTableCommand) logicalPlan; + try { + // insertIntoTableCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); + return new StreamingInsertJob(labelNameOptional.get(), + JobStatus.PENDING, + currentDbName, + comment, + ConnectContext.get().getCurrentUserIdentity(), + jobExecutionConfiguration, + System.currentTimeMillis(), + sql, + (StreamingJobProperties) properties); + } catch (Exception e) { + throw new AnalysisException(e.getMessage()); + } } else { throw new AnalysisException("Not support this sql : " + sql + " Command class is " + logicalPlan.getClass().getName() + "."); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index a5a428c78158c5..8bf31d941df1d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -83,6 +83,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Supplier; @@ -539,46 +540,54 @@ public List getTargetColumns() { } // todo: add ut - public String getFirstTvfName() { + public UnboundTVFRelation getFirstTVF() { return getFirstTvfInPlan(getLogicalQuery()); } - private String getFirstTvfInPlan(LogicalPlan plan) { + private UnboundTVFRelation getFirstTvfInPlan(LogicalPlan plan) { if (plan instanceof UnboundTVFRelation) { UnboundTVFRelation tvfRelation = (UnboundTVFRelation) plan; - return tvfRelation.getFunctionName(); + return tvfRelation; } for (Plan child : plan.children()) { if (child instanceof LogicalPlan) { - String result = getFirstTvfInPlan((LogicalPlan) child); - if (!result.isEmpty()) { + UnboundTVFRelation result = getFirstTvfInPlan((LogicalPlan) child); + if (result != null) { return result; } } } - return ""; + return null; } // todo: add ut - public void rewriteTvfProperties(String functionName, Map props) { - rewriteTvfInPlan(originLogicalQuery, functionName, props); - if (logicalQuery.isPresent()) { - rewriteTvfInPlan(logicalQuery.get(), functionName, props); + public void rewriteFirstTvfProperties(String functionName, Map props) { + AtomicBoolean found = new AtomicBoolean(false); + rewriteFirstTvfInPlan(originLogicalQuery, functionName, props, found); + if (logicalQuery.isPresent() && !found.get()) { + rewriteFirstTvfInPlan(logicalQuery.get(), functionName, props, found); } } - private void rewriteTvfInPlan(LogicalPlan plan, String functionName, Map props) { + private void rewriteFirstTvfInPlan(LogicalPlan plan, + String functionName, Map props, AtomicBoolean found) { + if (found.get()) { + return; + } + if (plan instanceof UnboundTVFRelation) { UnboundTVFRelation tvfRelation = (UnboundTVFRelation) plan; if (functionName.equalsIgnoreCase(tvfRelation.getFunctionName())) { tvfRelation.getProperties().getMap().putAll(props); + found.set(true); + return; } } for (Plan child : plan.children()) { if (child instanceof LogicalPlan) { - rewriteTvfInPlan((LogicalPlan) child, functionName, props); + rewriteFirstTvfInPlan((LogicalPlan) child, functionName, props, found); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index 07032432aceba4..af79329f681930 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -182,6 +182,7 @@ import org.apache.doris.fs.remote.dfs.JFSFileSystem; import org.apache.doris.fs.remote.dfs.OFSFileSystem; import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.load.loadv2.BrokerLoadJob; import org.apache.doris.load.loadv2.BulkLoadJob; @@ -447,7 +448,8 @@ public class GsonUtils { jobExecutorRuntimeTypeAdapterFactory = RuntimeTypeAdapterFactory.of(org.apache.doris.job.base.AbstractJob.class, "clazz") .registerSubtype(InsertJob.class, InsertJob.class.getSimpleName()) - .registerSubtype(MTMVJob.class, MTMVJob.class.getSimpleName()); + .registerSubtype(MTMVJob.class, MTMVJob.class.getSimpleName()) + .registerSubtype(StreamingInsertJob.class, StreamingInsertJob.class.getSimpleName()); private static RuntimeTypeAdapterFactory mtmvSnapshotTypeAdapterFactory = RuntimeTypeAdapterFactory.of(MTMVSnapshotIf.class, "clazz") From 991de2db8f0009c605978bc9fc040b109c9d7475 Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 12 Sep 2025 17:50:47 +0800 Subject: [PATCH 07/25] [Feature](WIP) Fix streaming job problem (#55949) ### What problem does this PR solve? Fix streaming job problem --- .../apache/doris/job/base/AbstractJob.java | 5 +-- .../org/apache/doris/job/common/JobUtils.java | 30 ++++++++++++++++ .../job/executor/DispatchTaskHandler.java | 12 ++----- .../job/executor/TimerJobSchedulerTask.java | 4 +-- .../insert/streaming/StreamingInsertJob.java | 34 ++++++++++++------ .../insert/streaming/StreamingInsertTask.java | 35 ++++++++++++------- .../streaming/StreamingJobSchedulerTask.java | 3 +- .../job/offset/s3/S3SourceOffsetProvider.java | 22 +++++++----- .../doris/job/scheduler/JobScheduler.java | 7 ++-- .../job/scheduler/StreamingTaskScheduler.java | 5 +-- .../plans/commands/info/CreateJobInfo.java | 1 + .../insert/InsertIntoTableCommand.java | 3 -- 12 files changed, 107 insertions(+), 54 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 4db4292c04763e..83b84348ff77eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -246,7 +246,7 @@ private boolean canCreateTask(TaskType taskType) { switch (taskType) { case SCHEDULED: - return currentJobStatus.equals(JobStatus.RUNNING); + return currentJobStatus.equals(JobStatus.RUNNING) || currentJobStatus.equals(JobStatus.PENDING); case MANUAL: return currentJobStatus.equals(JobStatus.RUNNING) || currentJobStatus.equals(JobStatus.PAUSED); default: @@ -294,7 +294,8 @@ public void updateJobStatus(JobStatus newJobStatus) throws JobException { } String errorMsg = String.format("Can't update job %s status to the %s status", jobStatus.name(), newJobStatus.name()); - if (newJobStatus.equals(JobStatus.RUNNING) && !jobStatus.equals(JobStatus.PAUSED)) { + if (newJobStatus.equals(JobStatus.RUNNING) + && (!jobStatus.equals(JobStatus.PAUSED) && !jobStatus.equals(JobStatus.PENDING))) { throw new IllegalArgumentException(errorMsg); } if (newJobStatus.equals(JobStatus.STOPPED) && !jobStatus.equals(JobStatus.RUNNING)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java new file mode 100644 index 00000000000000..93ff5f41480b38 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java @@ -0,0 +1,30 @@ +// 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.job.common; + +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecuteType; + +public class JobUtils { + public static boolean checkNeedSchedule(AbstractJob job) { + if (job.getJobConfig().getExecuteType() == JobExecuteType.STREAMING) { + return !job.isFinalStatus(); + } + return job.getJobStatus() == JobStatus.RUNNING; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java index 77078896735a9d..35b1f351f723d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java @@ -18,9 +18,8 @@ package org.apache.doris.job.executor; import org.apache.doris.job.base.AbstractJob; -import org.apache.doris.job.base.JobExecuteType; -import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; +import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.disruptor.TimerJobEvent; import org.apache.doris.job.task.AbstractTask; @@ -56,7 +55,7 @@ public void onEvent(TimerJobEvent event) { log.info("job is null,may be job is deleted, ignore"); return; } - if (event.getJob().isReadyForScheduling(null) && checkStatus(event)) { + if (event.getJob().isReadyForScheduling(null) && JobUtils.checkNeedSchedule(event.getJob())) { List tasks = event.getJob().commonCreateTasks(TaskType.SCHEDULED, null); if (CollectionUtils.isEmpty(tasks)) { log.warn("job is ready for scheduling, but create task is empty, skip scheduler," @@ -78,11 +77,4 @@ public void onEvent(TimerJobEvent event) { log.warn("dispatch timer job error, task id is {}", event.getJob().getJobId(), e); } } - - private boolean checkStatus(TimerJobEvent event) { - if (event.getJob().getJobConfig().getExecuteType() == JobExecuteType.STREAMING) { - return !event.getJob().isFinalStatus(); - } - return event.getJob().getJobStatus() == JobStatus.RUNNING; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java index 4269fa0d8f3c52..0f58452ff7b065 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java @@ -18,7 +18,7 @@ package org.apache.doris.job.executor; import org.apache.doris.job.base.AbstractJob; -import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.disruptor.TaskDisruptor; import io.netty.util.Timeout; @@ -40,7 +40,7 @@ public TimerJobSchedulerTask(TaskDisruptor dispatchDisruptor, T job) { @Override public void run(Timeout timeout) { try { - if (!JobStatus.RUNNING.equals(job.getJobStatus())) { + if (!JobUtils.checkNeedSchedule(job)) { log.info("job status is not running, job id is {}, skip dispatch", this.job.getJobId()); return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 84833604f84683..84e42c6e1f62bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -29,10 +29,10 @@ import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.PauseReason; +import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertJob; -import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.SourceOffsetProviderFactory; import org.apache.doris.job.task.AbstractTask; @@ -42,6 +42,7 @@ import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; @@ -65,7 +66,7 @@ import java.util.Map; public class StreamingInsertJob extends AbstractJob> implements - TxnStateChangeCallback { + TxnStateChangeCallback, GsonPostProcessable { private final long dbId; private StreamingJobStatistic jobStatistic = new StreamingJobStatistic(); @SerializedName("fm") @@ -79,10 +80,16 @@ public class StreamingInsertJob extends AbstractJob createTasks(TaskType taskType, Map rfiles = new ArrayList<>(); + String startFile = currentOffset == null ? null : currentOffset.endFile; try (RemoteFileSystem fileSystem = FileSystemFactory.get(storageProperties)) { - maxRemoteEndFile = fileSystem.globListWithLimit(filePath, rfiles, currentOffset.endFile, + maxRemoteEndFile = fileSystem.globListWithLimit(filePath, rfiles, startFile, jobProperties.getS3BatchFiles(), jobProperties.getS3BatchSize()); - offset.setStartFile(currentOffset.endFile); + offset.setStartFile(startFile); offset.setEndFile(rfiles.get(rfiles.size() - 1)); offset.setFileLists(rfiles); } catch (Exception e) { @@ -90,15 +92,14 @@ public Offset getCurrentOffset() { } @Override - public InsertIntoTableCommand rewriteTvfParams(Offset nextOffset) { - S3Offset offset = (S3Offset) nextOffset; + public InsertIntoTableCommand rewriteTvfParams(Offset runningOffset) { + S3Offset offset = (S3Offset) runningOffset; Map props = new HashMap<>(); - //todo: need to change file list to glob string - props.put("uri", offset.getFileLists().toString()); - + String finalUri = "{" + String.join(",", offset.getFileLists()) + "}"; + props.put("uri", finalUri); InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(executeSql); - - command.rewriteFirstTvfProperties(getSourceType(), props); + //todo: command query plan is immutable + //command.rewriteFirstTvfProperties(getSourceType(), props); return command; } @@ -114,6 +115,9 @@ public void fetchRemoteMeta() { @Override public boolean hasMoreDataToConsume() { + if (currentOffset == null) { + return true; + } if (currentOffset.endFile.compareTo(maxRemoteEndFile) < 0) { return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java index 49445c46e0df53..ab03cce7c9195c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java @@ -24,6 +24,7 @@ import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.disruptor.TaskDisruptor; import org.apache.doris.job.exception.JobException; @@ -109,7 +110,7 @@ private void batchSchedulerTimerJob() { } public void scheduleOneJob(T job) throws JobException { - if (!job.getJobStatus().equals(JobStatus.RUNNING)) { + if (!JobUtils.checkNeedSchedule(job)) { return; } // not-schedule task @@ -144,7 +145,7 @@ public void scheduleOneJob(T job) throws JobException { } public void cycleTimerJobScheduler(T job) { - if (!job.getJobStatus().equals(JobStatus.RUNNING)) { + if (!JobUtils.checkNeedSchedule(job)) { return; } if (!JobExecuteType.RECURRING.equals(job.getJobConfig().getExecuteType())) { @@ -225,7 +226,7 @@ private void executeTimerJobIdsWithinLastTenMinutesWindow() { clearEndJob(job); continue; } - if (job.getJobStatus().equals(JobStatus.RUNNING) && job.getJobConfig().checkIsTimerJob()) { + if (JobUtils.checkNeedSchedule(job) && job.getJobConfig().checkIsTimerJob()) { cycleTimerJobScheduler(job, lastTimeWindowMs); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java index bdc27e983e17d9..3562ed028d21b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -97,16 +97,17 @@ private void scheduleOneTask(StreamingInsertTask task) throws JobException { task.getTaskId(), task.getJobId()); return; } - if (job.hasMoreDataToConsume()) { + if (!job.hasMoreDataToConsume()) { scheduleTaskWithDelay(task, 500); return; } - if (job.needDelayScheduleTask()) { + if (job.getLastScheduleTaskTimestamp() != -1 && job.needDelayScheduleTask()) { scheduleTaskWithDelay(task, 500); return; } log.info("prepare to schedule task, task id: {}, job id: {}", task.getTaskId(), task.getJobId()); task.execute(); + job.setLastScheduleTaskTimestamp(System.currentTimeMillis()); } private void scheduleTaskWithDelay(StreamingInsertTask task, long delayMs) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 61a8eb0a5c06ea..51d0dddc7cb664 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -130,6 +130,7 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti executeType = JobExecuteType.STREAMING; properties = new StreamingJobProperties(jobProperties); properties.validate(); + jobExecutionConfiguration.setImmediate(true); } jobExecutionConfiguration.setExecuteType(executeType); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 8bf31d941df1d5..04a1db682c5e38 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -565,9 +565,6 @@ private UnboundTVFRelation getFirstTvfInPlan(LogicalPlan plan) { public void rewriteFirstTvfProperties(String functionName, Map props) { AtomicBoolean found = new AtomicBoolean(false); rewriteFirstTvfInPlan(originLogicalQuery, functionName, props, found); - if (logicalQuery.isPresent() && !found.get()) { - rewriteFirstTvfInPlan(logicalQuery.get(), functionName, props, found); - } } private void rewriteFirstTvfInPlan(LogicalPlan plan, From ad763ce8e57a1f790d78117ca0756f79a4901418 Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 12 Sep 2025 22:15:05 +0800 Subject: [PATCH 08/25] [Feature](WIP) Add fetch meta and fix rewrite tvf problem (#55986) ### What problem does this PR solve? Add fetch meta and fix rewrite tvf problem --- .../insert/streaming/StreamingInsertJob.java | 64 +++++++++---- .../insert/streaming/StreamingInsertTask.java | 32 +++---- .../job/offset/SourceOffsetProvider.java | 25 +++-- .../job/offset/s3/S3SourceOffsetProvider.java | 91 ++++++++++++------- .../insert/InsertIntoTableCommand.java | 47 ++-------- 5 files changed, 146 insertions(+), 113 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 84e42c6e1f62bc..a4e05b650e35c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -56,6 +56,7 @@ import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; +import lombok.extern.log4j.Log4j2; import org.apache.commons.collections.CollectionUtils; import java.io.DataOutput; @@ -65,6 +66,7 @@ import java.util.List; import java.util.Map; +@Log4j2 public class StreamingInsertJob extends AbstractJob> implements TxnStateChangeCallback, GsonPostProcessable { private final long dbId; @@ -83,14 +85,16 @@ public class StreamingInsertJob extends AbstractJob originTvfProps; @Getter StreamingInsertTask runningStreamTask; SourceOffsetProvider offsetProvider; @Setter @Getter private long lastScheduleTaskTimestamp = -1L; + private InsertIntoTableCommand baseCommand; public StreamingInsertJob(String jobName, JobStatus jobStatus, @@ -105,17 +109,28 @@ public StreamingInsertJob(String jobName, jobConfig, createTimeMs, executeSql); this.dbId = ConnectContext.get().getCurrentDbId(); this.jobProperties = jobProperties; - this.tvfType = parseTvfType(); - this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); - this.offsetProvider.init(getExecuteSql(), jobProperties); + init(); } - private String parseTvfType() { - NereidsParser parser = new NereidsParser(); - InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(getExecuteSql()); - UnboundTVFRelation firstTVF = command.getFirstTVF(); - Preconditions.checkNotNull(firstTVF, "Only support insert sql with tvf"); - return firstTVF.getFunctionName(); + private void init() { + try { + UnboundTVFRelation currentTvf = getCurrentTvf(); + this.originTvfProps = currentTvf.getProperties().getMap(); + this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(currentTvf.getFunctionName()); + } catch (Exception ex) { + log.warn("init streaming insert job failed, sql: {}", getExecuteSql(), ex); + throw new RuntimeException("init streaming insert job failed, sql: " + getExecuteSql(), ex); + } + } + + private UnboundTVFRelation getCurrentTvf() throws Exception { + if (baseCommand == null) { + this.baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(getExecuteSql()); + } + List allTVFRelation = baseCommand.getAllTVFRelation(); + Preconditions.checkArgument(allTVFRelation.size() == 1, "Only support one source in insert streaming job"); + UnboundTVFRelation unboundTVFRelation = allTVFRelation.get(0); + return unboundTVFRelation; } @Override @@ -155,7 +170,14 @@ protected StreamingInsertTask createStreamingInsertTask() { } protected void fetchMeta() { - offsetProvider.fetchRemoteMeta(); + try { + if (originTvfProps == null) { + this.originTvfProps = getCurrentTvf().getProperties().getMap(); + } + offsetProvider.fetchRemoteMeta(originTvfProps); + } catch (Exception ex) { + log.warn("fetch remote meta failed, job id: {}", getJobId(), ex); + } } public boolean needScheduleTask() { @@ -228,8 +250,19 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getFailedTaskCount().get()))); trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getCanceledTaskCount().get()))); trow.addToColumnValue(new TCell().setStringVal(getComment())); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); - trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + + if (offsetProvider != null && offsetProvider.getSyncOffset() != null) { + trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getSyncOffset())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + if (offsetProvider != null && offsetProvider.getRemoteOffset() != null) { + trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getRemoteOffset())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + + trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getRemoteOffset())); trow.addToColumnValue(new TCell().setStringVal( jobStatistic == null ? FeConstants.null_string : jobStatistic.toJson())); trow.addToColumnValue(new TCell().setStringVal(failMsg == null ? FeConstants.null_string : failMsg.getMsg())); @@ -324,9 +357,8 @@ public void replayOnVisible(TransactionState txnState) { @Override public void gsonPostProcess() throws IOException { - if (offsetProvider == null && jobProperties != null && tvfType != null) { - this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); - // this.offsetProvider.init(getExecuteSql(), jobProperties); + if (offsetProvider == null) { + offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index 1227624fc8f5cc..73ac267e678fe9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -57,15 +57,15 @@ public class StreamingInsertTask { private Long startTimeMs; private Long finishTimeMs; private String sql; - private SourceOffsetProvider offsetProvider; private StmtExecutor stmtExecutor; - private InsertIntoTableCommand command; + private InsertIntoTableCommand taskCommand; private String currentDb; private UserIdentity userIdentity; private ConnectContext ctx; private Offset runningOffset; private AtomicBoolean isCanceled = new AtomicBoolean(false); private StreamingJobProperties jobProperties; + SourceOffsetProvider offsetProvider; public StreamingInsertTask(long jobId, long taskId, @@ -73,13 +73,13 @@ public StreamingInsertTask(long jobId, SourceOffsetProvider offsetProvider, String currentDb, StreamingJobProperties jobProperties, - UserIdentity userIdentity) { + UserIdentity userIdentity) { this.jobId = jobId; this.taskId = taskId; this.sql = sql; - this.offsetProvider = offsetProvider; this.userIdentity = userIdentity; this.currentDb = currentDb; + this.offsetProvider = offsetProvider; this.jobProperties = jobProperties; this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); this.createTimeMs = System.currentTimeMillis(); @@ -106,7 +106,7 @@ public void execute() throws JobException { } } - private void before() throws JobException { + private void before() throws Exception { this.status = TaskStatus.RUNNING; this.startTimeMs = System.currentTimeMillis(); @@ -117,12 +117,12 @@ private void before() throws JobException { ctx.setSessionVariable(jobProperties.getSessionVariable()); StatementContext statementContext = new StatementContext(); ctx.setStatementContext(statementContext); - offsetProvider.init(sql, jobProperties); - this.runningOffset = offsetProvider.getNextOffset(); - this.command = offsetProvider.rewriteTvfParams(runningOffset); - this.command.setLabelName(Optional.of(getJobId() + LABEL_SPLITTER + getTaskId())); - this.command.setJobId(getTaskId()); - stmtExecutor = new StmtExecutor(ctx, new LogicalPlanAdapter(command, ctx.getStatementContext())); + + this.runningOffset = offsetProvider.getNextOffset(jobProperties, jobProperties.getProperties()); + this.taskCommand = offsetProvider.rewriteTvfParams(sql, runningOffset); + this.taskCommand.setLabelName(Optional.of(getJobId() + LABEL_SPLITTER + getTaskId())); + this.taskCommand.setJobId(getTaskId()); + this.stmtExecutor = new StmtExecutor(ctx, new LogicalPlanAdapter(taskCommand, ctx.getStatementContext())); } private void run() throws JobException { @@ -134,7 +134,7 @@ private void run() throws JobException { log.info("task has been canceled, task id is {}", getTaskId()); return; } - command.run(ctx, stmtExecutor); + taskCommand.run(ctx, stmtExecutor); if (ctx.getState().getStateType() == QueryState.MysqlStateType.OK) { return; } else { @@ -142,13 +142,13 @@ private void run() throws JobException { } log.error( "streaming insert failed with {}, reason {}, to retry", - command.getLabelName(), + taskCommand.getLabelName(), errMsg); if (retry == MAX_RETRY) { errMsg = "reached max retry times, failed with" + errMsg; } } catch (Exception e) { - log.warn("execute insert task error, label is {},offset is {}", command.getLabelName(), + log.warn("execute insert task error, label is {},offset is {}", taskCommand.getLabelName(), runningOffset.toJson(), e); errMsg = Util.getRootCauseMessage(e); } @@ -209,8 +209,8 @@ public void closeOrReleaseResources() { if (null != stmtExecutor) { stmtExecutor = null; } - if (null != command) { - command = null; + if (null != taskCommand) { + taskCommand = null; } if (null != ctx) { ctx = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index 49a272c664f629..d9b2264d5b683d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -20,16 +20,13 @@ import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import java.util.Map; + /** * Interface for managing offsets and metadata of a data source. */ public interface SourceOffsetProvider { - /** - * init - */ - void init(String executeSql, StreamingJobProperties jobProperties); - /** * Get source type, e.g. s3, kafka * @return @@ -40,7 +37,7 @@ public interface SourceOffsetProvider { * Get next offset to consume * @return */ - Offset getNextOffset(); + Offset getNextOffset(StreamingJobProperties jobProps, Map properties); /** * Get current offset @@ -48,12 +45,24 @@ public interface SourceOffsetProvider { */ Offset getCurrentOffset(); + /** + * Get sync offset to show + * @return + */ + String getSyncOffset(); + + /** + * Get remote offset + * @return + */ + String getRemoteOffset(); + /** * Rewrite the TVF parameters in the SQL based on the current offset. * @param nextOffset * @return rewritten InsertIntoTableCommand */ - InsertIntoTableCommand rewriteTvfParams(Offset nextOffset); + InsertIntoTableCommand rewriteTvfParams(String executeSql, Offset nextOffset) throws Exception; /** * Update the offset of the source. @@ -64,7 +73,7 @@ public interface SourceOffsetProvider { /** * Fetch remote meta information, such as listing files in S3 or getting latest offsets in Kafka. */ - void fetchRemoteMeta(); + void fetchRemoteMeta(Map properties) throws Exception; /** * Whether there is more data to consume diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index b858a1e6431af3..a95f4af65c94b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -17,51 +17,35 @@ package org.apache.doris.job.offset.s3; -import org.apache.doris.common.UserException; import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.fs.FileSystemFactory; import org.apache.doris.fs.remote.RemoteFileSystem; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; -import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.trees.expressions.Properties; +import org.apache.doris.nereids.trees.expressions.functions.table.S3; +import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; +import org.apache.doris.qe.ConnectContext; +import com.google.common.collect.ImmutableList; import lombok.extern.log4j.Log4j2; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; @Log4j2 public class S3SourceOffsetProvider implements SourceOffsetProvider { - String executeSql; S3Offset currentOffset; String maxRemoteEndFile; - StreamingJobProperties jobProperties; - NereidsParser parser; - String filePath; - StorageProperties storageProperties; - - @Override - public void init(String executeSql, StreamingJobProperties jobProperties) { - //todo: check is already init - this.executeSql = executeSql; - this.jobProperties = jobProperties; - this.parser = new NereidsParser(); - InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(executeSql); - UnboundTVFRelation firstTVF = command.getFirstTVF(); - Map properties = firstTVF.getProperties().getMap(); - try { - this.storageProperties = StorageProperties.createPrimary(properties); - String uri = storageProperties.validateAndGetUri(properties); - this.filePath = storageProperties.validateAndNormalizeUri(uri); - } catch (UserException e) { - throw new RuntimeException("Failed check storage props, " + e.getMessage(), e); - } - } + InsertIntoTableCommand baseCommand; @Override public String getSourceType() { @@ -69,13 +53,17 @@ public String getSourceType() { } @Override - public S3Offset getNextOffset() { + public S3Offset getNextOffset(StreamingJobProperties jobProps, Map properties) { S3Offset offset = new S3Offset(); List rfiles = new ArrayList<>(); String startFile = currentOffset == null ? null : currentOffset.endFile; + String filePath = null; + StorageProperties storageProperties = StorageProperties.createPrimary(properties); try (RemoteFileSystem fileSystem = FileSystemFactory.get(storageProperties)) { + String uri = storageProperties.validateAndGetUri(properties); + filePath = storageProperties.validateAndNormalizeUri(uri); maxRemoteEndFile = fileSystem.globListWithLimit(filePath, rfiles, startFile, - jobProperties.getS3BatchFiles(), jobProperties.getS3BatchSize()); + jobProps.getS3BatchFiles(), jobProps.getS3BatchSize()); offset.setStartFile(startFile); offset.setEndFile(rfiles.get(rfiles.size() - 1)); offset.setFileLists(rfiles); @@ -92,15 +80,41 @@ public Offset getCurrentOffset() { } @Override - public InsertIntoTableCommand rewriteTvfParams(Offset runningOffset) { + public String getSyncOffset() { + if (currentOffset != null) { + return currentOffset.getEndFile(); + } + return null; + } + + @Override + public String getRemoteOffset() { + return maxRemoteEndFile; + } + + @Override + public InsertIntoTableCommand rewriteTvfParams(String executeSql, Offset runningOffset) throws Exception { S3Offset offset = (S3Offset) runningOffset; Map props = new HashMap<>(); String finalUri = "{" + String.join(",", offset.getFileLists()) + "}"; props.put("uri", finalUri); - InsertIntoTableCommand command = (InsertIntoTableCommand) parser.parseSingle(executeSql); - //todo: command query plan is immutable - //command.rewriteFirstTvfProperties(getSourceType(), props); - return command; + if (baseCommand == null) { + this.baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(executeSql); + this.baseCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); + } + + // rewrite plan + Plan rewritePlan = baseCommand.getLogicalQuery().rewriteUp(plan -> { + if (plan instanceof LogicalTVFRelation) { + LogicalTVFRelation originTvfRel = (LogicalTVFRelation) plan; + LogicalTVFRelation newRvfRel = new LogicalTVFRelation( + originTvfRel.getRelationId(), new S3(new Properties(props)), ImmutableList.of()); + return newRvfRel; + } + return plan; + }); + return new InsertIntoTableCommand((LogicalPlan) rewritePlan, Optional.empty(), Optional.empty(), + Optional.empty(), true, Optional.empty()); } @Override @@ -109,8 +123,17 @@ public void updateOffset(Offset offset) { } @Override - public void fetchRemoteMeta() { - // list object + public void fetchRemoteMeta(Map properties) throws Exception { + StorageProperties storageProperties = StorageProperties.createPrimary(properties); + String startFile = currentOffset == null ? null : currentOffset.endFile; + try (RemoteFileSystem fileSystem = FileSystemFactory.get(storageProperties)) { + String uri = storageProperties.validateAndGetUri(properties); + String filePath = storageProperties.validateAndNormalizeUri(uri); + maxRemoteEndFile = fileSystem.globListWithLimit(filePath, new ArrayList<>(), startFile, + 1, 1); + } catch (Exception e) { + throw e; + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 04a1db682c5e38..6736b16f8afeb0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -79,11 +79,10 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Supplier; @@ -540,51 +539,21 @@ public List getTargetColumns() { } // todo: add ut - public UnboundTVFRelation getFirstTVF() { - return getFirstTvfInPlan(getLogicalQuery()); + public List getAllTVFRelation() { + List tvfs = new ArrayList<>(); + findAllTVFInPlan(getLogicalQuery(), tvfs); + return tvfs; } - private UnboundTVFRelation getFirstTvfInPlan(LogicalPlan plan) { + private void findAllTVFInPlan(LogicalPlan plan, List tvfs) { if (plan instanceof UnboundTVFRelation) { UnboundTVFRelation tvfRelation = (UnboundTVFRelation) plan; - return tvfRelation; + tvfs.add(tvfRelation); } for (Plan child : plan.children()) { if (child instanceof LogicalPlan) { - UnboundTVFRelation result = getFirstTvfInPlan((LogicalPlan) child); - if (result != null) { - return result; - } - } - } - return null; - } - - // todo: add ut - public void rewriteFirstTvfProperties(String functionName, Map props) { - AtomicBoolean found = new AtomicBoolean(false); - rewriteFirstTvfInPlan(originLogicalQuery, functionName, props, found); - } - - private void rewriteFirstTvfInPlan(LogicalPlan plan, - String functionName, Map props, AtomicBoolean found) { - if (found.get()) { - return; - } - - if (plan instanceof UnboundTVFRelation) { - UnboundTVFRelation tvfRelation = (UnboundTVFRelation) plan; - if (functionName.equalsIgnoreCase(tvfRelation.getFunctionName())) { - tvfRelation.getProperties().getMap().putAll(props); - found.set(true); - return; - } - } - - for (Plan child : plan.children()) { - if (child instanceof LogicalPlan) { - rewriteFirstTvfInPlan((LogicalPlan) child, functionName, props, found); + findAllTVFInPlan((LogicalPlan) child, tvfs); } } } From ec65dcbd5a36d816942669b4ae1bcbb44922828a Mon Sep 17 00:00:00 2001 From: hui lai Date: Sun, 14 Sep 2025 11:51:49 +0800 Subject: [PATCH 09/25] [feat](streaming job) implement offset persistence and replay in cloud mode (#55975) ### What problem does this PR solve? Implement offset persistence and replay in cloud mode. --- cloud/src/common/bvars.cpp | 5 + cloud/src/common/bvars.h | 3 + cloud/src/meta-service/meta_service.h | 13 ++ cloud/src/meta-service/meta_service_txn.cpp | 137 ++++++++++++++++++ cloud/src/meta-store/keys.cpp | 13 +- cloud/src/meta-store/keys.h | 5 + .../doris/cloud/rpc/MetaServiceClient.java | 6 + .../doris/cloud/rpc/MetaServiceProxy.java | 5 + .../CloudGlobalTransactionMgr.java | 14 ++ .../doris/cloud/transaction/TxnUtil.java | 38 +++++ .../insert/streaming/StreamingInsertJob.java | 49 +++++++ .../StreamingTaskTxnCommitAttachment.java | 22 ++- .../org/apache/doris/job/offset/Offset.java | 4 + .../apache/doris/job/offset/s3/S3Offset.java | 10 ++ .../doris/transaction/TransactionState.java | 5 +- gensrc/proto/cloud.proto | 31 +++- 16 files changed, 353 insertions(+), 7 deletions(-) diff --git a/cloud/src/common/bvars.cpp b/cloud/src/common/bvars.cpp index e495b5ea95b956..8ea24c4a1baade 100644 --- a/cloud/src/common/bvars.cpp +++ b/cloud/src/common/bvars.cpp @@ -85,6 +85,7 @@ BvarLatencyRecorderWithTag g_bvar_ms_remove_delete_bitmap("ms", "remove_delete_b BvarLatencyRecorderWithTag g_bvar_ms_remove_delete_bitmap_update_lock("ms", "remove_delete_bitmap_update_lock"); BvarLatencyRecorderWithTag g_bvar_ms_get_instance("ms", "get_instance"); BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach("ms", "get_rl_task_commit_attach"); +BvarLatencyRecorderWithTag g_bvar_ms_get_streaming_task_commit_attach("ms", "get_streaming_task_commit_attach"); BvarLatencyRecorderWithTag g_bvar_ms_reset_rl_progress("ms", "reset_rl_progress"); BvarLatencyRecorderWithTag g_bvar_ms_get_txn_id("ms", "get_txn_id"); BvarLatencyRecorderWithTag g_bvar_ms_start_tablet_job("ms", "start_tablet_job"); @@ -364,6 +365,8 @@ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter("rpc_kv_precommit_txn_ge mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter("rpc_kv_precommit_txn_put_counter",{"instance_id"}); // get_rl_task_commit_attach mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter("rpc_kv_get_rl_task_commit_attach_get_counter",{"instance_id"}); +// get_streaming_task_commit_attach +mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_counter("rpc_kv_get_streaming_task_commit_attach_get_counter",{"instance_id"}); // reset_rl_progress mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter("rpc_kv_reset_rl_progress_get_counter",{"instance_id"}); mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter("rpc_kv_reset_rl_progress_put_counter",{"instance_id"}); @@ -527,6 +530,8 @@ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_bytes("rpc_kv_precommit_txn_get_ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_bytes("rpc_kv_precommit_txn_put_bytes",{"instance_id"}); // get_rl_task_commit_attach mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_bytes("rpc_kv_get_rl_task_commit_attach_get_bytes",{"instance_id"}); +// get_streaming_task_commit_attach +mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_bytes("rpc_kv_get_streaming_task_commit_attach_get_bytes",{"instance_id"}); // reset_rl_progress mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_bytes("rpc_kv_reset_rl_progress_get_bytes",{"instance_id"}); mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_bytes("rpc_kv_reset_rl_progress_put_bytes",{"instance_id"}); diff --git a/cloud/src/common/bvars.h b/cloud/src/common/bvars.h index 8fb5973249fd2d..e72d0fcd3760d1 100644 --- a/cloud/src/common/bvars.h +++ b/cloud/src/common/bvars.h @@ -251,6 +251,7 @@ extern BvarLatencyRecorderWithTag g_bvar_ms_get_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_get_instance; extern BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_streaming_task_commit_attach; extern BvarLatencyRecorderWithTag g_bvar_ms_reset_rl_progress; extern BvarLatencyRecorderWithTag g_bvar_ms_get_txn_id; extern BvarLatencyRecorderWithTag g_bvar_ms_check_kv; @@ -470,6 +471,7 @@ extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_counter; @@ -582,6 +584,7 @@ extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_bytes; +extern mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_bytes; diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 38cc77aebb00af..c7a37277fdc083 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -319,6 +319,11 @@ class MetaServiceImpl : public cloud::MetaService { GetRLTaskCommitAttachResponse* response, ::google::protobuf::Closure* done) override; + void get_streaming_task_commit_attach(::google::protobuf::RpcController* controller, + const GetStreamingTaskCommitAttachRequest* request, + GetStreamingTaskCommitAttachResponse* response, + ::google::protobuf::Closure* done) override; + void reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, ::google::protobuf::Closure* done) override; @@ -820,6 +825,14 @@ class MetaServiceProxy final : public MetaService { done); } + void get_streaming_task_commit_attach(::google::protobuf::RpcController* controller, + const GetStreamingTaskCommitAttachRequest* request, + GetStreamingTaskCommitAttachResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_streaming_task_commit_attach, controller, request, + response, done); + } + void reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, ::google::protobuf::Closure* done) override { diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index ce128f6d30e485..bd4cf6fb036f98 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -610,6 +610,75 @@ void put_routine_load_progress(MetaServiceCode& code, std::string& msg, << " routine load new progress: " << new_progress_info.ShortDebugString(); } +void put_streaming_job_meta(MetaServiceCode& code, std::string& msg, const std::string& instance_id, + const CommitTxnRequest* request, Transaction* txn, int64_t db_id) { + std::stringstream ss; + int64_t txn_id = request->txn_id(); + if (!request->has_commit_attachment()) { + ss << "failed to get commit attachment from req, db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + TxnCommitAttachmentPB txn_commit_attachment = request->commit_attachment(); + StreamingTaskCommitAttachmentPB commit_attachment = + txn_commit_attachment.streaming_task_txn_commit_attachment(); + int64_t job_id = commit_attachment.job_id(); + + std::string streaming_meta_key; + std::string streaming_meta_val; + bool prev_meta_existed = true; + StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; + streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); + TxnErrorCode err = txn->get(streaming_meta_key, &streaming_meta_val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + prev_meta_existed = false; + } else { + code = cast_as(err); + ss << "failed to get streaming job meta, db_id=" << db_id << " txn_id=" << txn_id + << " err=" << err; + msg = ss.str(); + return; + } + } + + StreamingTaskCommitAttachmentPB new_meta_info; + if (prev_meta_existed) { + if (!new_meta_info.ParseFromString(streaming_meta_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse streaming job meta, db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + new_meta_info.set_scanned_rows(new_meta_info.scanned_rows() + + commit_attachment.scanned_rows()); + new_meta_info.set_load_bytes(new_meta_info.load_bytes() + commit_attachment.load_bytes()); + new_meta_info.set_file_number(new_meta_info.file_number() + + commit_attachment.file_number()); + new_meta_info.set_file_size(new_meta_info.file_size() + commit_attachment.file_size()); + } else { + new_meta_info.set_job_id(commit_attachment.job_id()); + new_meta_info.set_scanned_rows(commit_attachment.scanned_rows()); + new_meta_info.set_load_bytes(commit_attachment.load_bytes()); + new_meta_info.set_file_number(commit_attachment.file_number()); + new_meta_info.set_file_size(commit_attachment.file_size()); + } + if (commit_attachment.has_offset()) { + new_meta_info.set_offset(commit_attachment.offset()); + } + std::string new_meta_val; + if (!new_meta_info.SerializeToString(&new_meta_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize new streaming meta val, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + txn->put(streaming_meta_key, new_meta_val); + LOG(INFO) << "put streaming_meta_key key=" << hex(streaming_meta_key) + << " streaming job new meta: " << new_meta_info.ShortDebugString(); +} + void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcController* controller, const GetRLTaskCommitAttachRequest* request, GetRLTaskCommitAttachResponse* response, @@ -678,6 +747,64 @@ void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcControlle } } +void MetaServiceImpl::get_streaming_task_commit_attach( + ::google::protobuf::RpcController* controller, + const GetStreamingTaskCommitAttachRequest* request, + GetStreamingTaskCommitAttachResponse* response, ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_streaming_task_commit_attach, get); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_streaming_task_commit_attach) + + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "filed to create txn, err=" << err; + msg = ss.str(); + return; + } + + if (!request->has_db_id() || !request->has_job_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty db_id or job_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + + int64_t db_id = request->db_id(); + int64_t job_id = request->job_id(); + std::string streaming_meta_key; + std::string streaming_meta_val; + StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; + streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); + err = txn->get(streaming_meta_key, &streaming_meta_val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::STREAMING_JOB_PROGRESS_NOT_FOUND; + ss << "progress info not found, db_id=" << db_id << " job_id=" << job_id << " err=" << err; + msg = ss.str(); + return; + } else if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get progress info, db_id=" << db_id << " job_id=" << job_id + << " err=" << err; + msg = ss.str(); + return; + } + + StreamingTaskCommitAttachmentPB* commit_attach = response->mutable_commit_attach(); + if (!commit_attach->ParseFromString(streaming_meta_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse meta info, db_id=" << db_id << " job_id=" << job_id; + msg = ss.str(); + return; + } +} + void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, @@ -1572,6 +1699,11 @@ void MetaServiceImpl::commit_txn_immediately( put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } + if (txn_info.load_job_source_type() == + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { + put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + } + LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) << " txn_id=" << txn_id; LOG(INFO) << "commit_txn put_size=" << txn->put_bytes() @@ -1965,6 +2097,11 @@ void MetaServiceImpl::commit_txn_eventually( put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } + if (txn_info.load_job_source_type() == + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { + put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + } + // save versions for partition int64_t version_update_time_ms = duration_cast(system_clock::now().time_since_epoch()).count(); diff --git a/cloud/src/meta-store/keys.cpp b/cloud/src/meta-store/keys.cpp index e23f84771aba04..7b2b75c4d555eb 100644 --- a/cloud/src/meta-store/keys.cpp +++ b/cloud/src/meta-store/keys.cpp @@ -64,6 +64,7 @@ static const char* STATS_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_RL_PROGRESS = "routine_load_progress"; +static const char* JOB_KEY_INFIX_STREAMING_JOB_META = "streaming_job_meta"; static const char* JOB_KEY_INFIX_RESTORE_TABLET = "restore_tablet"; static const char* JOB_KEY_INFIX_RESTORE_ROWSET = "restore_rowset"; @@ -144,7 +145,7 @@ static void encode_prefix(const T& t, std::string* key) { MetaDeleteBitmapInfo, MetaDeleteBitmapUpdateLockInfo, MetaPendingDeleteBitmapInfo, PartitionVersionKeyInfo, RecycleIndexKeyInfo, RecyclePartKeyInfo, RecycleRowsetKeyInfo, RecycleTxnKeyInfo, RecycleStageKeyInfo, StatsTabletKeyInfo, TableVersionKeyInfo, JobRestoreTabletKeyInfo, JobRestoreRowsetKeyInfo, - JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, + JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, StreamingJobMetaKeyInfo, CopyJobKeyInfo, CopyFileKeyInfo, StorageVaultKeyInfo, MetaSchemaPBDictionaryInfo, MowTabletJobInfo>); @@ -181,7 +182,8 @@ static void encode_prefix(const T& t, std::string* key) { encode_bytes(STATS_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v - || std::is_same_v) { + || std::is_same_v + || std::is_same_v) { encode_bytes(JOB_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v) { @@ -463,6 +465,13 @@ void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out) encode_int64(std::get<2>(in), out); // job_id } +void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "job" ${instance_id} + encode_bytes(JOB_KEY_INFIX_STREAMING_JOB_META, out); // "streaming_job_meta" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // job_id +} + void job_restore_tablet_key(const JobRestoreTabletKeyInfo& in, std::string* out) { encode_prefix(in, out); // 0x01 "job" ${instance_id} encode_bytes(JOB_KEY_INFIX_RESTORE_TABLET, out); // "restore_tablet" diff --git a/cloud/src/meta-store/keys.h b/cloud/src/meta-store/keys.h index 8ccd974e0b757a..3b9b234c57453a 100644 --- a/cloud/src/meta-store/keys.h +++ b/cloud/src/meta-store/keys.h @@ -218,6 +218,9 @@ using MetaPendingDeleteBitmapInfo = BasicKeyInfo<24 , std::tuple>; +// 0:instance_id 1:db_id 2:job_id +using StreamingJobMetaKeyInfo = BasicKeyInfo<52, std::tuple>; + // 0:instance_id 1:vault_id using StorageVaultKeyInfo = BasicKeyInfo<26, std::tuple>; @@ -407,6 +410,8 @@ void job_tablet_key(const JobTabletKeyInfo& in, std::string* out); static inline std::string job_tablet_key(const JobTabletKeyInfo& in) { std::string s; job_tablet_key(in, &s); return s; } void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out); static inline std::string rl_job_progress_key_info(const RLJobProgressKeyInfo& in) { std::string s; rl_job_progress_key_info(in, &s); return s; } +void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out); +static inline std::string streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in) { std::string s; streaming_job_meta_key_info(in, &s); return s; } std::string copy_key_prefix(std::string_view instance_id); void copy_job_key(const CopyJobKeyInfo& in, std::string* out); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java index f17625a89eaaf2..2b0673d6453d13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java @@ -492,4 +492,10 @@ public Cloud. ResetRLProgressResponse resetRLProgress(Cloud. ResetRLProgressRequ return blockingStub.withDeadlineAfter(Config.meta_service_brpc_timeout_ms, TimeUnit.MILLISECONDS) .createInstance(request); } + + public Cloud.GetStreamingTaskCommitAttachResponse + getStreamingTaskCommitAttach(Cloud.GetStreamingTaskCommitAttachRequest request) { + return blockingStub.withDeadlineAfter(Config.meta_service_brpc_timeout_ms, TimeUnit.MILLISECONDS) + .getStreamingTaskCommitAttach(request); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java index b351942cbe2d11..8710209ff8abab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java @@ -432,4 +432,9 @@ public Cloud.ResetRLProgressResponse resetRLProgress(Cloud.ResetRLProgressReques public Cloud.CreateInstanceResponse createInstance(Cloud.CreateInstanceRequest request) throws RpcException { return w.executeRequest((client) -> client.createInstance(request)); } + + public Cloud.GetStreamingTaskCommitAttachResponse getStreamingTaskCommitAttach( + Cloud.GetStreamingTaskCommitAttachRequest request) throws RpcException { + return w.executeRequest((client) -> client.getStreamingTaskCommitAttach(request)); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index fddb6ed720887b..c3b9c321fa903a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -85,6 +85,7 @@ import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.event.DataChangeEvent; +import org.apache.doris.job.extensions.insert.streaming.StreamingTaskTxnCommitAttachment; import org.apache.doris.load.loadv2.LoadJobFinalOperation; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; import org.apache.doris.metric.MetricRepo; @@ -619,6 +620,19 @@ private void commitTransactionWithoutLock(long dbId, List tableList, long } builder.setCommitAttachment(TxnUtil .rlTaskTxnCommitAttachmentToPb(rlTaskTxnCommitAttachment)); + } else if (txnCommitAttachment instanceof StreamingTaskTxnCommitAttachment) { + StreamingTaskTxnCommitAttachment streamingTaskTxnCommitAttachment = + (StreamingTaskTxnCommitAttachment) txnCommitAttachment; + TxnStateChangeCallback cb = callbackFactory.getCallback(streamingTaskTxnCommitAttachment.getTaskId()); + if (cb != null) { + // use a temporary transaction state to do before commit check, + // what actually works is the transactionId + TransactionState tmpTxnState = new TransactionState(); + tmpTxnState.setTransactionId(transactionId); + cb.beforeCommitted(tmpTxnState); + } + builder.setCommitAttachment(TxnUtil + .streamingTaskTxnCommitAttachmentToPb(streamingTaskTxnCommitAttachment)); } else { throw new UserException("invalid txnCommitAttachment"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java index 3aca54cd150dc6..4155e6c5e67ab7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java @@ -19,6 +19,7 @@ import org.apache.doris.cloud.proto.Cloud.RLTaskTxnCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.RoutineLoadProgressPB; +import org.apache.doris.cloud.proto.Cloud.StreamingTaskCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB.EtlStatusPB; @@ -28,6 +29,7 @@ import org.apache.doris.cloud.proto.Cloud.TxnInfoPB; import org.apache.doris.cloud.proto.Cloud.TxnSourceTypePB; import org.apache.doris.cloud.proto.Cloud.UniqueIdPB; +import org.apache.doris.job.extensions.insert.streaming.StreamingTaskTxnCommitAttachment; import org.apache.doris.load.EtlStatus; import org.apache.doris.load.FailMsg; import org.apache.doris.load.loadv2.JobState; @@ -268,6 +270,42 @@ public static RLTaskTxnCommitAttachment rtTaskTxnCommitAttachmentFromPb( return new RLTaskTxnCommitAttachment(txnCommitAttachmentPB.getRlTaskTxnCommitAttachment()); } + public static TxnCommitAttachmentPB streamingTaskTxnCommitAttachmentToPb(StreamingTaskTxnCommitAttachment + streamingTaskTxnCommitAttachment) { + if (LOG.isDebugEnabled()) { + LOG.debug("streamingTaskTxnCommitAttachment:{}", streamingTaskTxnCommitAttachment); + } + TxnCommitAttachmentPB.Builder attachementBuilder = TxnCommitAttachmentPB.newBuilder(); + attachementBuilder.setType(TxnCommitAttachmentPB.Type.STREAMING_TASK_TXN_COMMIT_ATTACHMENT); + + StreamingTaskCommitAttachmentPB.Builder builder = + StreamingTaskCommitAttachmentPB.newBuilder(); + + builder.setJobId(streamingTaskTxnCommitAttachment.getJobId()) + .setTaskId(streamingTaskTxnCommitAttachment.getTaskId()) + .setScannedRows(streamingTaskTxnCommitAttachment.getScannedRows()) + .setLoadBytes(streamingTaskTxnCommitAttachment.getLoadBytes()) + .setFileNumber(streamingTaskTxnCommitAttachment.getFileNumber()) + .setFileSize(streamingTaskTxnCommitAttachment.getFileSize()); + + if (streamingTaskTxnCommitAttachment.getOffset() != null) { + builder.setOffset(streamingTaskTxnCommitAttachment.getOffset().endOffset()); + } + + attachementBuilder.setStreamingTaskTxnCommitAttachment(builder.build()); + return attachementBuilder.build(); + } + + public static StreamingTaskTxnCommitAttachment streamingTaskTxnCommitAttachmentFromPb( + TxnCommitAttachmentPB txnCommitAttachmentPB) { + StreamingTaskCommitAttachmentPB streamingTaskCommitAttachmentPB = + txnCommitAttachmentPB.getStreamingTaskTxnCommitAttachment(); + if (LOG.isDebugEnabled()) { + LOG.debug("StreamingTaskCommitAttachmentPB={}", streamingTaskCommitAttachmentPB); + } + return new StreamingTaskTxnCommitAttachment(streamingTaskCommitAttachmentPB); + } + public static LoadJobFinalOperation loadJobFinalOperationFromPb(TxnCommitAttachmentPB txnCommitAttachmentPB) { LoadJobFinalOperationPB loadJobFinalOperationPB = txnCommitAttachmentPB.getLoadJobFinalOperation(); if (LOG.isDebugEnabled()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index a4e05b650e35c7..b18ffac380ba1a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -19,6 +19,9 @@ import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.rpc.MetaServiceProxy; +import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; @@ -46,6 +49,7 @@ import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.transaction.TransactionException; @@ -136,6 +140,9 @@ private UnboundTVFRelation getCurrentTvf() throws Exception { @Override public void updateJobStatus(JobStatus status) throws JobException { super.updateJobStatus(status); + if (isFinalStatus()) { + Env.getCurrentGlobalTransactionMgr().getCallbackFactory().removeCallback(getJobId()); + } } @Override @@ -225,6 +232,17 @@ private void updateJobStatisticAndOffset(StreamingTaskTxnCommitAttachment attach offsetProvider.updateOffset(attachment.getOffset()); } + @Override + public void onRegister() throws JobException { + Env.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(this); + } + + @Override + public void onReplayCreate() throws JobException { + onRegister(); + super.onReplayCreate(); + } + @Override public ShowResultSetMetaData getTaskMetaData() { return InsertJob.TASK_META_DATA; @@ -305,6 +323,8 @@ public void beforeCommitted(TransactionState txnState) throws TransactionExcepti LoadJob loadJob = loadJobs.get(0); LoadStatistic loadStatistic = loadJob.getLoadStatistic(); txnState.setTxnCommitAttachment(new StreamingTaskTxnCommitAttachment( + getJobId(), + runningStreamTask.getTaskId(), loadStatistic.getScannedRows(), loadStatistic.getLoadBytes(), loadStatistic.getFileNumber(), @@ -333,6 +353,35 @@ public void replayOnCommitted(TransactionState txnState) { updateJobStatisticAndOffset(attachment); } + public void replayOnCloudMode() throws UserException { + Cloud.GetStreamingTaskCommitAttachRequest.Builder builder = + Cloud.GetStreamingTaskCommitAttachRequest.newBuilder(); + builder.setCloudUniqueId(Config.cloud_unique_id); + builder.setDbId(dbId); + builder.setJobId(getJobId()); + + Cloud.GetStreamingTaskCommitAttachResponse response; + try { + response = MetaServiceProxy.getInstance().getStreamingTaskCommitAttach(builder.build()); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + log.warn("failed to get streaming task commit attach, response: {}", response); + if (response.getStatus().getCode() == Cloud.MetaServiceCode.STREAMING_JOB_PROGRESS_NOT_FOUND) { + log.warn("not found streaming job progress, response: {}", response); + return; + } else { + throw new UserException(response.getStatus().getMsg()); + } + } + } catch (RpcException e) { + log.info("failed to get streaming task commit attach {}", e); + throw new UserException(e.getMessage()); + } + + StreamingTaskTxnCommitAttachment commitAttach = + new StreamingTaskTxnCommitAttachment(response.getCommitAttach()); + updateJobStatisticAndOffset(commitAttach); + } + @Override public void afterAborted(TransactionState txnState, boolean txnOperated, String txnStatusChangeReason) throws UserException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java index 8a8768dee34b22..8660ed94739c0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java @@ -17,6 +17,7 @@ package org.apache.doris.job.extensions.insert.streaming; +import org.apache.doris.cloud.proto.Cloud.StreamingTaskCommitAttachmentPB; import org.apache.doris.job.offset.Offset; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnCommitAttachment; @@ -26,9 +27,11 @@ public class StreamingTaskTxnCommitAttachment extends TxnCommitAttachment { - public StreamingTaskTxnCommitAttachment(long scannedRows, long loadBytes, - long fileNumber, long fileSize, Offset offset) { - super(TransactionState.LoadJobSourceType.BACKEND_STREAMING); + public StreamingTaskTxnCommitAttachment(long jobId, long taskId, + long scannedRows, long loadBytes, long fileNumber, long fileSize, Offset offset) { + super(TransactionState.LoadJobSourceType.STREAMING_JOB); + this.jobId = jobId; + this.taskId = taskId; this.scannedRows = scannedRows; this.loadBytes = loadBytes; this.fileNumber = fileNumber; @@ -36,6 +39,19 @@ public StreamingTaskTxnCommitAttachment(long scannedRows, long loadBytes, this.offset = offset; } + public StreamingTaskTxnCommitAttachment(StreamingTaskCommitAttachmentPB pb) { + super(TransactionState.LoadJobSourceType.STREAMING_JOB); + this.scannedRows = pb.getScannedRows(); + this.loadBytes = pb.getLoadBytes(); + this.fileNumber = pb.getFileNumber(); + this.fileSize = pb.getFileSize(); + this.offset.setEndOffset(pb.getOffset()); + } + + @Getter + private long jobId; + @Getter + private long taskId; @SerializedName(value = "sr") @Getter private long scannedRows; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java index 095f0a5e6bf2bc..a3b0689bfc5466 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java @@ -19,4 +19,8 @@ public interface Offset { String toJson(); + + void setEndOffset(String endOffset); + + String endOffset(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index 95271ba60ab1e6..dd6927935bb6a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -32,6 +32,16 @@ public class S3Offset implements Offset { String endFile; List fileLists; + @Override + public void setEndOffset(String endOffset) { + this.endFile = endOffset; + } + + @Override + public String endOffset() { + return endFile; + } + @Override public String toJson() { return GsonUtils.GSON.toJson(this); diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java index 53361106124564..8fc7cec7c2665b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionState.java @@ -75,7 +75,8 @@ public enum LoadJobSourceType { BACKEND_STREAMING(2), // streaming load use this type INSERT_STREAMING(3), // insert stmt (streaming type), update stmt use this type ROUTINE_LOAD_TASK(4), // routine load task use this type - BATCH_LOAD_JOB(5); // load job v2 for broker load + BATCH_LOAD_JOB(5), // load job v2 for broker load + STREAMING_JOB(6); // streaming job use this type @SerializedName("f") private final int flag; @@ -100,6 +101,8 @@ public static LoadJobSourceType valueOf(int flag) { return ROUTINE_LOAD_TASK; case 5: return BATCH_LOAD_JOB; + case 6: + return STREAMING_JOB; default: return null; } diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index e277c24f8d7b94..7c630f03353f98 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -322,6 +322,7 @@ enum LoadJobSourceTypePB { LOAD_JOB_SRC_TYPE_INSERT_STREAMING = 3; // insert stmt (streaming type), update stmt use this type LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK = 4; // routine load task use this type LOAD_JOB_SRC_TYPE_BATCH_LOAD_JOB = 5; // load job v2 for broker load + LOAD_JOB_SRC_TYPE_STREAMING_JOB = 6; // streaming job use this type } enum TxnStatusPB { @@ -365,10 +366,21 @@ message RoutineLoadJobStatisticPB { optional int64 task_execution_time_ms = 5; } +message StreamingTaskCommitAttachmentPB { + optional int64 job_id = 1; + optional UniqueIdPB task_id = 2; + optional string offset = 3; + optional int64 scanned_rows = 4; + optional int64 load_bytes = 5; + optional int64 file_number = 6; + optional int64 file_size = 7; +} + message TxnCommitAttachmentPB { enum Type { LODD_JOB_FINAL_OPERATION = 0; RT_TASK_TXN_COMMIT_ATTACHMENT = 1; + STREAMING_TASK_TXN_COMMIT_ATTACHMENT = 2; } message LoadJobFinalOperationPB { message EtlStatusPB { @@ -425,6 +437,7 @@ message TxnCommitAttachmentPB { optional Type type = 1; optional LoadJobFinalOperationPB load_job_final_operation = 2; optional RLTaskTxnCommitAttachmentPB rl_task_txn_commit_attachment = 3; + optional StreamingTaskCommitAttachmentPB streaming_task_txn_commit_attachment = 4; } // For storing label -> txn_ids @@ -1698,7 +1711,8 @@ enum MetaServiceCode { JOB_ALREADY_SUCCESS = 5002; ROUTINE_LOAD_DATA_INCONSISTENT = 5003; ROUTINE_LOAD_PROGRESS_NOT_FOUND = 5004; - JOB_CHECK_ALTER_VERSION = 5005; + STREAMING_JOB_PROGRESS_NOT_FOUND = 5005; + JOB_CHECK_ALTER_VERSION = 5006; // Rate limit MAX_QPS_LIMIT = 6001; @@ -1876,6 +1890,18 @@ message ResetRLProgressResponse { optional MetaServiceResponseStatus status = 1; } +message GetStreamingTaskCommitAttachRequest { + optional string cloud_unique_id = 1; // For auth + optional int64 db_id = 2; + optional int64 job_id = 3; + optional string request_ip = 4; +} + +message GetStreamingTaskCommitAttachResponse { + optional MetaServiceResponseStatus status = 1; + optional StreamingTaskCommitAttachmentPB commit_attach = 2; +} + message CheckKeyInfos { repeated int64 db_ids = 1; repeated int64 table_ids = 2; @@ -2069,6 +2095,9 @@ service MetaService { rpc get_rl_task_commit_attach(GetRLTaskCommitAttachRequest) returns (GetRLTaskCommitAttachResponse); rpc reset_rl_progress(ResetRLProgressRequest) returns (ResetRLProgressResponse); + // streaming job meta + rpc get_streaming_task_commit_attach(GetStreamingTaskCommitAttachRequest) returns (GetStreamingTaskCommitAttachResponse); + // check KV rpc check_kv(CheckKVRequest) returns (CheckKVResponse); From 434b2b3ca251306b16c0ab1ee2ea1bfa75b0913b Mon Sep 17 00:00:00 2001 From: hui lai Date: Mon, 15 Sep 2025 17:40:22 +0800 Subject: [PATCH 10/25] [fix](streaming job) register listener id when begin transaction (#56056) ### What problem does this PR solve? Register listener id when begin transaction to ensure before/after commit logic would be executed. --- .../insert/streaming/StreamingInsertJob.java | 4 +- .../apache/doris/job/manager/JobManager.java | 2 + .../job/manager/StreamingTaskManager.java | 55 +++++++++++++++++++ .../job/scheduler/StreamingTaskScheduler.java | 12 ++-- .../commands/insert/OlapInsertExecutor.java | 15 ++++- gensrc/proto/cloud.proto | 2 +- 6 files changed, 79 insertions(+), 11 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index b18ffac380ba1a..24df73cbbac56e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -171,7 +171,7 @@ public List createTasks(TaskType taskType, Map, C> implements Writable { private JobScheduler jobScheduler; + @Getter + private final StreamingTaskManager streamingTaskManager = new StreamingTaskManager(); @Getter private StreamingTaskScheduler streamingTaskScheduler; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java new file mode 100644 index 00000000000000..7c6fecee14a179 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.manager; + +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; + +import lombok.Getter; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; + +public class StreamingTaskManager { + @Getter + private final LinkedBlockingDeque needScheduleTasksQueue = new LinkedBlockingDeque<>(); + @Getter + private List runningTasks = Collections.synchronizedList(new ArrayList<>()); + + public void registerTask(StreamingInsertTask task) { + needScheduleTasksQueue.add(task); + } + + public StreamingInsertTask getStreamingInsertTaskById(long taskId) { + synchronized (runningTasks) { + return runningTasks.stream() + .filter(task -> task.getTaskId() == taskId) + .findFirst() + .orElse(null); + } + } + + public void addRunningTask(StreamingInsertTask task) { + runningTasks.add(task); + } + + public void removeRunningTask(StreamingInsertTask task) { + runningTasks.remove(task); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java index 3562ed028d21b2..3fbae399303c61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -37,7 +37,6 @@ @Log4j2 public class StreamingTaskScheduler extends MasterDaemon { - private final LinkedBlockingDeque needScheduleTasksQueue = new LinkedBlockingDeque<>(); private final ThreadPoolExecutor threadPool = new ThreadPoolExecutor( 0, Config.max_streaming_job_num, @@ -63,12 +62,10 @@ protected void runAfterCatalogReady() { } } - public void registerTask(StreamingInsertTask task) { - needScheduleTasksQueue.add(task); - } - private void process() throws InterruptedException { List tasks = new ArrayList<>(); + LinkedBlockingDeque needScheduleTasksQueue = + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().getNeedScheduleTasksQueue(); tasks.add(needScheduleTasksQueue.take()); needScheduleTasksQueue.drainTo(tasks); scheduleTasks(tasks); @@ -106,13 +103,14 @@ private void scheduleOneTask(StreamingInsertTask task) throws JobException { return; } log.info("prepare to schedule task, task id: {}, job id: {}", task.getTaskId(), task.getJobId()); - task.execute(); job.setLastScheduleTaskTimestamp(System.currentTimeMillis()); + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().addRunningTask(task); + task.execute(); } private void scheduleTaskWithDelay(StreamingInsertTask task, long delayMs) { delayScheduler.schedule(() -> { - needScheduleTasksQueue.add(task); + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().registerTask(task); }, delayMs, TimeUnit.MILLISECONDS); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java index 47ef9cc3c2ebca..f41d8893f020e3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java @@ -31,6 +31,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import org.apache.doris.load.EtlJobType; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -98,16 +99,26 @@ public void beginTransaction() { throw new BeginTransactionException("current running txns on db is larger than limit"); } this.txnId = Env.getCurrentGlobalTransactionMgr().beginTransaction( - database.getId(), ImmutableList.of(table.getId()), labelName, + database.getId(), ImmutableList.of(table.getId()), labelName, null, new TxnCoordinator(TxnSourceType.FE, 0, FrontendOptions.getLocalHostAddress(), ExecuteEnv.getInstance().getStartupTime()), - LoadJobSourceType.INSERT_STREAMING, ctx.getExecTimeoutS()); + LoadJobSourceType.INSERT_STREAMING, getListenerId(), ctx.getExecTimeoutS()); } catch (Exception e) { throw new AnalysisException("begin transaction failed. " + e.getMessage(), e); } } + private long getListenerId() { + long listenerId = -1; + StreamingInsertTask streamingInsertTask = + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().getStreamingInsertTaskById(jobId); + if (streamingInsertTask != null) { + listenerId = streamingInsertTask.getJobId(); + } + return listenerId; + } + @Override public void finalizeSink(PlanFragment fragment, DataSink sink, PhysicalSink physicalSink) { OlapTableSink olapTableSink = (OlapTableSink) sink; diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 7c630f03353f98..aa6bcd28359734 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -368,7 +368,7 @@ message RoutineLoadJobStatisticPB { message StreamingTaskCommitAttachmentPB { optional int64 job_id = 1; - optional UniqueIdPB task_id = 2; + optional int64 task_id = 2; optional string offset = 3; optional int64 scanned_rows = 4; optional int64 load_bytes = 5; From 33902107d3cd2ef387d6847f62f0ea3f2174d40c Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 17 Sep 2025 09:47:00 +0800 Subject: [PATCH 11/25] [Feature](WIP) Add create job case and fix job bug (#56119) ### What problem does this PR solve? Add create job case and fix job bug --- .../property/storage/StorageProperties.java | 6 +- .../java/org/apache/doris/fs/FileSystem.java | 2 +- .../org/apache/doris/fs/obj/S3ObjStorage.java | 16 ++-- .../apache/doris/fs/remote/RemoteFile.java | 18 ++++ .../apache/doris/fs/remote/S3FileSystem.java | 2 +- .../apache/doris/job/base/AbstractJob.java | 6 +- .../apache/doris/job/base/JobProperties.java | 3 - .../insert/streaming/StreamingInsertJob.java | 55 ++++++++---- .../insert/streaming/StreamingInsertTask.java | 16 +++- .../streaming/StreamingJobProperties.java | 8 +- .../job/offset/SourceOffsetProvider.java | 2 +- .../apache/doris/job/offset/s3/S3Offset.java | 2 +- .../job/offset/s3/S3SourceOffsetProvider.java | 60 +++++++------ .../trees/plans/commands/AlterJobCommand.java | 23 ++--- .../plans/commands/info/CreateJobInfo.java | 24 ++---- .../insert/InsertIntoTableCommand.java | 7 ++ .../doris/catalog/DropFunctionTest.java | 29 ++++++- .../test_streaming_insert_job.out | 22 +++++ .../test_streaming_insert_job.groovy | 86 +++++++++++++++++++ 19 files changed, 288 insertions(+), 99 deletions(-) create mode 100644 regression-test/data/job_p0/streaming_job/test_streaming_insert_job.out create mode 100644 regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java index 2ce87f9ffb9772..422d20a7560293 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java @@ -159,8 +159,6 @@ public static StorageProperties createPrimary(Map origProps) { private static final List, StorageProperties>> PROVIDERS = Arrays.asList( - props -> (isFsSupport(props, FS_HDFS_SUPPORT) - || HdfsProperties.guessIsMe(props)) ? new HdfsProperties(props) : null, props -> ((isFsSupport(props, FS_OSS_HDFS_SUPPORT) || isFsSupport(props, DEPRECATED_OSS_HDFS_SUPPORT)) || OSSHdfsProperties.guessIsMe(props)) ? new OSSHdfsProperties(props) : null, @@ -181,7 +179,9 @@ public static StorageProperties createPrimary(Map origProps) { props -> (isFsSupport(props, FS_BROKER_SUPPORT) || BrokerProperties.guessIsMe(props)) ? new BrokerProperties(props) : null, props -> (isFsSupport(props, FS_LOCAL_SUPPORT) - || LocalProperties.guessIsMe(props)) ? new LocalProperties(props) : null + || LocalProperties.guessIsMe(props)) ? new LocalProperties(props) : null, + props -> (isFsSupport(props, FS_HDFS_SUPPORT) + || HdfsProperties.guessIsMe(props)) ? new HdfsProperties(props) : null ); protected StorageProperties(Type type, Map origProps) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java index cfbb3e560f39db..b2c6957ce382ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java @@ -124,7 +124,7 @@ default Status globList(String remotePath, List result) { * @param fileNumLimit limit the total number of files to be listed. * @return */ - default String globListWithLimit(String remotePath, List result, + default String globListWithLimit(String remotePath, List result, String startFile, long fileSizeLimit, long fileNumLimit) { throw new UnsupportedOperationException("Unsupported operation glob list with limit on current file system."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java index 0a4c9159881d70..4c79df0acf705a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java @@ -642,7 +642,7 @@ public Status globList(String remotePath, List result, boolean fileN * * @return The largest file name after listObject this time */ - public String globListWithLimit(String remotePath, List result, String startFile, + public String globListWithLimit(String remotePath, List result, String startFile, long fileSizeLimit, long fileNumLimit) { long roundCnt = 0; long elementCnt = 0; @@ -704,9 +704,16 @@ public String globListWithLimit(String remotePath, List result, String s } matchCnt++; + RemoteFile remoteFile = new RemoteFile(objPath.getFileName().toString(), + !isPrefix, + isPrefix ? -1 : obj.size(), + isPrefix ? -1 : obj.size(), + isPrefix ? 0 : obj.lastModified().toEpochMilli() + ); + remoteFile.setBucket(bucket); + remoteFile.setParentPath(objPath.getParent().toString()); matchFileSize += obj.size(); - String remoteFileName = "s3://" + bucket + "/" + objPath; - result.add(remoteFileName); + result.add(remoteFile); if (reachLimit(result.size(), matchFileSize, fileSizeLimit, fileNumLimit)) { break; @@ -718,8 +725,7 @@ public String globListWithLimit(String remotePath, List result, String s } //record current last object file name S3Object lastS3Object = response.contents().get(response.contents().size() - 1); - java.nio.file.Path lastObjPath = Paths.get(lastS3Object.key()); - currentMaxFile = "s3://" + bucket + "/" + lastObjPath; + currentMaxFile = lastS3Object.key(); isTruncated = response.isTruncated(); if (isTruncated) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java index 1f6f0225278b07..ac1f8add9475f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java @@ -36,6 +36,8 @@ public class RemoteFile { private long modificationTime; private Path path; BlockLocation[] blockLocations; + private String parentPath; + private String bucket; public RemoteFile(String name, boolean isFile, long size, long blockSize) { this(name, null, isFile, !isFile, size, blockSize, 0, null); @@ -75,6 +77,22 @@ public void setPath(Path path) { this.path = path; } + public String getBucket() { + return bucket; + } + + public void setBucket(String bucket) { + this.bucket = bucket; + } + + public String getParentPath() { + return parentPath; + } + + public void setParentPath(String parentPath) { + this.parentPath = parentPath; + } + public boolean isFile() { return isFile; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java index 9c409a66a2942c..d4e5a23c20e4cb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java @@ -69,7 +69,7 @@ public Status globList(String remotePath, List result, boolean fileN } @Override - public String globListWithLimit(String remotePath, List result, String startFile, + public String globListWithLimit(String remotePath, List result, String startFile, long fileSizeLimit, long fileNumLimit) { S3ObjStorage objStorage = (S3ObjStorage) this.objStorage; return objStorage.globListWithLimit(remotePath, result, startFile, fileSizeLimit, fileNumLimit); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 83b84348ff77eb..d08942460f541f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -99,13 +99,13 @@ public abstract class AbstractJob implements Job properties; private StreamingJobProperties jobProperties; @Getter @SerializedName("tvf") @@ -108,26 +111,40 @@ public StreamingInsertJob(String jobName, JobExecutionConfiguration jobConfig, Long createTimeMs, String executeSql, - StreamingJobProperties jobProperties) { + Map properties) { super(getNextJobId(), jobName, jobStatus, dbName, comment, createUser, jobConfig, createTimeMs, executeSql); this.dbId = ConnectContext.get().getCurrentDbId(); - this.jobProperties = jobProperties; + this.properties = properties; init(); } private void init() { try { + this.jobProperties = new StreamingJobProperties(properties); + jobProperties.validate(); + // build time definition + JobExecutionConfiguration execConfig = getJobConfig(); + TimerDefinition timerDefinition = new TimerDefinition(); + timerDefinition.setInterval(jobProperties.getMaxIntervalSecond()); + timerDefinition.setIntervalUnit(IntervalUnit.SECOND); + timerDefinition.setStartTimeMs(execConfig.getTimerDefinition().getStartTimeMs()); + execConfig.setTimerDefinition(timerDefinition); + UnboundTVFRelation currentTvf = getCurrentTvf(); + this.tvfType = currentTvf.getFunctionName(); this.originTvfProps = currentTvf.getProperties().getMap(); this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(currentTvf.getFunctionName()); + } catch (AnalysisException ae) { + log.warn("parse streaming insert job failed, props: {}", properties, ae); + throw new RuntimeException("parse streaming insert job failed, " + ae.getMessage()); } catch (Exception ex) { log.warn("init streaming insert job failed, sql: {}", getExecuteSql(), ex); - throw new RuntimeException("init streaming insert job failed, sql: " + getExecuteSql(), ex); + throw new RuntimeException("init streaming insert job failed, " + ex.getMessage()); } } - private UnboundTVFRelation getCurrentTvf() throws Exception { + private UnboundTVFRelation getCurrentTvf() { if (baseCommand == null) { this.baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(getExecuteSql()); } @@ -170,7 +187,7 @@ public List createTasks(TaskType taskType, Map originTvfProps; SourceOffsetProvider offsetProvider; public StreamingInsertTask(long jobId, @@ -73,6 +76,7 @@ public StreamingInsertTask(long jobId, SourceOffsetProvider offsetProvider, String currentDb, StreamingJobProperties jobProperties, + Map originTvfProps, UserIdentity userIdentity) { this.jobId = jobId; this.taskId = taskId; @@ -81,6 +85,7 @@ public StreamingInsertTask(long jobId, this.currentDb = currentDb; this.offsetProvider = offsetProvider; this.jobProperties = jobProperties; + this.originTvfProps = originTvfProps; this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); this.createTimeMs = System.currentTimeMillis(); } @@ -118,8 +123,15 @@ private void before() throws Exception { StatementContext statementContext = new StatementContext(); ctx.setStatementContext(statementContext); - this.runningOffset = offsetProvider.getNextOffset(jobProperties, jobProperties.getProperties()); - this.taskCommand = offsetProvider.rewriteTvfParams(sql, runningOffset); + this.runningOffset = offsetProvider.getNextOffset(jobProperties, originTvfProps); + InsertIntoTableCommand baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(sql); + StmtExecutor baseStmtExecutor = + new StmtExecutor(ctx, new LogicalPlanAdapter(baseCommand, ctx.getStatementContext())); + baseCommand.initPlan(ctx, baseStmtExecutor, false); + if (!baseCommand.getParsedPlan().isPresent()) { + throw new JobException("Can not get Parsed plan"); + } + this.taskCommand = offsetProvider.rewriteTvfParams(baseCommand, runningOffset); this.taskCommand.setLabelName(Optional.of(getJobId() + LABEL_SPLITTER + getTaskId())); this.taskCommand.setJobId(getTaskId()); this.stmtExecutor = new StmtExecutor(ctx, new LogicalPlanAdapter(taskCommand, ctx.getStatementContext())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java index 4f463a090d5225..e71b169ef21c3d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java @@ -24,6 +24,7 @@ import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.SessionVariable; +import com.google.gson.annotations.SerializedName; import lombok.Data; import java.util.HashMap; @@ -43,15 +44,18 @@ public class StreamingJobProperties implements JobProperties { private final Map properties; private long maxIntervalSecond; - private int maxRetry; private long s3BatchFiles; private long s3BatchSize; public StreamingJobProperties(Map jobProperties) { this.properties = jobProperties; + if (properties.isEmpty()) { + this.maxIntervalSecond = DEFAULT_MAX_INTERVAL_SECOND; + this.s3BatchFiles = DEFAULT_S3_BATCH_FILES; + this.s3BatchSize = DEFAULT_S3_BATCH_SIZE; + } } - @Override public void validate() throws AnalysisException { this.maxIntervalSecond = Util.getLongPropertyOrDefault( properties.get(StreamingJobProperties.MAX_INTERVAL_SECOND_PROPERTY), diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index d9b2264d5b683d..e670dac553cf4b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -62,7 +62,7 @@ public interface SourceOffsetProvider { * @param nextOffset * @return rewritten InsertIntoTableCommand */ - InsertIntoTableCommand rewriteTvfParams(String executeSql, Offset nextOffset) throws Exception; + InsertIntoTableCommand rewriteTvfParams(InsertIntoTableCommand originCommand, Offset nextOffset); /** * Update the offset of the source. diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index dd6927935bb6a9..cbd8645cc594b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -30,7 +30,7 @@ public class S3Offset implements Offset { String startFile; String endFile; - List fileLists; + String fileLists; @Override public void setEndOffset(String endOffset) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index a95f4af65c94b3..df4a344f06402b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -19,20 +19,18 @@ import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.fs.FileSystemFactory; +import org.apache.doris.fs.remote.RemoteFile; import org.apache.doris.fs.remote.RemoteFileSystem; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; -import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.trees.expressions.Properties; -import org.apache.doris.nereids.trees.expressions.functions.table.S3; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; -import org.apache.doris.qe.ConnectContext; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; import lombok.extern.log4j.Log4j2; import java.util.ArrayList; @@ -40,12 +38,12 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; @Log4j2 public class S3SourceOffsetProvider implements SourceOffsetProvider { S3Offset currentOffset; String maxRemoteEndFile; - InsertIntoTableCommand baseCommand; @Override public String getSourceType() { @@ -54,19 +52,29 @@ public String getSourceType() { @Override public S3Offset getNextOffset(StreamingJobProperties jobProps, Map properties) { + Map copiedProps = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + copiedProps.putAll(properties); S3Offset offset = new S3Offset(); - List rfiles = new ArrayList<>(); + List rfiles = new ArrayList<>(); String startFile = currentOffset == null ? null : currentOffset.endFile; String filePath = null; - StorageProperties storageProperties = StorageProperties.createPrimary(properties); + StorageProperties storageProperties = StorageProperties.createPrimary(copiedProps); try (RemoteFileSystem fileSystem = FileSystemFactory.get(storageProperties)) { - String uri = storageProperties.validateAndGetUri(properties); + String uri = storageProperties.validateAndGetUri(copiedProps); filePath = storageProperties.validateAndNormalizeUri(uri); maxRemoteEndFile = fileSystem.globListWithLimit(filePath, rfiles, startFile, jobProps.getS3BatchFiles(), jobProps.getS3BatchSize()); offset.setStartFile(startFile); - offset.setEndFile(rfiles.get(rfiles.size() - 1)); - offset.setFileLists(rfiles); + //todo: The path may be in the form of bucket/dir/*/*, + // but currently only the case where the last segment is * is handled. + if (!rfiles.isEmpty()) { + String bucket = rfiles.get(0).getBucket(); + String parentPath = rfiles.get(0).getParentPath(); + String filePaths = rfiles.stream().map(RemoteFile::getName).collect(Collectors.joining(",", "{", "}")); + String finalFiles = String.format("s3://%s/%s/%s", bucket, parentPath, filePaths); + offset.setEndFile(String.format("s3://%s/%s/%s", bucket, parentPath, rfiles.get(rfiles.size() - 1).getName())); + offset.setFileLists(finalFiles); + } } catch (Exception e) { log.warn("list path exception, path={}", filePath, e); throw new RuntimeException(e); @@ -93,23 +101,18 @@ public String getRemoteOffset() { } @Override - public InsertIntoTableCommand rewriteTvfParams(String executeSql, Offset runningOffset) throws Exception { + public InsertIntoTableCommand rewriteTvfParams(InsertIntoTableCommand originCommand, Offset runningOffset) { S3Offset offset = (S3Offset) runningOffset; Map props = new HashMap<>(); - String finalUri = "{" + String.join(",", offset.getFileLists()) + "}"; - props.put("uri", finalUri); - if (baseCommand == null) { - this.baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(executeSql); - this.baseCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); - } - // rewrite plan - Plan rewritePlan = baseCommand.getLogicalQuery().rewriteUp(plan -> { - if (plan instanceof LogicalTVFRelation) { - LogicalTVFRelation originTvfRel = (LogicalTVFRelation) plan; - LogicalTVFRelation newRvfRel = new LogicalTVFRelation( - originTvfRel.getRelationId(), new S3(new Properties(props)), ImmutableList.of()); - return newRvfRel; + Plan rewritePlan = originCommand.getParsedPlan().get().rewriteUp(plan -> { + if (plan instanceof UnboundTVFRelation) { + UnboundTVFRelation originTvfRel = (UnboundTVFRelation) plan; + Map oriMap = originTvfRel.getProperties().getMap(); + props.putAll(oriMap); + props.put("uri", offset.getFileLists()); + return new UnboundTVFRelation( + originTvfRel.getRelationId(), originTvfRel.getFunctionName(), new Properties(props)); } return plan; }); @@ -120,14 +123,17 @@ public InsertIntoTableCommand rewriteTvfParams(String executeSql, Offset running @Override public void updateOffset(Offset offset) { this.currentOffset = (S3Offset) offset; + this.currentOffset.setFileLists(null); } @Override public void fetchRemoteMeta(Map properties) throws Exception { - StorageProperties storageProperties = StorageProperties.createPrimary(properties); + Map copiedProps = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + copiedProps.putAll(properties); + StorageProperties storageProperties = StorageProperties.createPrimary(copiedProps); String startFile = currentOffset == null ? null : currentOffset.endFile; try (RemoteFileSystem fileSystem = FileSystemFactory.get(storageProperties)) { - String uri = storageProperties.validateAndGetUri(properties); + String uri = storageProperties.validateAndGetUri(copiedProps); String filePath = storageProperties.validateAndNormalizeUri(uri); maxRemoteEndFile = fileSystem.globListWithLimit(filePath, new ArrayList<>(), startFile, 1, 1); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 53a90893b81444..b1823de4a1d50f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -22,13 +22,9 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecuteType; -import org.apache.doris.job.base.JobExecutionConfiguration; -import org.apache.doris.job.base.TimerDefinition; -import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; -import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.qe.ConnectContext; @@ -81,25 +77,18 @@ private AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws JobExcepti if (job instanceof StreamingInsertJob) { StreamingInsertJob originJob = (StreamingInsertJob) job; String updateSQL = StringUtils.isEmpty(sql) ? originJob.getExecuteSql() : sql; - Map updateProps = properties == null || properties.isEmpty() ? originJob.getJobProperties() - .getProperties() : properties; - StreamingJobProperties streamJobProps = new StreamingJobProperties(updateProps); - // rebuild time definition - JobExecutionConfiguration execConfig = originJob.getJobConfig(); - TimerDefinition timerDefinition = new TimerDefinition(); - timerDefinition.setInterval(streamJobProps.getMaxIntervalSecond()); - timerDefinition.setIntervalUnit(IntervalUnit.SECOND); - timerDefinition.setStartTimeMs(execConfig.getTimerDefinition().getStartTimeMs()); - execConfig.setTimerDefinition(timerDefinition); + Map updateProps = + properties == null || properties.isEmpty() ? originJob.getProperties() : properties; + return new StreamingInsertJob(jobName, job.getJobStatus(), job.getCurrentDbName(), job.getComment(), ConnectContext.get().getCurrentUserIdentity(), - execConfig, + originJob.getJobConfig(), System.currentTimeMillis(), updateSQL, - streamJobProps); + updateProps); } else { throw new JobException("Unsupported job type for ALTER:" + job.getJobType()); } @@ -117,7 +106,7 @@ private void validate() throws Exception { if (job.getJobConfig().getExecuteType().equals(JobExecuteType.STREAMING) && job instanceof StreamingInsertJob) { StreamingInsertJob streamingJob = (StreamingInsertJob) job; - boolean proCheck = checkProperties(streamingJob.getJobProperties().getProperties()); + boolean proCheck = checkProperties(streamingJob.getProperties()); boolean sqlCheck = checkSql(streamingJob.getExecuteSql()); if (!proCheck && !sqlCheck) { throw new AnalysisException("No properties or sql changed in ALTER JOB"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 51d0dddc7cb664..40a4d5830590f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -125,11 +125,8 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti // check its insert stmt,currently only support insert stmt JobExecutionConfiguration jobExecutionConfiguration = new JobExecutionConfiguration(); JobExecuteType executeType = intervalOptional.isPresent() ? JobExecuteType.RECURRING : JobExecuteType.ONE_TIME; - JobProperties properties = null; if (streamingJob) { executeType = JobExecuteType.STREAMING; - properties = new StreamingJobProperties(jobProperties); - properties.validate(); jobExecutionConfiguration.setImmediate(true); } jobExecutionConfiguration.setExecuteType(executeType); @@ -138,21 +135,18 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti if (executeType.equals(JobExecuteType.ONE_TIME)) { buildOnceJob(timerDefinition, jobExecutionConfiguration); } else if (executeType.equals(JobExecuteType.STREAMING)) { - buildStreamingJob(timerDefinition, properties); + buildStreamingJob(timerDefinition); } else { buildRecurringJob(timerDefinition, jobExecutionConfiguration); } jobExecutionConfiguration.setTimerDefinition(timerDefinition); - return analyzeAndCreateJob(executeSql, dbName, jobExecutionConfiguration, properties); + return analyzeAndCreateJob(executeSql, dbName, jobExecutionConfiguration, jobProperties); } - private void buildStreamingJob(TimerDefinition timerDefinition, JobProperties props) - throws AnalysisException { - StreamingJobProperties properties = (StreamingJobProperties) props; - timerDefinition.setInterval(properties.getMaxIntervalSecond()); + private void buildStreamingJob(TimerDefinition timerDefinition) { + // timerDefinition.setInterval(properties.getMaxIntervalSecond()); timerDefinition.setIntervalUnit(IntervalUnit.SECOND); timerDefinition.setStartTimeMs(System.currentTimeMillis()); - properties.validate(); } /** @@ -237,7 +231,7 @@ protected static void checkAuth() throws AnalysisException { */ private AbstractJob analyzeAndCreateJob(String sql, String currentDbName, JobExecutionConfiguration jobExecutionConfiguration, - JobProperties properties) throws UserException { + Map properties) throws UserException { if (jobExecutionConfiguration.getExecuteType().equals(JobExecuteType.STREAMING)) { return analyzeAndCreateStreamingInsertJob(sql, currentDbName, jobExecutionConfiguration, properties); } else { @@ -271,13 +265,13 @@ private AbstractJob analyzeAndCreateInsertJob(String sql, String currentDbName, } private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String currentDbName, - JobExecutionConfiguration jobExecutionConfiguration, JobProperties properties) throws UserException { + JobExecutionConfiguration jobExecutionConfiguration, Map properties) throws UserException { NereidsParser parser = new NereidsParser(); LogicalPlan logicalPlan = parser.parseSingle(sql); if (logicalPlan instanceof InsertIntoTableCommand) { - // InsertIntoTableCommand insertIntoTableCommand = (InsertIntoTableCommand) logicalPlan; + InsertIntoTableCommand insertIntoTableCommand = (InsertIntoTableCommand) logicalPlan; try { - // insertIntoTableCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); + insertIntoTableCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); return new StreamingInsertJob(labelNameOptional.get(), JobStatus.PENDING, currentDbName, @@ -286,7 +280,7 @@ private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String curren jobExecutionConfiguration, System.currentTimeMillis(), sql, - (StreamingJobProperties) properties); + properties); } catch (Exception e) { throw new AnalysisException(e.getMessage()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 6736b16f8afeb0..e774499f3617cb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -61,6 +61,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalJdbcTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -103,6 +104,7 @@ public class InsertIntoTableCommand extends Command implements NeedAuditEncrypti private Optional logicalQuery; private Optional labelName; private Optional branchName; + private Optional parsedPlan; /** * When source it's from job scheduler,it will be set. */ @@ -153,6 +155,10 @@ public LogicalPlan getLogicalQuery() { return logicalQuery.orElse(originLogicalQuery); } + public Optional getParsedPlan() { + return parsedPlan; + } + protected void setLogicalQuery(LogicalPlan logicalQuery) { this.logicalQuery = Optional.of(logicalQuery); } @@ -233,6 +239,7 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor stmtExec throw new IllegalStateException(e.getMessage(), e); } insertExecutor = buildResult.executor; + parsedPlan = Optional.ofNullable(buildResult.planner.getParsedPlan()); if (!needBeginTransaction) { return insertExecutor; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java index bef0bb92d20593..f05f1791f2fe91 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java @@ -20,22 +20,31 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.trees.expressions.Properties; +import org.apache.doris.nereids.trees.expressions.functions.table.S3; +import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.CreateFunctionCommand; import org.apache.doris.nereids.trees.plans.commands.DropFunctionCommand; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; import org.apache.doris.utframe.DorisAssert; import org.apache.doris.utframe.UtFrameUtils; +import com.google.common.collect.ImmutableList; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import java.io.File; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.UUID; @@ -63,9 +72,27 @@ public static void teardown() { public void testDropGlobalFunction() throws Exception { ConnectContext ctx = UtFrameUtils.createDefaultCtx(); // 1. create database db1 - String sql = "create database db1;"; + //String sql = "create database db1;"; + String sql = "insert into db1.tb select * from s3('url'='s3://a/*.csv')"; NereidsParser nereidsParser = new NereidsParser(); LogicalPlan logicalPlan = nereidsParser.parseSingle(sql); + InsertIntoTableCommand baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(sql); + baseCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); + Map map = new HashMap<>(); + map.put("url" ,"s3:/xxxx/*."); + // rewrite plan + Plan rewritePlan = baseCommand.getLogicalQuery().rewriteUp(plan -> { + if (plan instanceof LogicalTVFRelation) { + LogicalTVFRelation originTvfRel = (LogicalTVFRelation) plan; + LogicalTVFRelation newRvfRel = new LogicalTVFRelation( + originTvfRel.getRelationId(), new S3(new Properties(map)), ImmutableList.of()); + return newRvfRel; + } + return plan; + }); + InsertIntoTableCommand s = new InsertIntoTableCommand((LogicalPlan) rewritePlan, Optional.empty(), Optional.empty(), + Optional.empty(), true, Optional.empty()); + StmtExecutor stmtExecutor = new StmtExecutor(connectContext, sql); if (logicalPlan instanceof CreateDatabaseCommand) { ((CreateDatabaseCommand) logicalPlan).run(connectContext, stmtExecutor); diff --git a/regression-test/data/job_p0/streaming_job/test_streaming_insert_job.out b/regression-test/data/job_p0/streaming_job/test_streaming_insert_job.out new file mode 100644 index 00000000000000..867b1f9432ba17 --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/test_streaming_insert_job.out @@ -0,0 +1,22 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +1 Emily 25 +2 Benjamin 35 +3 Olivia 28 +4 Alexander 60 +5 Ava 17 +6 William 69 +7 Sophia 32 +8 James 64 +9 Emma 37 +10 Liam 64 +11 Alexander 34 +12 Isabella 43 +13 Benjamin 56 +14 Sophia 12 +15 Christopher 33 +16 Emma 23 +17 Michael 11 +18 Olivia 38 +19 Daniel 19 +20 Ava 28 \ No newline at end of file diff --git a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy new file mode 100644 index 00000000000000..bd8ee2759fd138 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy @@ -0,0 +1,86 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_insert_job") { + def tableName = "test_streaming_insert_job_tbl" + def jobName = "test_streaming_insert_job_name" + + sql """drop table if exists `${tableName}` force""" + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + `c1` int NULL, + `c2` string NULL, + `c3` int NULL, + ) ENGINE=OLAP + DUPLICATE KEY(`c1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`c1`) BUCKETS 3 + PROPERTIES ("replication_allocation" = "tag.location.default: 1"); + """ + + + // create recurring job + sql """ + CREATE JOB ${jobName} ON STREAMING DO INSERT INTO ${tableName} + SELECT * FROM S3 + ( + "uri" = "s3://${s3BucketName}/regression/load/data/example_[0-1].csv", + "format" = "csv", + "provider" = "${getS3Provider()}", + "column_separator" = ",", + "s3.endpoint" = "${getS3Endpoint()}", + "s3.region" = "${getS3Region()}", + "s3.access_key" = "${getS3AK()}", + "s3.secret_key" = "${getS3SK()}" + ); + """ + Awaitility.await().atMost(30, SECONDS).until( + { + print("check success task count") + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='STREAMING' """ + // check job status and succeed task count larger than 1 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + + sql """ + PAUSE JOB where jobname = '${jobName}' + """ + def pausedJobStatus = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + assert pausedJobStatus.get(0).get(0) == "PAUSED" + + qt_select """ SELECT * FROM ${tableName} order by c1 """ + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + +} From 406276e268cbf529bf408331664fa42511a7c3ec Mon Sep 17 00:00:00 2001 From: hui lai Date: Wed, 17 Sep 2025 13:54:25 +0800 Subject: [PATCH 12/25] [fix](streaming job) add task commit check and job event lock to ensure exactly-once semantics (#56135) ### What problem does this PR solve? Add task commit check and job event lock to ensure exactly-once semantics. --- .../insert/streaming/StreamingInsertJob.java | 86 +++++++++++++------ .../insert/streaming/StreamingInsertTask.java | 6 +- 2 files changed, 63 insertions(+), 29 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 387714e8805dce..650b9f412b2952 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -73,6 +73,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; @Log4j2 public class StreamingInsertJob extends AbstractJob> implements @@ -102,6 +103,7 @@ public class StreamingInsertJob extends AbstractJob taskIds = new ArrayList<>(); - taskIds.add(runningStreamTask.getTaskId()); - List loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIds); - if (loadJobs.size() != 1) { - throw new TransactionException("load job not found, insert job id is " + runningStreamTask.getTaskId()); + boolean shouldRealseLock = false; + lock.writeLock().lock(); + try { + ArrayList taskIds = new ArrayList<>(); + taskIds.add(runningStreamTask.getTaskId()); + List loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIds); + if (loadJobs.size() != 1) { + throw new TransactionException("load job not found, insert job id is " + runningStreamTask.getTaskId()); + } + LoadJob loadJob = loadJobs.get(0); + + if (txnState.getTransactionId() != loadJob.getTransactionId() + || !runningStreamTask.getStatus().equals(TaskStatus.RUNNING)) { + shouldRealseLock = true; + throw new TransactionException("txn " + txnState.getTransactionId() + "should be aborted."); + } + + LoadStatistic loadStatistic = loadJob.getLoadStatistic(); + txnState.setTxnCommitAttachment(new StreamingTaskTxnCommitAttachment( + getJobId(), + runningStreamTask.getTaskId(), + loadStatistic.getScannedRows(), + loadStatistic.getLoadBytes(), + loadStatistic.getFileNumber(), + loadStatistic.getTotalFileSizeB(), + runningStreamTask.getRunningOffset())); + } finally { + if (shouldRealseLock) { + lock.writeLock().unlock(); + } } - LoadJob loadJob = loadJobs.get(0); - LoadStatistic loadStatistic = loadJob.getLoadStatistic(); - txnState.setTxnCommitAttachment(new StreamingTaskTxnCommitAttachment( - getJobId(), - runningStreamTask.getTaskId(), - loadStatistic.getScannedRows(), - loadStatistic.getLoadBytes(), - loadStatistic.getFileNumber(), - loadStatistic.getTotalFileSizeB(), - runningStreamTask.getRunningOffset())); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index b1e22a0a7cbc5c..c171173fd3c772 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -206,7 +206,11 @@ public void onFail(String errMsg) throws JobException { streamingInsertJob.onStreamTaskFail(this); } - public void cancel(boolean needWaitCancelComplete) throws Exception { + public void cancel(boolean needWaitCancelComplete) { + if (TaskStatus.SUCCESS.equals(status) || TaskStatus.FAILED.equals(status) + || TaskStatus.CANCELED.equals(status)) { + return; + } if (isCanceled.get()) { return; } From a541e086d45410f4d08fea75bbebe23ee05fb6d8 Mon Sep 17 00:00:00 2001 From: hui lai Date: Wed, 17 Sep 2025 15:16:23 +0800 Subject: [PATCH 13/25] [fix](streaming job) fix compile error (#56141) ### What problem does this PR solve? Fix compile error --- .../insert/streaming/StreamingInsertJob.java | 14 ++++++------- .../streaming/StreamingJobProperties.java | 1 - .../apache/doris/job/offset/s3/S3Offset.java | 2 -- .../job/offset/s3/S3SourceOffsetProvider.java | 3 ++- .../plans/commands/info/CreateJobInfo.java | 2 -- .../insert/InsertIntoTableCommand.java | 1 - .../doris/transaction/TransactionState.java | 4 ++++ .../doris/catalog/DropFunctionTest.java | 20 +------------------ 8 files changed, 14 insertions(+), 33 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 650b9f412b2952..b872040c4c354a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -43,7 +43,6 @@ import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.SourceOffsetProviderFactory; import org.apache.doris.job.task.AbstractTask; -import org.apache.doris.load.FailMsg; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; @@ -238,11 +237,11 @@ public void onTaskSuccess(StreamingJobSchedulerTask task) throws JobException { public void onStreamTaskFail(StreamingInsertTask task) throws JobException { try { - failedTaskCount.incrementAndGet(); - Env.getCurrentEnv().getJobManager().getStreamingTaskManager().removeRunningTask(task); - if (getJobConfig().getExecuteType().equals(JobExecuteType.INSTANT)) { - this.pauseReason = new PauseReason(InternalErrorCode.INTERNAL_ERR, task.getErrMsg()); - } + failedTaskCount.incrementAndGet(); + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().removeRunningTask(task); + if (getJobConfig().getExecuteType().equals(JobExecuteType.INSTANT)) { + this.pauseReason = new PauseReason(InternalErrorCode.INTERNAL_ERR, task.getErrMsg()); + } } finally { lock.writeLock().unlock(); } @@ -321,7 +320,8 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal( jobStatistic == null ? FeConstants.null_string : jobStatistic.toJson())); - trow.addToColumnValue(new TCell().setStringVal(pauseReason == null ? FeConstants.null_string : pauseReason.getMsg())); + trow.addToColumnValue( + new TCell().setStringVal(pauseReason == null ? FeConstants.null_string : pauseReason.getMsg())); return trow; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java index e71b169ef21c3d..a79bc1b230d6d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java @@ -24,7 +24,6 @@ import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.SessionVariable; -import com.google.gson.annotations.SerializedName; import lombok.Data; import java.util.HashMap; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index cbd8645cc594b9..2ab2030fbbb3e9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -23,8 +23,6 @@ import lombok.Getter; import lombok.Setter; -import java.util.List; - @Getter @Setter public class S3Offset implements Offset { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index df4a344f06402b..f63333468fa426 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -72,7 +72,8 @@ public S3Offset getNextOffset(StreamingJobProperties jobProps, Map map = new HashMap<>(); - map.put("url" ,"s3:/xxxx/*."); - // rewrite plan - Plan rewritePlan = baseCommand.getLogicalQuery().rewriteUp(plan -> { - if (plan instanceof LogicalTVFRelation) { - LogicalTVFRelation originTvfRel = (LogicalTVFRelation) plan; - LogicalTVFRelation newRvfRel = new LogicalTVFRelation( - originTvfRel.getRelationId(), new S3(new Properties(map)), ImmutableList.of()); - return newRvfRel; - } - return plan; - }); - InsertIntoTableCommand s = new InsertIntoTableCommand((LogicalPlan) rewritePlan, Optional.empty(), Optional.empty(), - Optional.empty(), true, Optional.empty()); + map.put("url", "s3:/xxxx/*."); StmtExecutor stmtExecutor = new StmtExecutor(connectContext, sql); if (logicalPlan instanceof CreateDatabaseCommand) { From cbf74bf50b9173eac23bd712f6b15959e242fb5b Mon Sep 17 00:00:00 2001 From: hui lai Date: Wed, 17 Sep 2025 15:16:34 +0800 Subject: [PATCH 14/25] [fix](streaming job) fix register callback id invalid (#56142) ### What problem does this PR solve? Fix register callback id invalid. --- .../commands/insert/OlapInsertExecutor.java | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java index f41d8893f020e3..b7141dd9ed0928 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java @@ -99,26 +99,16 @@ public void beginTransaction() { throw new BeginTransactionException("current running txns on db is larger than limit"); } this.txnId = Env.getCurrentGlobalTransactionMgr().beginTransaction( - database.getId(), ImmutableList.of(table.getId()), labelName, null, + database.getId(), ImmutableList.of(table.getId()), labelName, new TxnCoordinator(TxnSourceType.FE, 0, FrontendOptions.getLocalHostAddress(), ExecuteEnv.getInstance().getStartupTime()), - LoadJobSourceType.INSERT_STREAMING, getListenerId(), ctx.getExecTimeoutS()); + LoadJobSourceType.INSERT_STREAMING, ctx.getExecTimeoutS()); } catch (Exception e) { throw new AnalysisException("begin transaction failed. " + e.getMessage(), e); } } - private long getListenerId() { - long listenerId = -1; - StreamingInsertTask streamingInsertTask = - Env.getCurrentEnv().getJobManager().getStreamingTaskManager().getStreamingInsertTaskById(jobId); - if (streamingInsertTask != null) { - listenerId = streamingInsertTask.getJobId(); - } - return listenerId; - } - @Override public void finalizeSink(PlanFragment fragment, DataSink sink, PhysicalSink physicalSink) { OlapTableSink olapTableSink = (OlapTableSink) sink; @@ -199,6 +189,7 @@ protected void beforeExec() { @Override protected void onComplete() throws UserException { + setTxnCallbackId(); if (ctx.getState().getStateType() == MysqlStateType.ERR) { try { String errMsg = Strings.emptyToNull(ctx.getState().getErrorMessage()); @@ -234,6 +225,18 @@ protected void onComplete() throws UserException { } } + private void setTxnCallbackId() { + TransactionState state = Env.getCurrentGlobalTransactionMgr().getTransactionState(database.getId(), txnId); + if (state == null) { + throw new AnalysisException("txn does not exist: " + txnId); + } + StreamingInsertTask streamingInsertTask = + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().getStreamingInsertTaskById(jobId); + if (streamingInsertTask != null) { + state.setCallbackId(streamingInsertTask.getJobId()); + } + } + @Override protected void onFail(Throwable t) { errMsg = t.getMessage() == null ? "unknown reason" : t.getMessage(); From 0fbe6836bbe33c81e00995d89d934a5089f483bd Mon Sep 17 00:00:00 2001 From: hui lai Date: Wed, 17 Sep 2025 15:53:01 +0800 Subject: [PATCH 15/25] [revert](streaming job) revert "implement offset persistence and replay in cloud mode" (#56149) ### What problem does this PR solve? Revert "implement offset persistence and replay in cloud mode" --- cloud/src/common/bvars.cpp | 5 - cloud/src/common/bvars.h | 3 - cloud/src/meta-service/meta_service.h | 13 -- cloud/src/meta-service/meta_service_txn.cpp | 137 ------------------ cloud/src/meta-store/keys.cpp | 13 +- cloud/src/meta-store/keys.h | 5 - .../doris/cloud/rpc/MetaServiceClient.java | 6 - .../doris/cloud/rpc/MetaServiceProxy.java | 5 - .../CloudGlobalTransactionMgr.java | 14 -- .../doris/cloud/transaction/TxnUtil.java | 38 ----- .../insert/streaming/StreamingInsertJob.java | 44 ------ .../StreamingTaskTxnCommitAttachment.java | 10 -- .../org/apache/doris/job/offset/Offset.java | 4 - .../apache/doris/job/offset/s3/S3Offset.java | 10 -- gensrc/proto/cloud.proto | 31 +--- 15 files changed, 3 insertions(+), 335 deletions(-) diff --git a/cloud/src/common/bvars.cpp b/cloud/src/common/bvars.cpp index 8ea24c4a1baade..e495b5ea95b956 100644 --- a/cloud/src/common/bvars.cpp +++ b/cloud/src/common/bvars.cpp @@ -85,7 +85,6 @@ BvarLatencyRecorderWithTag g_bvar_ms_remove_delete_bitmap("ms", "remove_delete_b BvarLatencyRecorderWithTag g_bvar_ms_remove_delete_bitmap_update_lock("ms", "remove_delete_bitmap_update_lock"); BvarLatencyRecorderWithTag g_bvar_ms_get_instance("ms", "get_instance"); BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach("ms", "get_rl_task_commit_attach"); -BvarLatencyRecorderWithTag g_bvar_ms_get_streaming_task_commit_attach("ms", "get_streaming_task_commit_attach"); BvarLatencyRecorderWithTag g_bvar_ms_reset_rl_progress("ms", "reset_rl_progress"); BvarLatencyRecorderWithTag g_bvar_ms_get_txn_id("ms", "get_txn_id"); BvarLatencyRecorderWithTag g_bvar_ms_start_tablet_job("ms", "start_tablet_job"); @@ -365,8 +364,6 @@ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter("rpc_kv_precommit_txn_ge mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter("rpc_kv_precommit_txn_put_counter",{"instance_id"}); // get_rl_task_commit_attach mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter("rpc_kv_get_rl_task_commit_attach_get_counter",{"instance_id"}); -// get_streaming_task_commit_attach -mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_counter("rpc_kv_get_streaming_task_commit_attach_get_counter",{"instance_id"}); // reset_rl_progress mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter("rpc_kv_reset_rl_progress_get_counter",{"instance_id"}); mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter("rpc_kv_reset_rl_progress_put_counter",{"instance_id"}); @@ -530,8 +527,6 @@ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_bytes("rpc_kv_precommit_txn_get_ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_bytes("rpc_kv_precommit_txn_put_bytes",{"instance_id"}); // get_rl_task_commit_attach mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_bytes("rpc_kv_get_rl_task_commit_attach_get_bytes",{"instance_id"}); -// get_streaming_task_commit_attach -mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_bytes("rpc_kv_get_streaming_task_commit_attach_get_bytes",{"instance_id"}); // reset_rl_progress mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_bytes("rpc_kv_reset_rl_progress_get_bytes",{"instance_id"}); mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_bytes("rpc_kv_reset_rl_progress_put_bytes",{"instance_id"}); diff --git a/cloud/src/common/bvars.h b/cloud/src/common/bvars.h index e72d0fcd3760d1..8fb5973249fd2d 100644 --- a/cloud/src/common/bvars.h +++ b/cloud/src/common/bvars.h @@ -251,7 +251,6 @@ extern BvarLatencyRecorderWithTag g_bvar_ms_get_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_get_instance; extern BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach; -extern BvarLatencyRecorderWithTag g_bvar_ms_get_streaming_task_commit_attach; extern BvarLatencyRecorderWithTag g_bvar_ms_reset_rl_progress; extern BvarLatencyRecorderWithTag g_bvar_ms_get_txn_id; extern BvarLatencyRecorderWithTag g_bvar_ms_check_kv; @@ -471,7 +470,6 @@ extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter; -extern mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_counter; @@ -584,7 +582,6 @@ extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_bytes; -extern mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_bytes; diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index c7a37277fdc083..38cc77aebb00af 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -319,11 +319,6 @@ class MetaServiceImpl : public cloud::MetaService { GetRLTaskCommitAttachResponse* response, ::google::protobuf::Closure* done) override; - void get_streaming_task_commit_attach(::google::protobuf::RpcController* controller, - const GetStreamingTaskCommitAttachRequest* request, - GetStreamingTaskCommitAttachResponse* response, - ::google::protobuf::Closure* done) override; - void reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, ::google::protobuf::Closure* done) override; @@ -825,14 +820,6 @@ class MetaServiceProxy final : public MetaService { done); } - void get_streaming_task_commit_attach(::google::protobuf::RpcController* controller, - const GetStreamingTaskCommitAttachRequest* request, - GetStreamingTaskCommitAttachResponse* response, - ::google::protobuf::Closure* done) override { - call_impl(&cloud::MetaService::get_streaming_task_commit_attach, controller, request, - response, done); - } - void reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, ::google::protobuf::Closure* done) override { diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index bd4cf6fb036f98..ce128f6d30e485 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -610,75 +610,6 @@ void put_routine_load_progress(MetaServiceCode& code, std::string& msg, << " routine load new progress: " << new_progress_info.ShortDebugString(); } -void put_streaming_job_meta(MetaServiceCode& code, std::string& msg, const std::string& instance_id, - const CommitTxnRequest* request, Transaction* txn, int64_t db_id) { - std::stringstream ss; - int64_t txn_id = request->txn_id(); - if (!request->has_commit_attachment()) { - ss << "failed to get commit attachment from req, db_id=" << db_id << " txn_id=" << txn_id; - msg = ss.str(); - return; - } - TxnCommitAttachmentPB txn_commit_attachment = request->commit_attachment(); - StreamingTaskCommitAttachmentPB commit_attachment = - txn_commit_attachment.streaming_task_txn_commit_attachment(); - int64_t job_id = commit_attachment.job_id(); - - std::string streaming_meta_key; - std::string streaming_meta_val; - bool prev_meta_existed = true; - StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; - streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); - TxnErrorCode err = txn->get(streaming_meta_key, &streaming_meta_val); - if (err != TxnErrorCode::TXN_OK) { - if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { - prev_meta_existed = false; - } else { - code = cast_as(err); - ss << "failed to get streaming job meta, db_id=" << db_id << " txn_id=" << txn_id - << " err=" << err; - msg = ss.str(); - return; - } - } - - StreamingTaskCommitAttachmentPB new_meta_info; - if (prev_meta_existed) { - if (!new_meta_info.ParseFromString(streaming_meta_val)) { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - ss << "failed to parse streaming job meta, db_id=" << db_id << " txn_id=" << txn_id; - msg = ss.str(); - return; - } - new_meta_info.set_scanned_rows(new_meta_info.scanned_rows() + - commit_attachment.scanned_rows()); - new_meta_info.set_load_bytes(new_meta_info.load_bytes() + commit_attachment.load_bytes()); - new_meta_info.set_file_number(new_meta_info.file_number() + - commit_attachment.file_number()); - new_meta_info.set_file_size(new_meta_info.file_size() + commit_attachment.file_size()); - } else { - new_meta_info.set_job_id(commit_attachment.job_id()); - new_meta_info.set_scanned_rows(commit_attachment.scanned_rows()); - new_meta_info.set_load_bytes(commit_attachment.load_bytes()); - new_meta_info.set_file_number(commit_attachment.file_number()); - new_meta_info.set_file_size(commit_attachment.file_size()); - } - if (commit_attachment.has_offset()) { - new_meta_info.set_offset(commit_attachment.offset()); - } - std::string new_meta_val; - if (!new_meta_info.SerializeToString(&new_meta_val)) { - code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize new streaming meta val, txn_id=" << txn_id; - msg = ss.str(); - return; - } - - txn->put(streaming_meta_key, new_meta_val); - LOG(INFO) << "put streaming_meta_key key=" << hex(streaming_meta_key) - << " streaming job new meta: " << new_meta_info.ShortDebugString(); -} - void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcController* controller, const GetRLTaskCommitAttachRequest* request, GetRLTaskCommitAttachResponse* response, @@ -747,64 +678,6 @@ void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcControlle } } -void MetaServiceImpl::get_streaming_task_commit_attach( - ::google::protobuf::RpcController* controller, - const GetStreamingTaskCommitAttachRequest* request, - GetStreamingTaskCommitAttachResponse* response, ::google::protobuf::Closure* done) { - RPC_PREPROCESS(get_streaming_task_commit_attach, get); - instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); - if (instance_id.empty()) { - code = MetaServiceCode::INVALID_ARGUMENT; - msg = "empty instance_id"; - LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); - return; - } - RPC_RATE_LIMIT(get_streaming_task_commit_attach) - - TxnErrorCode err = txn_kv_->create_txn(&txn); - if (err != TxnErrorCode::TXN_OK) { - code = cast_as(err); - ss << "filed to create txn, err=" << err; - msg = ss.str(); - return; - } - - if (!request->has_db_id() || !request->has_job_id()) { - code = MetaServiceCode::INVALID_ARGUMENT; - msg = "empty db_id or job_id"; - LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); - return; - } - - int64_t db_id = request->db_id(); - int64_t job_id = request->job_id(); - std::string streaming_meta_key; - std::string streaming_meta_val; - StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; - streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); - err = txn->get(streaming_meta_key, &streaming_meta_val); - if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { - code = MetaServiceCode::STREAMING_JOB_PROGRESS_NOT_FOUND; - ss << "progress info not found, db_id=" << db_id << " job_id=" << job_id << " err=" << err; - msg = ss.str(); - return; - } else if (err != TxnErrorCode::TXN_OK) { - code = cast_as(err); - ss << "failed to get progress info, db_id=" << db_id << " job_id=" << job_id - << " err=" << err; - msg = ss.str(); - return; - } - - StreamingTaskCommitAttachmentPB* commit_attach = response->mutable_commit_attach(); - if (!commit_attach->ParseFromString(streaming_meta_val)) { - code = MetaServiceCode::PROTOBUF_PARSE_ERR; - ss << "failed to parse meta info, db_id=" << db_id << " job_id=" << job_id; - msg = ss.str(); - return; - } -} - void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, @@ -1699,11 +1572,6 @@ void MetaServiceImpl::commit_txn_immediately( put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } - if (txn_info.load_job_source_type() == - LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { - put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); - } - LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) << " txn_id=" << txn_id; LOG(INFO) << "commit_txn put_size=" << txn->put_bytes() @@ -2097,11 +1965,6 @@ void MetaServiceImpl::commit_txn_eventually( put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } - if (txn_info.load_job_source_type() == - LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { - put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); - } - // save versions for partition int64_t version_update_time_ms = duration_cast(system_clock::now().time_since_epoch()).count(); diff --git a/cloud/src/meta-store/keys.cpp b/cloud/src/meta-store/keys.cpp index 7b2b75c4d555eb..e23f84771aba04 100644 --- a/cloud/src/meta-store/keys.cpp +++ b/cloud/src/meta-store/keys.cpp @@ -64,7 +64,6 @@ static const char* STATS_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_RL_PROGRESS = "routine_load_progress"; -static const char* JOB_KEY_INFIX_STREAMING_JOB_META = "streaming_job_meta"; static const char* JOB_KEY_INFIX_RESTORE_TABLET = "restore_tablet"; static const char* JOB_KEY_INFIX_RESTORE_ROWSET = "restore_rowset"; @@ -145,7 +144,7 @@ static void encode_prefix(const T& t, std::string* key) { MetaDeleteBitmapInfo, MetaDeleteBitmapUpdateLockInfo, MetaPendingDeleteBitmapInfo, PartitionVersionKeyInfo, RecycleIndexKeyInfo, RecyclePartKeyInfo, RecycleRowsetKeyInfo, RecycleTxnKeyInfo, RecycleStageKeyInfo, StatsTabletKeyInfo, TableVersionKeyInfo, JobRestoreTabletKeyInfo, JobRestoreRowsetKeyInfo, - JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, StreamingJobMetaKeyInfo, + JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, CopyJobKeyInfo, CopyFileKeyInfo, StorageVaultKeyInfo, MetaSchemaPBDictionaryInfo, MowTabletJobInfo>); @@ -182,8 +181,7 @@ static void encode_prefix(const T& t, std::string* key) { encode_bytes(STATS_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v - || std::is_same_v - || std::is_same_v) { + || std::is_same_v) { encode_bytes(JOB_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v) { @@ -465,13 +463,6 @@ void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out) encode_int64(std::get<2>(in), out); // job_id } -void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out) { - encode_prefix(in, out); // 0x01 "job" ${instance_id} - encode_bytes(JOB_KEY_INFIX_STREAMING_JOB_META, out); // "streaming_job_meta" - encode_int64(std::get<1>(in), out); // db_id - encode_int64(std::get<2>(in), out); // job_id -} - void job_restore_tablet_key(const JobRestoreTabletKeyInfo& in, std::string* out) { encode_prefix(in, out); // 0x01 "job" ${instance_id} encode_bytes(JOB_KEY_INFIX_RESTORE_TABLET, out); // "restore_tablet" diff --git a/cloud/src/meta-store/keys.h b/cloud/src/meta-store/keys.h index 3b9b234c57453a..8ccd974e0b757a 100644 --- a/cloud/src/meta-store/keys.h +++ b/cloud/src/meta-store/keys.h @@ -218,9 +218,6 @@ using MetaPendingDeleteBitmapInfo = BasicKeyInfo<24 , std::tuple>; -// 0:instance_id 1:db_id 2:job_id -using StreamingJobMetaKeyInfo = BasicKeyInfo<52, std::tuple>; - // 0:instance_id 1:vault_id using StorageVaultKeyInfo = BasicKeyInfo<26, std::tuple>; @@ -410,8 +407,6 @@ void job_tablet_key(const JobTabletKeyInfo& in, std::string* out); static inline std::string job_tablet_key(const JobTabletKeyInfo& in) { std::string s; job_tablet_key(in, &s); return s; } void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out); static inline std::string rl_job_progress_key_info(const RLJobProgressKeyInfo& in) { std::string s; rl_job_progress_key_info(in, &s); return s; } -void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out); -static inline std::string streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in) { std::string s; streaming_job_meta_key_info(in, &s); return s; } std::string copy_key_prefix(std::string_view instance_id); void copy_job_key(const CopyJobKeyInfo& in, std::string* out); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java index 2b0673d6453d13..f17625a89eaaf2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java @@ -492,10 +492,4 @@ public Cloud. ResetRLProgressResponse resetRLProgress(Cloud. ResetRLProgressRequ return blockingStub.withDeadlineAfter(Config.meta_service_brpc_timeout_ms, TimeUnit.MILLISECONDS) .createInstance(request); } - - public Cloud.GetStreamingTaskCommitAttachResponse - getStreamingTaskCommitAttach(Cloud.GetStreamingTaskCommitAttachRequest request) { - return blockingStub.withDeadlineAfter(Config.meta_service_brpc_timeout_ms, TimeUnit.MILLISECONDS) - .getStreamingTaskCommitAttach(request); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java index 8710209ff8abab..b351942cbe2d11 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java @@ -432,9 +432,4 @@ public Cloud.ResetRLProgressResponse resetRLProgress(Cloud.ResetRLProgressReques public Cloud.CreateInstanceResponse createInstance(Cloud.CreateInstanceRequest request) throws RpcException { return w.executeRequest((client) -> client.createInstance(request)); } - - public Cloud.GetStreamingTaskCommitAttachResponse getStreamingTaskCommitAttach( - Cloud.GetStreamingTaskCommitAttachRequest request) throws RpcException { - return w.executeRequest((client) -> client.getStreamingTaskCommitAttach(request)); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index c3b9c321fa903a..fddb6ed720887b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -85,7 +85,6 @@ import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.event.DataChangeEvent; -import org.apache.doris.job.extensions.insert.streaming.StreamingTaskTxnCommitAttachment; import org.apache.doris.load.loadv2.LoadJobFinalOperation; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; import org.apache.doris.metric.MetricRepo; @@ -620,19 +619,6 @@ private void commitTransactionWithoutLock(long dbId, List
tableList, long } builder.setCommitAttachment(TxnUtil .rlTaskTxnCommitAttachmentToPb(rlTaskTxnCommitAttachment)); - } else if (txnCommitAttachment instanceof StreamingTaskTxnCommitAttachment) { - StreamingTaskTxnCommitAttachment streamingTaskTxnCommitAttachment = - (StreamingTaskTxnCommitAttachment) txnCommitAttachment; - TxnStateChangeCallback cb = callbackFactory.getCallback(streamingTaskTxnCommitAttachment.getTaskId()); - if (cb != null) { - // use a temporary transaction state to do before commit check, - // what actually works is the transactionId - TransactionState tmpTxnState = new TransactionState(); - tmpTxnState.setTransactionId(transactionId); - cb.beforeCommitted(tmpTxnState); - } - builder.setCommitAttachment(TxnUtil - .streamingTaskTxnCommitAttachmentToPb(streamingTaskTxnCommitAttachment)); } else { throw new UserException("invalid txnCommitAttachment"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java index 4155e6c5e67ab7..3aca54cd150dc6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java @@ -19,7 +19,6 @@ import org.apache.doris.cloud.proto.Cloud.RLTaskTxnCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.RoutineLoadProgressPB; -import org.apache.doris.cloud.proto.Cloud.StreamingTaskCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB.EtlStatusPB; @@ -29,7 +28,6 @@ import org.apache.doris.cloud.proto.Cloud.TxnInfoPB; import org.apache.doris.cloud.proto.Cloud.TxnSourceTypePB; import org.apache.doris.cloud.proto.Cloud.UniqueIdPB; -import org.apache.doris.job.extensions.insert.streaming.StreamingTaskTxnCommitAttachment; import org.apache.doris.load.EtlStatus; import org.apache.doris.load.FailMsg; import org.apache.doris.load.loadv2.JobState; @@ -270,42 +268,6 @@ public static RLTaskTxnCommitAttachment rtTaskTxnCommitAttachmentFromPb( return new RLTaskTxnCommitAttachment(txnCommitAttachmentPB.getRlTaskTxnCommitAttachment()); } - public static TxnCommitAttachmentPB streamingTaskTxnCommitAttachmentToPb(StreamingTaskTxnCommitAttachment - streamingTaskTxnCommitAttachment) { - if (LOG.isDebugEnabled()) { - LOG.debug("streamingTaskTxnCommitAttachment:{}", streamingTaskTxnCommitAttachment); - } - TxnCommitAttachmentPB.Builder attachementBuilder = TxnCommitAttachmentPB.newBuilder(); - attachementBuilder.setType(TxnCommitAttachmentPB.Type.STREAMING_TASK_TXN_COMMIT_ATTACHMENT); - - StreamingTaskCommitAttachmentPB.Builder builder = - StreamingTaskCommitAttachmentPB.newBuilder(); - - builder.setJobId(streamingTaskTxnCommitAttachment.getJobId()) - .setTaskId(streamingTaskTxnCommitAttachment.getTaskId()) - .setScannedRows(streamingTaskTxnCommitAttachment.getScannedRows()) - .setLoadBytes(streamingTaskTxnCommitAttachment.getLoadBytes()) - .setFileNumber(streamingTaskTxnCommitAttachment.getFileNumber()) - .setFileSize(streamingTaskTxnCommitAttachment.getFileSize()); - - if (streamingTaskTxnCommitAttachment.getOffset() != null) { - builder.setOffset(streamingTaskTxnCommitAttachment.getOffset().endOffset()); - } - - attachementBuilder.setStreamingTaskTxnCommitAttachment(builder.build()); - return attachementBuilder.build(); - } - - public static StreamingTaskTxnCommitAttachment streamingTaskTxnCommitAttachmentFromPb( - TxnCommitAttachmentPB txnCommitAttachmentPB) { - StreamingTaskCommitAttachmentPB streamingTaskCommitAttachmentPB = - txnCommitAttachmentPB.getStreamingTaskTxnCommitAttachment(); - if (LOG.isDebugEnabled()) { - LOG.debug("StreamingTaskCommitAttachmentPB={}", streamingTaskCommitAttachmentPB); - } - return new StreamingTaskTxnCommitAttachment(streamingTaskCommitAttachmentPB); - } - public static LoadJobFinalOperation loadJobFinalOperationFromPb(TxnCommitAttachmentPB txnCommitAttachmentPB) { LoadJobFinalOperationPB loadJobFinalOperationPB = txnCommitAttachmentPB.getLoadJobFinalOperation(); if (LOG.isDebugEnabled()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index b872040c4c354a..f2ce1ad5e02a36 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -19,10 +19,7 @@ import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; -import org.apache.doris.cloud.proto.Cloud; -import org.apache.doris.cloud.rpc.MetaServiceProxy; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; import org.apache.doris.common.InternalErrorCode; import org.apache.doris.common.UserException; @@ -52,7 +49,6 @@ import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.transaction.TransactionException; @@ -269,17 +265,6 @@ private void updateJobStatisticAndOffset(StreamingTaskTxnCommitAttachment attach offsetProvider.updateOffset(attachment.getOffset()); } - @Override - public void onRegister() throws JobException { - Env.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(this); - } - - @Override - public void onReplayCreate() throws JobException { - onRegister(); - super.onReplayCreate(); - } - @Override public ShowResultSetMetaData getTaskMetaData() { return InsertJob.TASK_META_DATA; @@ -405,35 +390,6 @@ public void replayOnCommitted(TransactionState txnState) { updateJobStatisticAndOffset(attachment); } - public void replayOnCloudMode() throws UserException { - Cloud.GetStreamingTaskCommitAttachRequest.Builder builder = - Cloud.GetStreamingTaskCommitAttachRequest.newBuilder(); - builder.setCloudUniqueId(Config.cloud_unique_id); - builder.setDbId(dbId); - builder.setJobId(getJobId()); - - Cloud.GetStreamingTaskCommitAttachResponse response; - try { - response = MetaServiceProxy.getInstance().getStreamingTaskCommitAttach(builder.build()); - if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { - log.warn("failed to get streaming task commit attach, response: {}", response); - if (response.getStatus().getCode() == Cloud.MetaServiceCode.STREAMING_JOB_PROGRESS_NOT_FOUND) { - log.warn("not found streaming job progress, response: {}", response); - return; - } else { - throw new UserException(response.getStatus().getMsg()); - } - } - } catch (RpcException e) { - log.info("failed to get streaming task commit attach {}", e); - throw new UserException(e.getMessage()); - } - - StreamingTaskTxnCommitAttachment commitAttach = - new StreamingTaskTxnCommitAttachment(response.getCommitAttach()); - updateJobStatisticAndOffset(commitAttach); - } - @Override public void afterAborted(TransactionState txnState, boolean txnOperated, String txnStatusChangeReason) throws UserException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java index 8660ed94739c0f..744f83080aac17 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java @@ -17,7 +17,6 @@ package org.apache.doris.job.extensions.insert.streaming; -import org.apache.doris.cloud.proto.Cloud.StreamingTaskCommitAttachmentPB; import org.apache.doris.job.offset.Offset; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnCommitAttachment; @@ -39,15 +38,6 @@ public StreamingTaskTxnCommitAttachment(long jobId, long taskId, this.offset = offset; } - public StreamingTaskTxnCommitAttachment(StreamingTaskCommitAttachmentPB pb) { - super(TransactionState.LoadJobSourceType.STREAMING_JOB); - this.scannedRows = pb.getScannedRows(); - this.loadBytes = pb.getLoadBytes(); - this.fileNumber = pb.getFileNumber(); - this.fileSize = pb.getFileSize(); - this.offset.setEndOffset(pb.getOffset()); - } - @Getter private long jobId; @Getter diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java index a3b0689bfc5466..095f0a5e6bf2bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java @@ -19,8 +19,4 @@ public interface Offset { String toJson(); - - void setEndOffset(String endOffset); - - String endOffset(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index 2ab2030fbbb3e9..f76707b2453074 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -30,16 +30,6 @@ public class S3Offset implements Offset { String endFile; String fileLists; - @Override - public void setEndOffset(String endOffset) { - this.endFile = endOffset; - } - - @Override - public String endOffset() { - return endFile; - } - @Override public String toJson() { return GsonUtils.GSON.toJson(this); diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index aa6bcd28359734..e277c24f8d7b94 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -322,7 +322,6 @@ enum LoadJobSourceTypePB { LOAD_JOB_SRC_TYPE_INSERT_STREAMING = 3; // insert stmt (streaming type), update stmt use this type LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK = 4; // routine load task use this type LOAD_JOB_SRC_TYPE_BATCH_LOAD_JOB = 5; // load job v2 for broker load - LOAD_JOB_SRC_TYPE_STREAMING_JOB = 6; // streaming job use this type } enum TxnStatusPB { @@ -366,21 +365,10 @@ message RoutineLoadJobStatisticPB { optional int64 task_execution_time_ms = 5; } -message StreamingTaskCommitAttachmentPB { - optional int64 job_id = 1; - optional int64 task_id = 2; - optional string offset = 3; - optional int64 scanned_rows = 4; - optional int64 load_bytes = 5; - optional int64 file_number = 6; - optional int64 file_size = 7; -} - message TxnCommitAttachmentPB { enum Type { LODD_JOB_FINAL_OPERATION = 0; RT_TASK_TXN_COMMIT_ATTACHMENT = 1; - STREAMING_TASK_TXN_COMMIT_ATTACHMENT = 2; } message LoadJobFinalOperationPB { message EtlStatusPB { @@ -437,7 +425,6 @@ message TxnCommitAttachmentPB { optional Type type = 1; optional LoadJobFinalOperationPB load_job_final_operation = 2; optional RLTaskTxnCommitAttachmentPB rl_task_txn_commit_attachment = 3; - optional StreamingTaskCommitAttachmentPB streaming_task_txn_commit_attachment = 4; } // For storing label -> txn_ids @@ -1711,8 +1698,7 @@ enum MetaServiceCode { JOB_ALREADY_SUCCESS = 5002; ROUTINE_LOAD_DATA_INCONSISTENT = 5003; ROUTINE_LOAD_PROGRESS_NOT_FOUND = 5004; - STREAMING_JOB_PROGRESS_NOT_FOUND = 5005; - JOB_CHECK_ALTER_VERSION = 5006; + JOB_CHECK_ALTER_VERSION = 5005; // Rate limit MAX_QPS_LIMIT = 6001; @@ -1890,18 +1876,6 @@ message ResetRLProgressResponse { optional MetaServiceResponseStatus status = 1; } -message GetStreamingTaskCommitAttachRequest { - optional string cloud_unique_id = 1; // For auth - optional int64 db_id = 2; - optional int64 job_id = 3; - optional string request_ip = 4; -} - -message GetStreamingTaskCommitAttachResponse { - optional MetaServiceResponseStatus status = 1; - optional StreamingTaskCommitAttachmentPB commit_attach = 2; -} - message CheckKeyInfos { repeated int64 db_ids = 1; repeated int64 table_ids = 2; @@ -2095,9 +2069,6 @@ service MetaService { rpc get_rl_task_commit_attach(GetRLTaskCommitAttachRequest) returns (GetRLTaskCommitAttachResponse); rpc reset_rl_progress(ResetRLProgressRequest) returns (ResetRLProgressResponse); - // streaming job meta - rpc get_streaming_task_commit_attach(GetStreamingTaskCommitAttachRequest) returns (GetStreamingTaskCommitAttachResponse); - // check KV rpc check_kv(CheckKVRequest) returns (CheckKVResponse); From 23c6fd771be575aa3cb1138f352387c8d32d0ca4 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 17 Sep 2025 17:27:32 +0800 Subject: [PATCH 16/25] Revert "[revert](streaming job) revert "implement offset persistence and replay in cloud mode"" (#56156) Reverts apache/doris#56149 --- cloud/src/common/bvars.cpp | 5 + cloud/src/common/bvars.h | 3 + cloud/src/meta-service/meta_service.h | 13 ++ cloud/src/meta-service/meta_service_txn.cpp | 137 ++++++++++++++++++ cloud/src/meta-store/keys.cpp | 13 +- cloud/src/meta-store/keys.h | 5 + .../doris/cloud/rpc/MetaServiceClient.java | 6 + .../doris/cloud/rpc/MetaServiceProxy.java | 5 + .../CloudGlobalTransactionMgr.java | 14 ++ .../doris/cloud/transaction/TxnUtil.java | 38 +++++ .../insert/streaming/StreamingInsertJob.java | 44 ++++++ .../StreamingTaskTxnCommitAttachment.java | 10 ++ .../org/apache/doris/job/offset/Offset.java | 4 + .../apache/doris/job/offset/s3/S3Offset.java | 10 ++ gensrc/proto/cloud.proto | 31 +++- 15 files changed, 335 insertions(+), 3 deletions(-) diff --git a/cloud/src/common/bvars.cpp b/cloud/src/common/bvars.cpp index e495b5ea95b956..8ea24c4a1baade 100644 --- a/cloud/src/common/bvars.cpp +++ b/cloud/src/common/bvars.cpp @@ -85,6 +85,7 @@ BvarLatencyRecorderWithTag g_bvar_ms_remove_delete_bitmap("ms", "remove_delete_b BvarLatencyRecorderWithTag g_bvar_ms_remove_delete_bitmap_update_lock("ms", "remove_delete_bitmap_update_lock"); BvarLatencyRecorderWithTag g_bvar_ms_get_instance("ms", "get_instance"); BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach("ms", "get_rl_task_commit_attach"); +BvarLatencyRecorderWithTag g_bvar_ms_get_streaming_task_commit_attach("ms", "get_streaming_task_commit_attach"); BvarLatencyRecorderWithTag g_bvar_ms_reset_rl_progress("ms", "reset_rl_progress"); BvarLatencyRecorderWithTag g_bvar_ms_get_txn_id("ms", "get_txn_id"); BvarLatencyRecorderWithTag g_bvar_ms_start_tablet_job("ms", "start_tablet_job"); @@ -364,6 +365,8 @@ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter("rpc_kv_precommit_txn_ge mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter("rpc_kv_precommit_txn_put_counter",{"instance_id"}); // get_rl_task_commit_attach mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter("rpc_kv_get_rl_task_commit_attach_get_counter",{"instance_id"}); +// get_streaming_task_commit_attach +mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_counter("rpc_kv_get_streaming_task_commit_attach_get_counter",{"instance_id"}); // reset_rl_progress mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter("rpc_kv_reset_rl_progress_get_counter",{"instance_id"}); mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter("rpc_kv_reset_rl_progress_put_counter",{"instance_id"}); @@ -527,6 +530,8 @@ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_bytes("rpc_kv_precommit_txn_get_ mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_bytes("rpc_kv_precommit_txn_put_bytes",{"instance_id"}); // get_rl_task_commit_attach mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_bytes("rpc_kv_get_rl_task_commit_attach_get_bytes",{"instance_id"}); +// get_streaming_task_commit_attach +mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_bytes("rpc_kv_get_streaming_task_commit_attach_get_bytes",{"instance_id"}); // reset_rl_progress mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_bytes("rpc_kv_reset_rl_progress_get_bytes",{"instance_id"}); mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_bytes("rpc_kv_reset_rl_progress_put_bytes",{"instance_id"}); diff --git a/cloud/src/common/bvars.h b/cloud/src/common/bvars.h index 8fb5973249fd2d..e72d0fcd3760d1 100644 --- a/cloud/src/common/bvars.h +++ b/cloud/src/common/bvars.h @@ -251,6 +251,7 @@ extern BvarLatencyRecorderWithTag g_bvar_ms_get_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status; extern BvarLatencyRecorderWithTag g_bvar_ms_get_instance; extern BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach; +extern BvarLatencyRecorderWithTag g_bvar_ms_get_streaming_task_commit_attach; extern BvarLatencyRecorderWithTag g_bvar_ms_reset_rl_progress; extern BvarLatencyRecorderWithTag g_bvar_ms_get_txn_id; extern BvarLatencyRecorderWithTag g_bvar_ms_check_kv; @@ -470,6 +471,7 @@ extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_counter; +extern mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_counter; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_counter; @@ -582,6 +584,7 @@ extern mBvarInt64Adder g_bvar_rpc_kv_begin_txn_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_precommit_txn_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_get_rl_task_commit_attach_get_bytes; +extern mBvarInt64Adder g_bvar_rpc_kv_get_streaming_task_commit_attach_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_get_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_put_bytes; extern mBvarInt64Adder g_bvar_rpc_kv_reset_rl_progress_del_bytes; diff --git a/cloud/src/meta-service/meta_service.h b/cloud/src/meta-service/meta_service.h index 38cc77aebb00af..c7a37277fdc083 100644 --- a/cloud/src/meta-service/meta_service.h +++ b/cloud/src/meta-service/meta_service.h @@ -319,6 +319,11 @@ class MetaServiceImpl : public cloud::MetaService { GetRLTaskCommitAttachResponse* response, ::google::protobuf::Closure* done) override; + void get_streaming_task_commit_attach(::google::protobuf::RpcController* controller, + const GetStreamingTaskCommitAttachRequest* request, + GetStreamingTaskCommitAttachResponse* response, + ::google::protobuf::Closure* done) override; + void reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, ::google::protobuf::Closure* done) override; @@ -820,6 +825,14 @@ class MetaServiceProxy final : public MetaService { done); } + void get_streaming_task_commit_attach(::google::protobuf::RpcController* controller, + const GetStreamingTaskCommitAttachRequest* request, + GetStreamingTaskCommitAttachResponse* response, + ::google::protobuf::Closure* done) override { + call_impl(&cloud::MetaService::get_streaming_task_commit_attach, controller, request, + response, done); + } + void reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, ::google::protobuf::Closure* done) override { diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index ce128f6d30e485..bd4cf6fb036f98 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -610,6 +610,75 @@ void put_routine_load_progress(MetaServiceCode& code, std::string& msg, << " routine load new progress: " << new_progress_info.ShortDebugString(); } +void put_streaming_job_meta(MetaServiceCode& code, std::string& msg, const std::string& instance_id, + const CommitTxnRequest* request, Transaction* txn, int64_t db_id) { + std::stringstream ss; + int64_t txn_id = request->txn_id(); + if (!request->has_commit_attachment()) { + ss << "failed to get commit attachment from req, db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + TxnCommitAttachmentPB txn_commit_attachment = request->commit_attachment(); + StreamingTaskCommitAttachmentPB commit_attachment = + txn_commit_attachment.streaming_task_txn_commit_attachment(); + int64_t job_id = commit_attachment.job_id(); + + std::string streaming_meta_key; + std::string streaming_meta_val; + bool prev_meta_existed = true; + StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; + streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); + TxnErrorCode err = txn->get(streaming_meta_key, &streaming_meta_val); + if (err != TxnErrorCode::TXN_OK) { + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + prev_meta_existed = false; + } else { + code = cast_as(err); + ss << "failed to get streaming job meta, db_id=" << db_id << " txn_id=" << txn_id + << " err=" << err; + msg = ss.str(); + return; + } + } + + StreamingTaskCommitAttachmentPB new_meta_info; + if (prev_meta_existed) { + if (!new_meta_info.ParseFromString(streaming_meta_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse streaming job meta, db_id=" << db_id << " txn_id=" << txn_id; + msg = ss.str(); + return; + } + new_meta_info.set_scanned_rows(new_meta_info.scanned_rows() + + commit_attachment.scanned_rows()); + new_meta_info.set_load_bytes(new_meta_info.load_bytes() + commit_attachment.load_bytes()); + new_meta_info.set_file_number(new_meta_info.file_number() + + commit_attachment.file_number()); + new_meta_info.set_file_size(new_meta_info.file_size() + commit_attachment.file_size()); + } else { + new_meta_info.set_job_id(commit_attachment.job_id()); + new_meta_info.set_scanned_rows(commit_attachment.scanned_rows()); + new_meta_info.set_load_bytes(commit_attachment.load_bytes()); + new_meta_info.set_file_number(commit_attachment.file_number()); + new_meta_info.set_file_size(commit_attachment.file_size()); + } + if (commit_attachment.has_offset()) { + new_meta_info.set_offset(commit_attachment.offset()); + } + std::string new_meta_val; + if (!new_meta_info.SerializeToString(&new_meta_val)) { + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "failed to serialize new streaming meta val, txn_id=" << txn_id; + msg = ss.str(); + return; + } + + txn->put(streaming_meta_key, new_meta_val); + LOG(INFO) << "put streaming_meta_key key=" << hex(streaming_meta_key) + << " streaming job new meta: " << new_meta_info.ShortDebugString(); +} + void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcController* controller, const GetRLTaskCommitAttachRequest* request, GetRLTaskCommitAttachResponse* response, @@ -678,6 +747,64 @@ void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcControlle } } +void MetaServiceImpl::get_streaming_task_commit_attach( + ::google::protobuf::RpcController* controller, + const GetStreamingTaskCommitAttachRequest* request, + GetStreamingTaskCommitAttachResponse* response, ::google::protobuf::Closure* done) { + RPC_PREPROCESS(get_streaming_task_commit_attach, get); + instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id()); + if (instance_id.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty instance_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + RPC_RATE_LIMIT(get_streaming_task_commit_attach) + + TxnErrorCode err = txn_kv_->create_txn(&txn); + if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "filed to create txn, err=" << err; + msg = ss.str(); + return; + } + + if (!request->has_db_id() || !request->has_job_id()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "empty db_id or job_id"; + LOG(INFO) << msg << ", cloud_unique_id=" << request->cloud_unique_id(); + return; + } + + int64_t db_id = request->db_id(); + int64_t job_id = request->job_id(); + std::string streaming_meta_key; + std::string streaming_meta_val; + StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; + streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); + err = txn->get(streaming_meta_key, &streaming_meta_val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = MetaServiceCode::STREAMING_JOB_PROGRESS_NOT_FOUND; + ss << "progress info not found, db_id=" << db_id << " job_id=" << job_id << " err=" << err; + msg = ss.str(); + return; + } else if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get progress info, db_id=" << db_id << " job_id=" << job_id + << " err=" << err; + msg = ss.str(); + return; + } + + StreamingTaskCommitAttachmentPB* commit_attach = response->mutable_commit_attach(); + if (!commit_attach->ParseFromString(streaming_meta_val)) { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + ss << "failed to parse meta info, db_id=" << db_id << " job_id=" << job_id; + msg = ss.str(); + return; + } +} + void MetaServiceImpl::reset_rl_progress(::google::protobuf::RpcController* controller, const ResetRLProgressRequest* request, ResetRLProgressResponse* response, @@ -1572,6 +1699,11 @@ void MetaServiceImpl::commit_txn_immediately( put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } + if (txn_info.load_job_source_type() == + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { + put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + } + LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) << " txn_id=" << txn_id; LOG(INFO) << "commit_txn put_size=" << txn->put_bytes() @@ -1965,6 +2097,11 @@ void MetaServiceImpl::commit_txn_eventually( put_routine_load_progress(code, msg, instance_id, request, txn.get(), db_id); } + if (txn_info.load_job_source_type() == + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { + put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + } + // save versions for partition int64_t version_update_time_ms = duration_cast(system_clock::now().time_since_epoch()).count(); diff --git a/cloud/src/meta-store/keys.cpp b/cloud/src/meta-store/keys.cpp index e23f84771aba04..7b2b75c4d555eb 100644 --- a/cloud/src/meta-store/keys.cpp +++ b/cloud/src/meta-store/keys.cpp @@ -64,6 +64,7 @@ static const char* STATS_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_RL_PROGRESS = "routine_load_progress"; +static const char* JOB_KEY_INFIX_STREAMING_JOB_META = "streaming_job_meta"; static const char* JOB_KEY_INFIX_RESTORE_TABLET = "restore_tablet"; static const char* JOB_KEY_INFIX_RESTORE_ROWSET = "restore_rowset"; @@ -144,7 +145,7 @@ static void encode_prefix(const T& t, std::string* key) { MetaDeleteBitmapInfo, MetaDeleteBitmapUpdateLockInfo, MetaPendingDeleteBitmapInfo, PartitionVersionKeyInfo, RecycleIndexKeyInfo, RecyclePartKeyInfo, RecycleRowsetKeyInfo, RecycleTxnKeyInfo, RecycleStageKeyInfo, StatsTabletKeyInfo, TableVersionKeyInfo, JobRestoreTabletKeyInfo, JobRestoreRowsetKeyInfo, - JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, + JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, StreamingJobMetaKeyInfo, CopyJobKeyInfo, CopyFileKeyInfo, StorageVaultKeyInfo, MetaSchemaPBDictionaryInfo, MowTabletJobInfo>); @@ -181,7 +182,8 @@ static void encode_prefix(const T& t, std::string* key) { encode_bytes(STATS_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v - || std::is_same_v) { + || std::is_same_v + || std::is_same_v) { encode_bytes(JOB_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v) { @@ -463,6 +465,13 @@ void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out) encode_int64(std::get<2>(in), out); // job_id } +void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "job" ${instance_id} + encode_bytes(JOB_KEY_INFIX_STREAMING_JOB_META, out); // "streaming_job_meta" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // job_id +} + void job_restore_tablet_key(const JobRestoreTabletKeyInfo& in, std::string* out) { encode_prefix(in, out); // 0x01 "job" ${instance_id} encode_bytes(JOB_KEY_INFIX_RESTORE_TABLET, out); // "restore_tablet" diff --git a/cloud/src/meta-store/keys.h b/cloud/src/meta-store/keys.h index 8ccd974e0b757a..3b9b234c57453a 100644 --- a/cloud/src/meta-store/keys.h +++ b/cloud/src/meta-store/keys.h @@ -218,6 +218,9 @@ using MetaPendingDeleteBitmapInfo = BasicKeyInfo<24 , std::tuple>; +// 0:instance_id 1:db_id 2:job_id +using StreamingJobMetaKeyInfo = BasicKeyInfo<52, std::tuple>; + // 0:instance_id 1:vault_id using StorageVaultKeyInfo = BasicKeyInfo<26, std::tuple>; @@ -407,6 +410,8 @@ void job_tablet_key(const JobTabletKeyInfo& in, std::string* out); static inline std::string job_tablet_key(const JobTabletKeyInfo& in) { std::string s; job_tablet_key(in, &s); return s; } void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out); static inline std::string rl_job_progress_key_info(const RLJobProgressKeyInfo& in) { std::string s; rl_job_progress_key_info(in, &s); return s; } +void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out); +static inline std::string streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in) { std::string s; streaming_job_meta_key_info(in, &s); return s; } std::string copy_key_prefix(std::string_view instance_id); void copy_job_key(const CopyJobKeyInfo& in, std::string* out); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java index f17625a89eaaf2..2b0673d6453d13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceClient.java @@ -492,4 +492,10 @@ public Cloud. ResetRLProgressResponse resetRLProgress(Cloud. ResetRLProgressRequ return blockingStub.withDeadlineAfter(Config.meta_service_brpc_timeout_ms, TimeUnit.MILLISECONDS) .createInstance(request); } + + public Cloud.GetStreamingTaskCommitAttachResponse + getStreamingTaskCommitAttach(Cloud.GetStreamingTaskCommitAttachRequest request) { + return blockingStub.withDeadlineAfter(Config.meta_service_brpc_timeout_ms, TimeUnit.MILLISECONDS) + .getStreamingTaskCommitAttach(request); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java index b351942cbe2d11..8710209ff8abab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/rpc/MetaServiceProxy.java @@ -432,4 +432,9 @@ public Cloud.ResetRLProgressResponse resetRLProgress(Cloud.ResetRLProgressReques public Cloud.CreateInstanceResponse createInstance(Cloud.CreateInstanceRequest request) throws RpcException { return w.executeRequest((client) -> client.createInstance(request)); } + + public Cloud.GetStreamingTaskCommitAttachResponse getStreamingTaskCommitAttach( + Cloud.GetStreamingTaskCommitAttachRequest request) throws RpcException { + return w.executeRequest((client) -> client.getStreamingTaskCommitAttach(request)); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java index fddb6ed720887b..c3b9c321fa903a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java @@ -85,6 +85,7 @@ import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.event.DataChangeEvent; +import org.apache.doris.job.extensions.insert.streaming.StreamingTaskTxnCommitAttachment; import org.apache.doris.load.loadv2.LoadJobFinalOperation; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; import org.apache.doris.metric.MetricRepo; @@ -619,6 +620,19 @@ private void commitTransactionWithoutLock(long dbId, List
tableList, long } builder.setCommitAttachment(TxnUtil .rlTaskTxnCommitAttachmentToPb(rlTaskTxnCommitAttachment)); + } else if (txnCommitAttachment instanceof StreamingTaskTxnCommitAttachment) { + StreamingTaskTxnCommitAttachment streamingTaskTxnCommitAttachment = + (StreamingTaskTxnCommitAttachment) txnCommitAttachment; + TxnStateChangeCallback cb = callbackFactory.getCallback(streamingTaskTxnCommitAttachment.getTaskId()); + if (cb != null) { + // use a temporary transaction state to do before commit check, + // what actually works is the transactionId + TransactionState tmpTxnState = new TransactionState(); + tmpTxnState.setTransactionId(transactionId); + cb.beforeCommitted(tmpTxnState); + } + builder.setCommitAttachment(TxnUtil + .streamingTaskTxnCommitAttachmentToPb(streamingTaskTxnCommitAttachment)); } else { throw new UserException("invalid txnCommitAttachment"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java index 3aca54cd150dc6..4155e6c5e67ab7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java @@ -19,6 +19,7 @@ import org.apache.doris.cloud.proto.Cloud.RLTaskTxnCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.RoutineLoadProgressPB; +import org.apache.doris.cloud.proto.Cloud.StreamingTaskCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB; import org.apache.doris.cloud.proto.Cloud.TxnCommitAttachmentPB.LoadJobFinalOperationPB.EtlStatusPB; @@ -28,6 +29,7 @@ import org.apache.doris.cloud.proto.Cloud.TxnInfoPB; import org.apache.doris.cloud.proto.Cloud.TxnSourceTypePB; import org.apache.doris.cloud.proto.Cloud.UniqueIdPB; +import org.apache.doris.job.extensions.insert.streaming.StreamingTaskTxnCommitAttachment; import org.apache.doris.load.EtlStatus; import org.apache.doris.load.FailMsg; import org.apache.doris.load.loadv2.JobState; @@ -268,6 +270,42 @@ public static RLTaskTxnCommitAttachment rtTaskTxnCommitAttachmentFromPb( return new RLTaskTxnCommitAttachment(txnCommitAttachmentPB.getRlTaskTxnCommitAttachment()); } + public static TxnCommitAttachmentPB streamingTaskTxnCommitAttachmentToPb(StreamingTaskTxnCommitAttachment + streamingTaskTxnCommitAttachment) { + if (LOG.isDebugEnabled()) { + LOG.debug("streamingTaskTxnCommitAttachment:{}", streamingTaskTxnCommitAttachment); + } + TxnCommitAttachmentPB.Builder attachementBuilder = TxnCommitAttachmentPB.newBuilder(); + attachementBuilder.setType(TxnCommitAttachmentPB.Type.STREAMING_TASK_TXN_COMMIT_ATTACHMENT); + + StreamingTaskCommitAttachmentPB.Builder builder = + StreamingTaskCommitAttachmentPB.newBuilder(); + + builder.setJobId(streamingTaskTxnCommitAttachment.getJobId()) + .setTaskId(streamingTaskTxnCommitAttachment.getTaskId()) + .setScannedRows(streamingTaskTxnCommitAttachment.getScannedRows()) + .setLoadBytes(streamingTaskTxnCommitAttachment.getLoadBytes()) + .setFileNumber(streamingTaskTxnCommitAttachment.getFileNumber()) + .setFileSize(streamingTaskTxnCommitAttachment.getFileSize()); + + if (streamingTaskTxnCommitAttachment.getOffset() != null) { + builder.setOffset(streamingTaskTxnCommitAttachment.getOffset().endOffset()); + } + + attachementBuilder.setStreamingTaskTxnCommitAttachment(builder.build()); + return attachementBuilder.build(); + } + + public static StreamingTaskTxnCommitAttachment streamingTaskTxnCommitAttachmentFromPb( + TxnCommitAttachmentPB txnCommitAttachmentPB) { + StreamingTaskCommitAttachmentPB streamingTaskCommitAttachmentPB = + txnCommitAttachmentPB.getStreamingTaskTxnCommitAttachment(); + if (LOG.isDebugEnabled()) { + LOG.debug("StreamingTaskCommitAttachmentPB={}", streamingTaskCommitAttachmentPB); + } + return new StreamingTaskTxnCommitAttachment(streamingTaskCommitAttachmentPB); + } + public static LoadJobFinalOperation loadJobFinalOperationFromPb(TxnCommitAttachmentPB txnCommitAttachmentPB) { LoadJobFinalOperationPB loadJobFinalOperationPB = txnCommitAttachmentPB.getLoadJobFinalOperation(); if (LOG.isDebugEnabled()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index f2ce1ad5e02a36..b872040c4c354a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -19,7 +19,10 @@ import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.Env; +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.rpc.MetaServiceProxy; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; import org.apache.doris.common.InternalErrorCode; import org.apache.doris.common.UserException; @@ -49,6 +52,7 @@ import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.transaction.TransactionException; @@ -265,6 +269,17 @@ private void updateJobStatisticAndOffset(StreamingTaskTxnCommitAttachment attach offsetProvider.updateOffset(attachment.getOffset()); } + @Override + public void onRegister() throws JobException { + Env.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(this); + } + + @Override + public void onReplayCreate() throws JobException { + onRegister(); + super.onReplayCreate(); + } + @Override public ShowResultSetMetaData getTaskMetaData() { return InsertJob.TASK_META_DATA; @@ -390,6 +405,35 @@ public void replayOnCommitted(TransactionState txnState) { updateJobStatisticAndOffset(attachment); } + public void replayOnCloudMode() throws UserException { + Cloud.GetStreamingTaskCommitAttachRequest.Builder builder = + Cloud.GetStreamingTaskCommitAttachRequest.newBuilder(); + builder.setCloudUniqueId(Config.cloud_unique_id); + builder.setDbId(dbId); + builder.setJobId(getJobId()); + + Cloud.GetStreamingTaskCommitAttachResponse response; + try { + response = MetaServiceProxy.getInstance().getStreamingTaskCommitAttach(builder.build()); + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + log.warn("failed to get streaming task commit attach, response: {}", response); + if (response.getStatus().getCode() == Cloud.MetaServiceCode.STREAMING_JOB_PROGRESS_NOT_FOUND) { + log.warn("not found streaming job progress, response: {}", response); + return; + } else { + throw new UserException(response.getStatus().getMsg()); + } + } + } catch (RpcException e) { + log.info("failed to get streaming task commit attach {}", e); + throw new UserException(e.getMessage()); + } + + StreamingTaskTxnCommitAttachment commitAttach = + new StreamingTaskTxnCommitAttachment(response.getCommitAttach()); + updateJobStatisticAndOffset(commitAttach); + } + @Override public void afterAborted(TransactionState txnState, boolean txnOperated, String txnStatusChangeReason) throws UserException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java index 744f83080aac17..8660ed94739c0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java @@ -17,6 +17,7 @@ package org.apache.doris.job.extensions.insert.streaming; +import org.apache.doris.cloud.proto.Cloud.StreamingTaskCommitAttachmentPB; import org.apache.doris.job.offset.Offset; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnCommitAttachment; @@ -38,6 +39,15 @@ public StreamingTaskTxnCommitAttachment(long jobId, long taskId, this.offset = offset; } + public StreamingTaskTxnCommitAttachment(StreamingTaskCommitAttachmentPB pb) { + super(TransactionState.LoadJobSourceType.STREAMING_JOB); + this.scannedRows = pb.getScannedRows(); + this.loadBytes = pb.getLoadBytes(); + this.fileNumber = pb.getFileNumber(); + this.fileSize = pb.getFileSize(); + this.offset.setEndOffset(pb.getOffset()); + } + @Getter private long jobId; @Getter diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java index 095f0a5e6bf2bc..a3b0689bfc5466 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java @@ -19,4 +19,8 @@ public interface Offset { String toJson(); + + void setEndOffset(String endOffset); + + String endOffset(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index f76707b2453074..2ab2030fbbb3e9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -30,6 +30,16 @@ public class S3Offset implements Offset { String endFile; String fileLists; + @Override + public void setEndOffset(String endOffset) { + this.endFile = endOffset; + } + + @Override + public String endOffset() { + return endFile; + } + @Override public String toJson() { return GsonUtils.GSON.toJson(this); diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index e277c24f8d7b94..aa6bcd28359734 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -322,6 +322,7 @@ enum LoadJobSourceTypePB { LOAD_JOB_SRC_TYPE_INSERT_STREAMING = 3; // insert stmt (streaming type), update stmt use this type LOAD_JOB_SRC_TYPE_ROUTINE_LOAD_TASK = 4; // routine load task use this type LOAD_JOB_SRC_TYPE_BATCH_LOAD_JOB = 5; // load job v2 for broker load + LOAD_JOB_SRC_TYPE_STREAMING_JOB = 6; // streaming job use this type } enum TxnStatusPB { @@ -365,10 +366,21 @@ message RoutineLoadJobStatisticPB { optional int64 task_execution_time_ms = 5; } +message StreamingTaskCommitAttachmentPB { + optional int64 job_id = 1; + optional int64 task_id = 2; + optional string offset = 3; + optional int64 scanned_rows = 4; + optional int64 load_bytes = 5; + optional int64 file_number = 6; + optional int64 file_size = 7; +} + message TxnCommitAttachmentPB { enum Type { LODD_JOB_FINAL_OPERATION = 0; RT_TASK_TXN_COMMIT_ATTACHMENT = 1; + STREAMING_TASK_TXN_COMMIT_ATTACHMENT = 2; } message LoadJobFinalOperationPB { message EtlStatusPB { @@ -425,6 +437,7 @@ message TxnCommitAttachmentPB { optional Type type = 1; optional LoadJobFinalOperationPB load_job_final_operation = 2; optional RLTaskTxnCommitAttachmentPB rl_task_txn_commit_attachment = 3; + optional StreamingTaskCommitAttachmentPB streaming_task_txn_commit_attachment = 4; } // For storing label -> txn_ids @@ -1698,7 +1711,8 @@ enum MetaServiceCode { JOB_ALREADY_SUCCESS = 5002; ROUTINE_LOAD_DATA_INCONSISTENT = 5003; ROUTINE_LOAD_PROGRESS_NOT_FOUND = 5004; - JOB_CHECK_ALTER_VERSION = 5005; + STREAMING_JOB_PROGRESS_NOT_FOUND = 5005; + JOB_CHECK_ALTER_VERSION = 5006; // Rate limit MAX_QPS_LIMIT = 6001; @@ -1876,6 +1890,18 @@ message ResetRLProgressResponse { optional MetaServiceResponseStatus status = 1; } +message GetStreamingTaskCommitAttachRequest { + optional string cloud_unique_id = 1; // For auth + optional int64 db_id = 2; + optional int64 job_id = 3; + optional string request_ip = 4; +} + +message GetStreamingTaskCommitAttachResponse { + optional MetaServiceResponseStatus status = 1; + optional StreamingTaskCommitAttachmentPB commit_attach = 2; +} + message CheckKeyInfos { repeated int64 db_ids = 1; repeated int64 table_ids = 2; @@ -2069,6 +2095,9 @@ service MetaService { rpc get_rl_task_commit_attach(GetRLTaskCommitAttachRequest) returns (GetRLTaskCommitAttachResponse); rpc reset_rl_progress(ResetRLProgressRequest) returns (ResetRLProgressResponse); + // streaming job meta + rpc get_streaming_task_commit_attach(GetStreamingTaskCommitAttachRequest) returns (GetStreamingTaskCommitAttachResponse); + // check KV rpc check_kv(CheckKVRequest) returns (CheckKVResponse); From b2adc933555c4654984cac65c861a0a089b5da4d Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 17 Sep 2025 22:35:48 +0800 Subject: [PATCH 17/25] [Feature](WIP) Fix Alter Job and schedule bug etc (#56166) ### What problem does this PR solve? Fix Alter Job and schedule bug etc --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../doris/cloud/transaction/TxnUtil.java | 2 +- .../org/apache/doris/fs/obj/S3ObjStorage.java | 7 ++- .../job/base/JobExecutionConfiguration.java | 17 +------ .../{PauseReason.java => FailureReason.java} | 8 ++- .../job/extensions/insert/InsertJob.java | 2 + .../job/extensions/insert/InsertTask.java | 4 +- .../insert/streaming/StreamingInsertJob.java | 44 ++++++++++++---- .../insert/streaming/StreamingInsertTask.java | 4 +- .../streaming/StreamingJobSchedulerTask.java | 50 +++++++++++++----- .../StreamingTaskTxnCommitAttachment.java | 9 ++-- .../job/offset/SourceOffsetProvider.java | 6 +++ .../apache/doris/job/offset/s3/S3Offset.java | 2 + .../job/offset/s3/S3SourceOffsetProvider.java | 9 +++- .../job/scheduler/StreamingTaskScheduler.java | 2 + .../nereids/parser/LogicalPlanBuilder.java | 3 +- .../trees/plans/commands/AlterJobCommand.java | 4 +- .../apache/doris/persist/gson/GsonUtils.java | 5 +- .../doris/catalog/DropFunctionTest.java | 11 +--- .../test_streaming_insert_job.groovy | 51 +++++++++++++++++-- 20 files changed, 172 insertions(+), 70 deletions(-) rename fe/fe-core/src/main/java/org/apache/doris/job/common/{PauseReason.java => FailureReason.java} (91%) diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index a0ae7c6916e636..4b6dd3668d4ac5 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -111,7 +111,7 @@ supportedJobStatement commentSpec? DO supportedDmlStatement #createScheduledJob | PAUSE JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #pauseJob - | ALTER JOB FOR (jobNameKey=identifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob + | ALTER JOB FOR (jobName=multipartIdentifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob | DROP JOB (IF EXISTS)? WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #dropJob | RESUME JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #resumeJob | CANCEL TASK WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) AND (taskIdKey=identifier) EQ (taskIdValue=INTEGER_VALUE) #cancelJobTask diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java index 4155e6c5e67ab7..424ce238544410 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java @@ -289,7 +289,7 @@ public static TxnCommitAttachmentPB streamingTaskTxnCommitAttachmentToPb(Streami .setFileSize(streamingTaskTxnCommitAttachment.getFileSize()); if (streamingTaskTxnCommitAttachment.getOffset() != null) { - builder.setOffset(streamingTaskTxnCommitAttachment.getOffset().endOffset()); + builder.setOffset(streamingTaskTxnCommitAttachment.getOffset()); } attachementBuilder.setStreamingTaskTxnCommitAttachment(builder.build()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java index 4c79df0acf705a..9b0e5aeb80194d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java @@ -682,6 +682,7 @@ public String globListWithLimit(String remotePath, List result, Stri String currentMaxFile = ""; boolean isTruncated = false; + boolean reachLimit = false; do { roundCnt++; ListObjectsV2Response response = listObjectsV2(request); @@ -716,12 +717,16 @@ public String globListWithLimit(String remotePath, List result, Stri result.add(remoteFile); if (reachLimit(result.size(), matchFileSize, fileSizeLimit, fileNumLimit)) { + reachLimit = true; break; } objPath = objPath.getParent(); isPrefix = true; } + if (reachLimit) { + break; + } } //record current last object file name S3Object lastS3Object = response.contents().get(response.contents().size() - 1); @@ -733,7 +738,7 @@ public String globListWithLimit(String remotePath, List result, Stri .continuationToken(response.nextContinuationToken()) .build(); } - } while (isTruncated); + } while (isTruncated && !reachLimit); if (LOG.isDebugEnabled()) { LOG.debug("remotePath:{}, result:{}", remotePath, result); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java index 3d8c9afa36b290..423b436979f0d8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java @@ -124,22 +124,7 @@ public List getTriggerDelayTimes(Long currentTimeMs, Long startTimeMs, Lon return delayTimeSeconds; } - if (JobExecuteType.STREAMING.equals(executeType) && null != timerDefinition) { - if (null == timerDefinition.getStartTimeMs() || null != timerDefinition.getLatestSchedulerTimeMs()) { - return delayTimeSeconds; - } - - // If the job is already executed or in the schedule queue, or not within this schedule window - if (endTimeMs < timerDefinition.getStartTimeMs()) { - return delayTimeSeconds; - } - - delayTimeSeconds.add(queryDelayTimeSecond(currentTimeMs, timerDefinition.getStartTimeMs())); - this.timerDefinition.setLatestSchedulerTimeMs(timerDefinition.getStartTimeMs()); - return delayTimeSeconds; - } - - if (JobExecuteType.RECURRING.equals(executeType)) { + if (JobExecuteType.RECURRING.equals(executeType) || JobExecuteType.STREAMING.equals(executeType)) { if (timerDefinition.getStartTimeMs() > endTimeMs || null != timerDefinition.getEndTimeMs() && timerDefinition.getEndTimeMs() < startTimeMs) { return delayTimeSeconds; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/PauseReason.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/FailureReason.java similarity index 91% rename from fe/fe-core/src/main/java/org/apache/doris/job/common/PauseReason.java rename to fe/fe-core/src/main/java/org/apache/doris/job/common/FailureReason.java index 49a46327b32756..2acfb472a40c0c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/PauseReason.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/FailureReason.java @@ -27,17 +27,21 @@ import java.io.DataOutput; import java.io.IOException; -public class PauseReason implements Writable { +public class FailureReason implements Writable { @SerializedName(value = "code") private InternalErrorCode code; @SerializedName(value = "msg") private String msg; - public PauseReason(InternalErrorCode errCode, String msg) { + public FailureReason(InternalErrorCode errCode, String msg) { this.code = errCode; this.msg = msg; } + public FailureReason(String msg) { + this.msg = msg; + } + public InternalErrorCode getCode() { return code; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index 67b56dbc13df20..a4d6793b0b8bcb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -99,6 +99,7 @@ public class InsertJob extends AbstractJob> impl .addAll(COMMON_SCHEMA) .add(new Column("Comment", ScalarType.createStringType())) // only execute type = streaming need record + .add(new Column("Properties", ScalarType.createStringType())) .add(new Column("Progress", ScalarType.createStringType())) .add(new Column("RemoteOffset", ScalarType.createStringType())) .add(new Column("LoadStatistic", ScalarType.createStringType())) @@ -549,6 +550,7 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal(getComment())); trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); trow.addToColumnValue(new TCell().setStringVal( loadStatistic == null ? FeConstants.null_string : loadStatistic.toJson())); trow.addToColumnValue(new TCell().setStringVal(failMsg == null ? FeConstants.null_string : failMsg.getMsg())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index 34cfdf6edea874..261ca012311704 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -68,7 +68,8 @@ public class InsertTask extends AbstractTask { new Column("TrackingUrl", ScalarType.createStringType()), new Column("LoadStatistic", ScalarType.createStringType()), new Column("User", ScalarType.createStringType()), - new Column("Offset", ScalarType.createStringType())); + new Column("Offset", ScalarType.createStringType()), + new Column("OtherMsg", ScalarType.createStringType())); public static final ImmutableMap COLUMN_TO_INDEX; @@ -295,6 +296,7 @@ private TRow getPendingTaskTVFInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal("")); return trow; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index b872040c4c354a..157629520a9cb4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -24,7 +24,6 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; -import org.apache.doris.common.InternalErrorCode; import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; import org.apache.doris.common.util.TimeUtils; @@ -32,10 +31,10 @@ import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.common.FailureReason; import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; -import org.apache.doris.job.common.PauseReason; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; @@ -72,6 +71,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; @Log4j2 @@ -80,7 +80,8 @@ public class StreamingInsertJob extends AbstractJob loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIds); if (loadJobs.size() != 1) { + shouldRealseLock = true; throw new TransactionException("load job not found, insert job id is " + runningStreamTask.getTaskId()); } LoadJob loadJob = loadJobs.get(0); @@ -377,7 +387,7 @@ public void beforeCommitted(TransactionState txnState) throws TransactionExcepti loadStatistic.getLoadBytes(), loadStatistic.getFileNumber(), loadStatistic.getTotalFileSizeB(), - runningStreamTask.getRunningOffset())); + runningStreamTask.getRunningOffset().toJson())); } finally { if (shouldRealseLock) { lock.writeLock().unlock(); @@ -387,6 +397,7 @@ public void beforeCommitted(TransactionState txnState) throws TransactionExcepti @Override public void beforeAborted(TransactionState txnState) throws TransactionException { + } @Override @@ -403,6 +414,7 @@ public void replayOnCommitted(TransactionState txnState) { StreamingTaskTxnCommitAttachment attachment = (StreamingTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(); updateJobStatisticAndOffset(attachment); + succeedTaskCount.incrementAndGet(); } public void replayOnCloudMode() throws UserException { @@ -462,5 +474,19 @@ public void gsonPostProcess() throws IOException { if (jobProperties == null && properties != null) { jobProperties = new StreamingJobProperties(properties); } + + if (null == getSucceedTaskCount()) { + setSucceedTaskCount(new AtomicLong(0)); + } + if (null == getFailedTaskCount()) { + setFailedTaskCount(new AtomicLong(0)); + } + if (null == getCanceledTaskCount()) { + setCanceledTaskCount(new AtomicLong(0)); + } + + if (null == lock) { + this.lock = new ReentrantReadWriteLock(true); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index c171173fd3c772..3073142c151b09 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -55,6 +55,8 @@ public class StreamingInsertTask { @Setter private TaskStatus status; private String errMsg; + @Setter + private String otherMsg; private Long createTimeMs; private Long startTimeMs; private Long finishTimeMs; @@ -99,8 +101,8 @@ public void execute() throws JobException { if (TaskStatus.CANCELED.equals(status)) { return; } - onFail(e.getMessage()); log.warn("execute task error, job id is {}, task id is {}", jobId, taskId, e); + onFail(e.getMessage()); } finally { // The cancel logic will call the closeOrReleased Resources method by itself. // If it is also called here, diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index 6f083a82c553c9..b658c97c828f31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -17,15 +17,21 @@ package org.apache.doris.job.extensions.insert.streaming; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.InternalErrorCode; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.common.FailureReason; import org.apache.doris.job.common.JobStatus; -import org.apache.doris.job.common.PauseReason; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; +import java.util.Arrays; +import java.util.List; + public class StreamingJobSchedulerTask extends AbstractTask { private static final long BACK_OFF_BASIC_TIME_SEC = 10L; private static final long MAX_BACK_OFF_TIME_SEC = 60 * 5; @@ -55,15 +61,15 @@ public void run() throws JobException { } private void autoResumeHandler() throws JobException { - final PauseReason pauseReason = streamingInsertJob.getPauseReason(); + final FailureReason failureReason = streamingInsertJob.getFailureReason(); final long latestAutoResumeTimestamp = streamingInsertJob.getLatestAutoResumeTimestamp(); final long autoResumeCount = streamingInsertJob.getAutoResumeCount(); final long current = System.currentTimeMillis(); - if (pauseReason != null - && pauseReason.getCode() != InternalErrorCode.MANUAL_PAUSE_ERR - && pauseReason.getCode() != InternalErrorCode.TOO_MANY_FAILURE_ROWS_ERR - && pauseReason.getCode() != InternalErrorCode.CANNOT_RESUME_ERR) { + if (failureReason != null + && failureReason.getCode() != InternalErrorCode.MANUAL_PAUSE_ERR + && failureReason.getCode() != InternalErrorCode.TOO_MANY_FAILURE_ROWS_ERR + && failureReason.getCode() != InternalErrorCode.CANNOT_RESUME_ERR) { long autoResumeIntervalTimeSec = autoResumeCount < 5 ? Math.min((long) Math.pow(2, autoResumeCount) * BACK_OFF_BASIC_TIME_SEC, MAX_BACK_OFF_TIME_SEC) : MAX_BACK_OFF_TIME_SEC; @@ -107,20 +113,40 @@ public TRow getTvfInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(runningTask.getErrMsg())); // create time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(runningTask.getCreateTimeMs()))); - trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? "" + trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? FeConstants.null_string : TimeUtils.longToTimeString(runningTask.getStartTimeMs()))); // load end time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(runningTask.getFinishTimeMs()))); - // tracking url - trow.addToColumnValue(new TCell().setStringVal("trackingUrl")); - trow.addToColumnValue(new TCell().setStringVal("statistic")); + + List loadJobs = Env.getCurrentEnv().getLoadManager() + .queryLoadJobsByJobIds(Arrays.asList(runningTask.getTaskId())); + if (!loadJobs.isEmpty()) { + LoadJob loadJob = loadJobs.get(0); + if (loadJob.getLoadingStatus() != null && loadJob.getLoadingStatus().getTrackingUrl() != null) { + trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadingStatus().getTrackingUrl())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + + if (loadJob.getLoadStatistic() != null) { + trow.addToColumnValue(new TCell().setStringVal(loadJob.getLoadStatistic().toJson())); + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + } else { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } + if (runningTask.getUserIdentity() == null) { - trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); } else { trow.addToColumnValue(new TCell().setStringVal(runningTask.getUserIdentity().getQualifiedUser())); } - trow.addToColumnValue(new TCell().setStringVal(runningTask.getRunningOffset() == null ? "" + trow.addToColumnValue(new TCell().setStringVal(runningTask.getRunningOffset() == null ? FeConstants.null_string : runningTask.getRunningOffset().toJson())); + trow.addToColumnValue(new TCell().setStringVal(null == runningTask.getOtherMsg() + ? FeConstants.null_string : runningTask.getOtherMsg())); return trow; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java index 8660ed94739c0f..4b7590824b4ee2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java @@ -18,7 +18,6 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.cloud.proto.Cloud.StreamingTaskCommitAttachmentPB; -import org.apache.doris.job.offset.Offset; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TxnCommitAttachment; @@ -28,7 +27,7 @@ public class StreamingTaskTxnCommitAttachment extends TxnCommitAttachment { public StreamingTaskTxnCommitAttachment(long jobId, long taskId, - long scannedRows, long loadBytes, long fileNumber, long fileSize, Offset offset) { + long scannedRows, long loadBytes, long fileNumber, long fileSize, String offset) { super(TransactionState.LoadJobSourceType.STREAMING_JOB); this.jobId = jobId; this.taskId = taskId; @@ -45,7 +44,7 @@ public StreamingTaskTxnCommitAttachment(StreamingTaskCommitAttachmentPB pb) { this.loadBytes = pb.getLoadBytes(); this.fileNumber = pb.getFileNumber(); this.fileSize = pb.getFileSize(); - this.offset.setEndOffset(pb.getOffset()); + this.offset = pb.getOffset(); } @Getter @@ -66,7 +65,7 @@ public StreamingTaskTxnCommitAttachment(StreamingTaskCommitAttachmentPB pb) { private long fileSize; @SerializedName(value = "of") @Getter - private Offset offset; + private String offset; @Override public String toString() { @@ -75,7 +74,7 @@ public String toString() { + ", loadBytes=" + loadBytes + ", fileNumber=" + fileNumber + ", fileSize=" + fileSize - + ", offset=" + offset.toString() + + ", offset=" + offset + "]"; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index e670dac553cf4b..d7e1bee4669384 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -81,5 +81,11 @@ public interface SourceOffsetProvider { */ boolean hasMoreDataToConsume(); + /** + * Deserialize string offset to Offset + * @return + */ + Offset deserializeOffset(String offset); + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index 2ab2030fbbb3e9..3d260218886223 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -20,6 +20,7 @@ import org.apache.doris.job.offset.Offset; import org.apache.doris.persist.gson.GsonUtils; +import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; @@ -27,6 +28,7 @@ @Setter public class S3Offset implements Offset { String startFile; + @SerializedName("ef") String endFile; String fileLists; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index f63333468fa426..e429ae7375b151 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.persist.gson.GsonUtils; import com.google.common.collect.Maps; import lombok.extern.log4j.Log4j2; @@ -72,8 +73,7 @@ public S3Offset getNextOffset(StreamingJobProperties jobProps, Map { Env.getCurrentEnv().getJobManager().getStreamingTaskManager().registerTask(task); }, delayMs, TimeUnit.MILLISECONDS); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 80a0886017ea6d..a7b67f2ffda584 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -1163,11 +1163,10 @@ private void checkJobNameKey(String key, String keyFormat, DorisParser.Supported @Override public LogicalPlan visitAlterJob(DorisParser.AlterJobContext ctx) { - checkJobNameKey(stripQuotes(ctx.jobNameKey.getText()), JOB_NAME, ctx); Map properties = ctx.propertyClause() != null ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); String executeSql = getOriginSql(ctx.supportedDmlStatement()); - return new AlterJobCommand(stripQuotes(ctx.jobNameKey.getText()), properties, executeSql); + return new AlterJobCommand(ctx.jobName.getText(), properties, executeSql); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index b1823de4a1d50f..786c8184e407ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -80,7 +80,7 @@ private AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws JobExcepti Map updateProps = properties == null || properties.isEmpty() ? originJob.getProperties() : properties; - return new StreamingInsertJob(jobName, + StreamingInsertJob streamingInsertJob = new StreamingInsertJob(jobName, job.getJobStatus(), job.getCurrentDbName(), job.getComment(), @@ -89,6 +89,8 @@ private AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws JobExcepti System.currentTimeMillis(), updateSQL, updateProps); + streamingInsertJob.setJobId(job.getJobId()); + return streamingInsertJob; } else { throw new JobException("Unsupported job type for ALTER:" + job.getJobType()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index af79329f681930..dffd7a8c80e7db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -183,6 +183,7 @@ import org.apache.doris.fs.remote.dfs.OFSFileSystem; import org.apache.doris.job.extensions.insert.InsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingTaskTxnCommitAttachment; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.load.loadv2.BrokerLoadJob; import org.apache.doris.load.loadv2.BulkLoadJob; @@ -557,7 +558,9 @@ public class GsonUtils { .registerDefaultSubtype(TxnCommitAttachment.class) .registerSubtype(LoadJobFinalOperation.class, LoadJobFinalOperation.class.getSimpleName()) .registerSubtype(MiniLoadTxnCommitAttachment.class, MiniLoadTxnCommitAttachment.class.getSimpleName()) - .registerSubtype(RLTaskTxnCommitAttachment.class, RLTaskTxnCommitAttachment.class.getSimpleName()); + .registerSubtype(RLTaskTxnCommitAttachment.class, RLTaskTxnCommitAttachment.class.getSimpleName()) + .registerSubtype(StreamingTaskTxnCommitAttachment.class, + StreamingTaskTxnCommitAttachment.class.getSimpleName()); // runtime adapter for class "RoutineLoadProgress". private static RuntimeTypeAdapterFactory routineLoadTypeAdapterFactory diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java index b2de57f911001b..bef0bb92d20593 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropFunctionTest.java @@ -23,7 +23,6 @@ import org.apache.doris.nereids.trees.plans.commands.CreateDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.CreateFunctionCommand; import org.apache.doris.nereids.trees.plans.commands.DropFunctionCommand; -import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; @@ -36,9 +35,7 @@ import org.junit.Test; import java.io.File; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.UUID; @@ -66,15 +63,9 @@ public static void teardown() { public void testDropGlobalFunction() throws Exception { ConnectContext ctx = UtFrameUtils.createDefaultCtx(); // 1. create database db1 - //String sql = "create database db1;"; - String sql = "insert into db1.tb select * from s3('url'='s3://a/*.csv')"; + String sql = "create database db1;"; NereidsParser nereidsParser = new NereidsParser(); LogicalPlan logicalPlan = nereidsParser.parseSingle(sql); - InsertIntoTableCommand baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(sql); - baseCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); - Map map = new HashMap<>(); - map.put("url", "s3:/xxxx/*."); - StmtExecutor stmtExecutor = new StmtExecutor(connectContext, sql); if (logicalPlan instanceof CreateDatabaseCommand) { ((CreateDatabaseCommand) logicalPlan).run(connectContext, stmtExecutor); diff --git a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy index bd8ee2759fd138..3982876b9f4303 100644 --- a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy @@ -42,9 +42,13 @@ suite("test_streaming_insert_job") { """ - // create recurring job + // create streaming job sql """ - CREATE JOB ${jobName} ON STREAMING DO INSERT INTO ${tableName} + CREATE JOB ${jobName} + PROPERTIES( + "s3.batch_files" = "1" + ) + ON STREAMING DO INSERT INTO ${tableName} SELECT * FROM S3 ( "uri" = "s3://${s3BucketName}/regression/load/data/example_[0-1].csv", @@ -57,12 +61,13 @@ suite("test_streaming_insert_job") { "s3.secret_key" = "${getS3SK()}" ); """ - Awaitility.await().atMost(30, SECONDS).until( + Awaitility.await().atMost(30, SECONDS) + .pollInterval(1, SECONDS).until( { print("check success task count") def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='STREAMING' """ - // check job status and succeed task count larger than 1 - jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) } ) @@ -76,6 +81,40 @@ suite("test_streaming_insert_job") { qt_select """ SELECT * FROM ${tableName} order by c1 """ + def jobOffset = sql """ + select progress, remoteoffset from jobs("type"="insert") where Name='${jobName}' + """ + assert jobOffset.get(0).get(0) == "regression/load/data/example_1.csv" + assert jobOffset.get(0).get(1) == "regression/load/data/example_1.csv" + //todo check status + + // alter streaming job + sql """ + ALTER JOB FOR ${jobName} + PROPERTIES( + "s3.batch_files" = "1", + "session.insert_max_filter_ratio" = "0.5" + ) + INSERT INTO ${tableName} + SELECT * FROM S3 + ( + "uri" = "s3://${s3BucketName}/regression/load/data/example_[0-1].csv", + "format" = "csv", + "provider" = "${getS3Provider()}", + "column_separator" = ",", + "s3.endpoint" = "${getS3Endpoint()}", + "s3.region" = "${getS3Region()}", + "s3.access_key" = "${getS3AK()}", + "s3.secret_key" = "${getS3SK()}" + ); + """ + + def alterJobProperties = sql """ + select properties from jobs("type"="insert") where Name='${jobName}' + """ + assert alterJobProperties.get(0).get(0) == "{\"s3.batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" + + sql """ DROP JOB IF EXISTS where jobname = '${jobName}' """ @@ -83,4 +122,6 @@ suite("test_streaming_insert_job") { def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" assert jobCountRsp.get(0).get(0) == 0 + + } From 777ba161038433f520057142d957181666b9a142 Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 18 Sep 2025 22:31:37 +0800 Subject: [PATCH 18/25] [Featuren](WIP) add alter job op and fix bug (#56194) ### What problem does this PR solve? add alter job op and fix bug --- .../org/apache/doris/fs/obj/S3ObjStorage.java | 8 +- .../apache/doris/job/base/AbstractJob.java | 5 ++ .../job/extensions/insert/InsertJob.java | 6 +- .../job/extensions/insert/InsertTask.java | 7 +- .../insert/streaming/StreamingInsertJob.java | 62 ++++++++----- .../insert/streaming/StreamingInsertTask.java | 2 +- .../streaming/StreamingJobSchedulerTask.java | 23 +++-- .../apache/doris/job/manager/JobManager.java | 13 +++ .../apache/doris/job/offset/s3/S3Offset.java | 4 +- .../job/offset/s3/S3SourceOffsetProvider.java | 10 ++- .../apache/doris/job/task/AbstractTask.java | 2 +- .../apache/doris/journal/JournalEntity.java | 6 ++ .../trees/plans/commands/AlterJobCommand.java | 37 ++++---- .../plans/commands/CreateJobCommand.java | 2 +- .../plans/commands/ResumeJobCommand.java | 9 +- .../AlterStreamingJobOperationLog.java | 86 +++++++++++++++++++ .../org/apache/doris/persist/EditLog.java | 9 ++ .../apache/doris/persist/OperationType.java | 2 + .../test_streaming_insert_job.groovy | 66 ++++++++++---- 19 files changed, 276 insertions(+), 83 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java index 9b0e5aeb80194d..6f710b74c1b84b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java @@ -728,9 +728,11 @@ public String globListWithLimit(String remotePath, List result, Stri break; } } - //record current last object file name - S3Object lastS3Object = response.contents().get(response.contents().size() - 1); - currentMaxFile = lastS3Object.key(); + if (!response.contents().isEmpty()) { + //record current last object file name + S3Object lastS3Object = response.contents().get(response.contents().size() - 1); + currentMaxFile = lastS3Object.key(); + } isTruncated = response.isTruncated(); if (isTruncated) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index d08942460f541f..24be6358512532 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -31,6 +31,7 @@ import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.thrift.TCell; @@ -475,6 +476,10 @@ public void onReplayEnd(AbstractJob replayJob) throws JobException { log.info(new LogBuilder(LogKey.SCHEDULER_JOB, getJobId()).add("msg", "replay delete scheduler job").build()); } + public void onReplayUpdateStreaming(AlterStreamingJobOperationLog operationLog) { + log.info(new LogBuilder(LogKey.SCHEDULER_JOB, getJobId()).add("msg", "replay update streaming job").build()); + } + public boolean needPersist() { return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index 7658c3cb54d7e2..cc16cbb603d351 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -100,8 +100,8 @@ public class InsertJob extends AbstractJob> impl .add(new Column("Comment", ScalarType.createStringType())) // only execute type = streaming need record .add(new Column("Properties", ScalarType.createStringType())) - .add(new Column("Progress", ScalarType.createStringType())) - .add(new Column("RemoteOffset", ScalarType.createStringType())) + .add(new Column("ConsumedOffset", ScalarType.createStringType())) + .add(new Column("MaxOffset", ScalarType.createStringType())) .add(new Column("LoadStatistic", ScalarType.createStringType())) .add(new Column("ErrorMsg", ScalarType.createStringType())) .build(); @@ -120,9 +120,9 @@ public class InsertJob extends AbstractJob> impl .addColumn(new Column("TrackingUrl", ScalarType.createVarchar(200))) .addColumn(new Column("LoadStatistic", ScalarType.createVarchar(200))) .addColumn(new Column("User", ScalarType.createVarchar(50))) + .addColumn(new Column("FirstErrorMsg", ScalarType.createVarchar(200))) // only execute type = streaming need record .addColumn(new Column("Offset", ScalarType.createStringType())) - .addColumn(new Column("FirstErrorMsg", ScalarType.createVarchar(200))) .build(); public static final ImmutableMap COLUMN_TO_INDEX; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index 8ff292ea8c0c26..5875699fb0f50f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -68,9 +68,8 @@ public class InsertTask extends AbstractTask { new Column("TrackingUrl", ScalarType.createStringType()), new Column("LoadStatistic", ScalarType.createStringType()), new Column("User", ScalarType.createStringType()), - new Column("Offset", ScalarType.createStringType()), - new Column("OtherMsg", ScalarType.createStringType()), - new Column("FirstErrorMsg", ScalarType.createStringType())); + new Column("FirstErrorMsg", ScalarType.createStringType()), + new Column("Offset", ScalarType.createStringType())); public static final ImmutableMap COLUMN_TO_INDEX; @@ -277,8 +276,8 @@ public TRow getTvfInfo(String jobName) { } else { trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); } - trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(firstErrorMsg == null ? "" : firstErrorMsg)); + trow.addToColumnValue(new TCell().setStringVal("")); return trow; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 157629520a9cb4..70463894a660b0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -39,14 +39,15 @@ import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.extensions.insert.InsertTask; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.SourceOffsetProviderFactory; -import org.apache.doris.job.task.AbstractTask; -import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; @@ -87,10 +88,10 @@ public class StreamingInsertJob extends AbstractJob properties; + private Map properties; private StreamingJobProperties jobProperties; @Getter @SerializedName("tvf") @@ -114,7 +115,7 @@ public StreamingInsertJob(String jobName, Long createTimeMs, String executeSql, Map properties) { - super(getNextJobId(), jobName, jobStatus, dbName, comment, createUser, + super(Env.getCurrentEnv().getNextId(), jobName, jobStatus, dbName, comment, createUser, jobConfig, createTimeMs, executeSql); this.dbId = ConnectContext.get().getCurrentDbId(); this.properties = properties; @@ -148,6 +149,9 @@ private void init() { private UnboundTVFRelation getCurrentTvf() { if (baseCommand == null) { + ConnectContext ctx = InsertTask.makeConnectContext(getCreateUser(), getCurrentDbName()); + StatementContext statementContext = new StatementContext(); + ctx.setStatementContext(statementContext); this.baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(getExecuteSql()); } List allTVFRelation = baseCommand.getAllTVFRelation(); @@ -161,6 +165,9 @@ public void updateJobStatus(JobStatus status) throws JobException { lock.writeLock().lock(); try { super.updateJobStatus(status); + if (JobStatus.PAUSED.equals(getJobStatus())) { + clearRunningStreamTask(); + } if (isFinalStatus()) { Env.getCurrentGlobalTransactionMgr().getCallbackFactory().removeCallback(getJobId()); } @@ -193,7 +200,7 @@ public List createTasks(TaskType taskType, Map jobProperties.getMaxIntervalSecond() * 1000; @@ -224,6 +238,13 @@ public boolean hasMoreDataToConsume() { return offsetProvider.hasMoreDataToConsume(); } + @Override + public void logUpdateOperation() { + AlterStreamingJobOperationLog log = + new AlterStreamingJobOperationLog(this.getJobId(), this.getJobStatus(), properties, getExecuteSql()); + Env.getCurrentEnv().getEditLog().logUpdateStreamingJob(log); + } + @Override public void onTaskFail(StreamingJobSchedulerTask task) throws JobException { if (task.getErrMsg() != null) { @@ -287,6 +308,15 @@ public void onReplayCreate() throws JobException { super.onReplayCreate(); } + @Override + public void onReplayUpdateStreaming(AlterStreamingJobOperationLog operationLog) { + super.onReplayUpdateStreaming(operationLog); + setJobStatus(operationLog.getStatus()); + this.properties = operationLog.getJobProperties(); + this.jobProperties = new StreamingJobProperties(properties); + setExecuteSql(operationLog.getExecuteSql()); + } + @Override public ShowResultSetMetaData getTaskMetaData() { return InsertJob.TASK_META_DATA; @@ -361,25 +391,15 @@ public long getId() { @Override public void beforeCommitted(TransactionState txnState) throws TransactionException { - boolean shouldRealseLock = false; + boolean shouldReleaseLock = false; lock.writeLock().lock(); try { ArrayList taskIds = new ArrayList<>(); taskIds.add(runningStreamTask.getTaskId()); - List loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIds); - if (loadJobs.size() != 1) { - shouldRealseLock = true; - throw new TransactionException("load job not found, insert job id is " + runningStreamTask.getTaskId()); - } - LoadJob loadJob = loadJobs.get(0); - - if (txnState.getTransactionId() != loadJob.getTransactionId() - || !runningStreamTask.getStatus().equals(TaskStatus.RUNNING)) { - shouldRealseLock = true; - throw new TransactionException("txn " + txnState.getTransactionId() + "should be aborted."); - } + // todo: Check whether the taskid of runningtask is consistent with the taskid associated with txn - LoadStatistic loadStatistic = loadJob.getLoadStatistic(); + // todo: need get loadStatistic, load manager statistic is empty + LoadStatistic loadStatistic = new LoadStatistic(); txnState.setTxnCommitAttachment(new StreamingTaskTxnCommitAttachment( getJobId(), runningStreamTask.getTaskId(), @@ -389,7 +409,7 @@ public void beforeCommitted(TransactionState txnState) throws TransactionExcepti loadStatistic.getTotalFileSizeB(), runningStreamTask.getRunningOffset().toJson())); } finally { - if (shouldRealseLock) { + if (shouldReleaseLock) { lock.writeLock().unlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index 3073142c151b09..d5eb39cd064a3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -118,7 +118,7 @@ private void before() throws Exception { this.startTimeMs = System.currentTimeMillis(); if (isCanceled.get()) { - throw new JobException("Export executor has been canceled, task id: {}", getTaskId()); + throw new JobException("Streaming insert task has been canceled, task id: {}", getTaskId()); } ctx = InsertTask.makeConnectContext(userIdentity, currentDb); ctx.setSessionVariable(jobProperties.getSessionVariable()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index b658c97c828f31..ec4eb0036d0fdf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -29,6 +29,8 @@ import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; +import org.apache.commons.lang3.StringUtils; + import java.util.Arrays; import java.util.List; @@ -47,7 +49,6 @@ public void run() throws JobException { case PENDING: streamingInsertJob.createStreamingInsertTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); - streamingInsertJob.setAutoResumeCount(0); break; case RUNNING: streamingInsertJob.fetchMeta(); @@ -78,6 +79,7 @@ private void autoResumeHandler() throws JobException { if (autoResumeCount < Long.MAX_VALUE) { streamingInsertJob.setAutoResumeCount(autoResumeCount + 1); } + streamingInsertJob.createStreamingInsertTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); return; } @@ -86,9 +88,7 @@ private void autoResumeHandler() throws JobException { @Override protected void closeOrReleaseResources() { - if (streamingInsertJob.getRunningStreamTask() != null) { - streamingInsertJob.getRunningStreamTask().closeOrReleaseResources(); - } + // do nothing } @Override @@ -110,7 +110,17 @@ public TRow getTvfInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(jobName)); trow.addToColumnValue(new TCell().setStringVal(runningTask.getLabelName())); trow.addToColumnValue(new TCell().setStringVal(runningTask.getStatus().name())); - trow.addToColumnValue(new TCell().setStringVal(runningTask.getErrMsg())); + // err msg + String errMsg = ""; + if (StringUtils.isNotBlank(runningTask.getErrMsg()) + && !FeConstants.null_string.equals(runningTask.getErrMsg())) { + errMsg = runningTask.getErrMsg(); + } else { + errMsg = runningTask.getOtherMsg(); + } + trow.addToColumnValue(new TCell().setStringVal(StringUtils.isNotBlank(errMsg) + ? errMsg : FeConstants.null_string)); + // create time trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(runningTask.getCreateTimeMs()))); trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? FeConstants.null_string @@ -143,10 +153,9 @@ public TRow getTvfInfo(String jobName) { } else { trow.addToColumnValue(new TCell().setStringVal(runningTask.getUserIdentity().getQualifiedUser())); } + trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(runningTask.getRunningOffset() == null ? FeConstants.null_string : runningTask.getRunningOffset().toJson())); - trow.addToColumnValue(new TCell().setStringVal(null == runningTask.getOtherMsg() - ? FeConstants.null_string : runningTask.getOtherMsg())); return trow; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 2b8c7225b2add9..34fc29f1c8d8aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -45,6 +45,7 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; @@ -363,6 +364,18 @@ public void replayUpdateJob(T job) { .add("msg", "replay update scheduler job").build()); } + public void replayUpdateStreamingJob(AlterStreamingJobOperationLog log) { + Long jobId = log.getJobId(); + if (!jobMap.containsKey(jobId)) { + LOG.warn("replayUpdateStreamingJob not normal, jobId: {}, jobMap: {}", jobId, log); + return; + } + T job = jobMap.get(jobId); + job.onReplayUpdateStreaming(log); + LOG.info(new LogBuilder(LogKey.SCHEDULER_JOB, jobId) + .add("msg", "replay update streaming job").build()); + } + /** * Replay delete load job. we need to remove job from job map */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index 3d260218886223..3f7956f9fe976d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -27,9 +27,11 @@ @Getter @Setter public class S3Offset implements Offset { + // path/1.csv String startFile; - @SerializedName("ef") + @SerializedName("endFile") String endFile; + // s3://bucket/path/{1.csv,2.csv} String fileLists; @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index e429ae7375b151..d433e42f73ca01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -33,6 +33,7 @@ import com.google.common.collect.Maps; import lombok.extern.log4j.Log4j2; +import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; import java.util.HashMap; @@ -136,8 +137,13 @@ public void fetchRemoteMeta(Map properties) throws Exception { try (RemoteFileSystem fileSystem = FileSystemFactory.get(storageProperties)) { String uri = storageProperties.validateAndGetUri(copiedProps); String filePath = storageProperties.validateAndNormalizeUri(uri); - maxRemoteEndFile = fileSystem.globListWithLimit(filePath, new ArrayList<>(), startFile, - 1, 1); + List objects = new ArrayList<>(); + String endFile = fileSystem.globListWithLimit(filePath, objects, startFile, 1, 1); + if (!objects.isEmpty() && StringUtils.isNotEmpty(endFile)) { + maxRemoteEndFile = endFile; + } else { + maxRemoteEndFile = startFile; + } } catch (Exception e) { throw e; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java index 18c5f525295d0d..4e2ac653cf700f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java @@ -60,7 +60,7 @@ public AbstractTask() { taskId = getNextTaskId(); } - public static long getNextTaskId() { + private static long getNextTaskId() { // do not use Env.getNextId(), just generate id without logging return System.nanoTime() + RandomUtils.nextInt(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index cb6dee000ded01..f0f9d471ed9227 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -67,6 +67,7 @@ import org.apache.doris.persist.AlterLightSchemaChangeInfo; import org.apache.doris.persist.AlterMTMV; import org.apache.doris.persist.AlterRoutineLoadJobOperationLog; +import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.persist.AlterUserOperationLog; import org.apache.doris.persist.AlterViewInfo; import org.apache.doris.persist.AnalyzeDeletionLog; @@ -550,6 +551,11 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } + case OperationType.OP_UPDATE_STREAMING_JOB: { + data = AlterStreamingJobOperationLog.read(in); + isRead = true; + break; + } case OperationType.OP_CREATE_SCHEDULER_TASK: case OperationType.OP_DELETE_SCHEDULER_TASK: { //todo improve diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 786c8184e407ad..adde81fbfe90ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -33,6 +33,7 @@ import org.apache.commons.lang3.StringUtils; import java.util.Map; +import java.util.Objects; /** * alter job command. @@ -75,22 +76,16 @@ public R accept(PlanVisitor visitor, C context) { private AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws JobException { AbstractJob job = Env.getCurrentEnv().getJobManager().getJobByName(jobName); if (job instanceof StreamingInsertJob) { - StreamingInsertJob originJob = (StreamingInsertJob) job; - String updateSQL = StringUtils.isEmpty(sql) ? originJob.getExecuteSql() : sql; - Map updateProps = - properties == null || properties.isEmpty() ? originJob.getProperties() : properties; - - StreamingInsertJob streamingInsertJob = new StreamingInsertJob(jobName, - job.getJobStatus(), - job.getCurrentDbName(), - job.getComment(), - ConnectContext.get().getCurrentUserIdentity(), - originJob.getJobConfig(), - System.currentTimeMillis(), - updateSQL, - updateProps); - streamingInsertJob.setJobId(job.getJobId()); - return streamingInsertJob; + StreamingInsertJob updateJob = (StreamingInsertJob) job; + // update sql + if (StringUtils.isNotEmpty(sql)) { + updateJob.setExecuteSql(sql); + } + // update properties + if (!properties.isEmpty()) { + updateJob.getProperties().putAll(properties); + } + return updateJob; } else { throw new JobException("Unsupported job type for ALTER:" + job.getJobType()); } @@ -119,20 +114,20 @@ private void validate() throws Exception { } private boolean checkProperties(Map originProps) { - if (originProps.isEmpty()) { + if (this.properties == null || this.properties.isEmpty()) { return false; } - if (!originProps.equals(properties)) { + if (!Objects.equals(this.properties, originProps)) { return true; } return false; } - private boolean checkSql(String sql) { - if (sql == null || sql.isEmpty()) { + private boolean checkSql(String originSql) { + if (originSql == null || originSql.isEmpty()) { return false; } - if (!sql.equals(sql)) { + if (!originSql.equals(this.sql)) { return true; } return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java index fe81921211fe9b..baac59fd7b9305 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java @@ -67,7 +67,7 @@ private void validate() throws JobException { if (createJobInfo.streamingJob()) { int streamingJobCnt = Env.getCurrentEnv().getJobManager().getStreamingJobCnt(); if (streamingJobCnt >= Config.max_streaming_job_num) { - throw new JobException("Exceed max streaming job num limit " + Config.max_streaming_job_num); + throw new JobException("Exceed max streaming job num limit in fe.conf:" + Config.max_streaming_job_num); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.java index 3935f73f0e90cf..1ad21237a39884 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ResumeJobCommand.java @@ -21,7 +21,9 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -46,7 +48,12 @@ public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); } - ctx.getEnv().getJobManager().alterJobStatus(super.getJobName(), JobStatus.RUNNING); + AbstractJob job = ctx.getEnv().getJobManager().getJobByName(super.getJobName()); + if (job instanceof StreamingInsertJob) { + ctx.getEnv().getJobManager().alterJobStatus(super.getJobName(), JobStatus.PENDING); + } else { + ctx.getEnv().getJobManager().alterJobStatus(super.getJobName(), JobStatus.RUNNING); + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java new file mode 100644 index 00000000000000..785531fa0d14a9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java @@ -0,0 +1,86 @@ +// 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.job.common.JobStatus; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Map; + +public class AlterStreamingJobOperationLog implements Writable { + @SerializedName(value = "jid") + private long jobId; + @SerializedName(value = "js") + private JobStatus status; + @SerializedName(value = "jp") + private Map jobProperties; + @SerializedName(value = "sql") + String executeSql; + + public AlterStreamingJobOperationLog(long jobId, JobStatus status, + Map jobProperties, String executeSql) { + this.jobId = jobId; + this.status = status; + this.jobProperties = jobProperties; + this.executeSql = executeSql; + } + + public long getJobId() { + return jobId; + } + + public Map getJobProperties() { + return jobProperties; + } + + public String getExecuteSql() { + return executeSql; + } + + public JobStatus getStatus() { + return status; + } + + public static AlterStreamingJobOperationLog read(DataInput in) throws IOException { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, AlterStreamingJobOperationLog.class); + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } + + @Override + public String toString() { + return "AlterStreamingJobOperationLog{" + + "jobId=" + jobId + + ", status=" + status + + ", jobProperties=" + jobProperties + + ", executeSql='" + executeSql + '\'' + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 5abcc4e04646b2..74be0f5e4753f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -862,6 +862,11 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { Env.getCurrentEnv().getJobManager().replayDeleteJob(job); break; } + case OperationType.OP_UPDATE_STREAMING_JOB: { + AlterStreamingJobOperationLog log = (AlterStreamingJobOperationLog) journal.getData(); + Env.getCurrentEnv().getJobManager().replayUpdateStreamingJob(log); + break; + } /*case OperationType.OP_CREATE_SCHEDULER_TASK: { JobTask task = (JobTask) journal.getData(); Env.getCurrentEnv().getJobTaskManager().replayCreateTask(task); @@ -2038,6 +2043,10 @@ public void logUpdateJob(AbstractJob job) { logEdit(OperationType.OP_UPDATE_SCHEDULER_JOB, job); } + public void logUpdateStreamingJob(AlterStreamingJobOperationLog log) { + logEdit(OperationType.OP_UPDATE_STREAMING_JOB, log); + } + public void logDeleteJob(AbstractJob job) { logEdit(OperationType.OP_DELETE_SCHEDULER_JOB, job); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 2bbc03d4aeab87..12d59d086b9b74 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -414,6 +414,8 @@ public class OperationType { public static final short OP_OPERATE_KEY = 492; + public static final short OP_UPDATE_STREAMING_JOB = 493; + // For cloud. public static final short OP_UPDATE_CLOUD_REPLICA = 1000; @Deprecated diff --git a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy index 3982876b9f4303..d89bc8d2f0f221 100644 --- a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy @@ -61,15 +61,28 @@ suite("test_streaming_insert_job") { "s3.secret_key" = "${getS3SK()}" ); """ - Awaitility.await().atMost(30, SECONDS) - .pollInterval(1, SECONDS).until( - { - print("check success task count") - def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='STREAMING' """ - // check job status and succeed task count larger than 2 - jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) - } - ) + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + print("check success task count") + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name like '%${jobName}%' and ExecuteType='STREAMING' """ + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + println("show job: " + showjob) + println("show task: " + showtask) + throw ex; + } + + def jobResult = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + println("show success job: " + jobResult) + + qt_select """ SELECT * FROM ${tableName} order by c1 """ sql """ PAUSE JOB where jobname = '${jobName}' @@ -79,20 +92,20 @@ suite("test_streaming_insert_job") { """ assert pausedJobStatus.get(0).get(0) == "PAUSED" - qt_select """ SELECT * FROM ${tableName} order by c1 """ + def pauseShowTask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + assert pauseShowTask.size() == 0 + def jobOffset = sql """ - select progress, remoteoffset from jobs("type"="insert") where Name='${jobName}' + select ConsumedOffset, MaxOffset from jobs("type"="insert") where Name='${jobName}' """ assert jobOffset.get(0).get(0) == "regression/load/data/example_1.csv" assert jobOffset.get(0).get(1) == "regression/load/data/example_1.csv" - //todo check status // alter streaming job sql """ ALTER JOB FOR ${jobName} PROPERTIES( - "s3.batch_files" = "1", "session.insert_max_filter_ratio" = "0.5" ) INSERT INTO ${tableName} @@ -110,10 +123,31 @@ suite("test_streaming_insert_job") { """ def alterJobProperties = sql """ - select properties from jobs("type"="insert") where Name='${jobName}' + select status,properties,ConsumedOffset from jobs("type"="insert") where Name='${jobName}' + """ + assert alterJobProperties.get(0).get(0) == "PAUSED" + assert alterJobProperties.get(0).get(1) == "{\"s3.batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" + assert alterJobProperties.get(0).get(2) == "regression/load/data/example_1.csv" + + sql """ + RESUME JOB where jobname = '${jobName}' + """ + def resumeJobStatus = sql """ + select status,properties,ConsumedOffset from jobs("type"="insert") where Name='${jobName}' """ - assert alterJobProperties.get(0).get(0) == "{\"s3.batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" + assert resumeJobStatus.get(0).get(0) == "RUNNING" || resumeJobStatus.get(0).get(0) == "PENDING" + assert resumeJobStatus.get(0).get(1) == "{\"s3.batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" + assert resumeJobStatus.get(0).get(2) == "regression/load/data/example_1.csv" + Awaitility.await().atMost(60, SECONDS) + .pollInterval(1, SECONDS).until( + { + print("check create streaming task count") + def resumeShowTask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + // check streaming task create success + resumeShowTask.size() == 1 + } + ) sql """ DROP JOB IF EXISTS where jobname = '${jobName}' @@ -122,6 +156,4 @@ suite("test_streaming_insert_job") { def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" assert jobCountRsp.get(0).get(0) == 0 - - } From 4db2ec2cc237a555c242676f22e5680ec680baa2 Mon Sep 17 00:00:00 2001 From: hui lai Date: Fri, 19 Sep 2025 12:55:00 +0800 Subject: [PATCH 19/25] [fix](streaming job) fix some streaming job bug (#56221) ### What problem does this PR solve? Fix some streaming job bug. --- cloud/src/meta-service/meta_service_txn.cpp | 22 ++++++++++--- .../insert/streaming/StreamingInsertJob.java | 32 +++++++++++++------ .../streaming/StreamingJobSchedulerTask.java | 4 +-- .../job/scheduler/StreamingTaskScheduler.java | 23 +++++++------ gensrc/proto/cloud.proto | 4 +-- 5 files changed, 54 insertions(+), 31 deletions(-) diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index fd09f80d74c9de..22b2a449231f3c 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -610,12 +610,14 @@ void put_routine_load_progress(MetaServiceCode& code, std::string& msg, << " routine load new progress: " << new_progress_info.ShortDebugString(); } -void put_streaming_job_meta(MetaServiceCode& code, std::string& msg, const std::string& instance_id, - const CommitTxnRequest* request, Transaction* txn, int64_t db_id) { +void update_streaming_job_meta(MetaServiceCode& code, std::string& msg, + const std::string& instance_id, const CommitTxnRequest* request, + Transaction* txn, int64_t db_id) { std::stringstream ss; int64_t txn_id = request->txn_id(); if (!request->has_commit_attachment()) { - ss << "failed to get commit attachment from req, db_id=" << db_id << " txn_id=" << txn_id; + code = MetaServiceCode::INVALID_ARGUMENT; + ss << "missing commit attachment, db_id=" << db_id << " txn_id=" << txn_id; msg = ss.str(); return; } @@ -1706,7 +1708,12 @@ void MetaServiceImpl::commit_txn_immediately( if (txn_info.load_job_source_type() == LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { - put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + update_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "update_streaming_job_meta failed, txn_id=" << txn_id + << " code=" << code << " msg=" << msg; + return; + } } LOG(INFO) << "xxx commit_txn put recycle_key key=" << hex(recycle_key) @@ -2104,7 +2111,12 @@ void MetaServiceImpl::commit_txn_eventually( if (txn_info.load_job_source_type() == LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB) { - put_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + update_streaming_job_meta(code, msg, instance_id, request, txn.get(), db_id); + if (code != MetaServiceCode::OK) { + LOG(WARNING) << "update_streaming_job_meta failed, txn_id=" << txn_id + << " code=" << code << " msg=" << msg; + return; + } } // save versions for partition diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 70463894a660b0..7f882757356e63 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -147,6 +147,24 @@ private void init() { } } + private void readLock() { + lock.readLock().lock(); + } + + private void readUnlock() { + lock.readLock().unlock(); + } + + private void writeLock() { + lock.writeLock().lock(); + } + + private void writeUnlock() { + if (lock.writeLock().isHeldByCurrentThread()) { + lock.writeLock().unlock(); + } + } + private UnboundTVFRelation getCurrentTvf() { if (baseCommand == null) { ConnectContext ctx = InsertTask.makeConnectContext(getCreateUser(), getCurrentDbName()); @@ -229,11 +247,6 @@ public void clearRunningStreamTask() { } } - // When consumer to EOF, delay schedule task appropriately can avoid too many small transactions. - public boolean needDelayScheduleTask() { - return System.currentTimeMillis() - lastScheduleTaskTimestamp > jobProperties.getMaxIntervalSecond() * 1000; - } - public boolean hasMoreDataToConsume() { return offsetProvider.hasMoreDataToConsume(); } @@ -268,7 +281,7 @@ public void onStreamTaskFail(StreamingInsertTask task) throws JobException { this.failureReason = new FailureReason(task.getErrMsg()); } } finally { - lock.writeLock().unlock(); + writeUnlock(); } updateJobStatus(JobStatus.PAUSED); } @@ -282,7 +295,7 @@ public void onStreamTaskSuccess(StreamingInsertTask task) { //todo: maybe fetch from txn attachment? offsetProvider.updateOffset(task.getRunningOffset()); } finally { - lock.writeLock().unlock(); + writeUnlock(); } } @@ -392,7 +405,7 @@ public long getId() { @Override public void beforeCommitted(TransactionState txnState) throws TransactionException { boolean shouldReleaseLock = false; - lock.writeLock().lock(); + writeLock(); try { ArrayList taskIds = new ArrayList<>(); taskIds.add(runningStreamTask.getTaskId()); @@ -410,14 +423,13 @@ public void beforeCommitted(TransactionState txnState) throws TransactionExcepti runningStreamTask.getRunningOffset().toJson())); } finally { if (shouldReleaseLock) { - lock.writeLock().unlock(); + writeUnlock(); } } } @Override public void beforeAborted(TransactionState txnState) throws TransactionException { - } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index ec4eb0036d0fdf..de46ccedf50d16 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -49,6 +49,7 @@ public void run() throws JobException { case PENDING: streamingInsertJob.createStreamingInsertTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); + streamingInsertJob.setAutoResumeCount(0); break; case RUNNING: streamingInsertJob.fetchMeta(); @@ -79,8 +80,7 @@ private void autoResumeHandler() throws JobException { if (autoResumeCount < Long.MAX_VALUE) { streamingInsertJob.setAutoResumeCount(autoResumeCount + 1); } - streamingInsertJob.createStreamingInsertTask(); - streamingInsertJob.updateJobStatus(JobStatus.RUNNING); + streamingInsertJob.updateJobStatus(JobStatus.PENDING); return; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java index 6e664b404da02b..8cf06bbc1a45ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -21,7 +21,6 @@ import org.apache.doris.common.Config; import org.apache.doris.common.CustomThreadFactory; import org.apache.doris.common.util.MasterDaemon; -import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; @@ -74,38 +73,38 @@ private void process() throws InterruptedException { private void scheduleTasks(List tasks) { for (StreamingInsertTask task : tasks) { threadPool.execute(() -> { - try { - scheduleOneTask(task); - } catch (Exception e) { - log.error("Failed to schedule task, task id: {}, job id: {}", task.getTaskId(), task.getJobId(), e); - } + scheduleOneTask(task); }); } } - private void scheduleOneTask(StreamingInsertTask task) throws JobException { + private void scheduleOneTask(StreamingInsertTask task) { StreamingInsertJob job = (StreamingInsertJob) Env.getCurrentEnv().getJobManager().getJob(task.getJobId()); if (job == null) { log.warn("Job not found, job id: {}", task.getJobId()); return; } + + // reject invalid task if (!job.needScheduleTask()) { log.info("do not need to schedule invalid task, task id: {}, job id: {}", task.getTaskId(), task.getJobId()); return; } + // reject task if no more data to consume if (!job.hasMoreDataToConsume()) { scheduleTaskWithDelay(task, 500); return; } - if (job.getLastScheduleTaskTimestamp() != -1 && job.needDelayScheduleTask()) { - scheduleTaskWithDelay(task, 500); - return; - } log.info("prepare to schedule task, task id: {}, job id: {}", task.getTaskId(), task.getJobId()); job.setLastScheduleTaskTimestamp(System.currentTimeMillis()); Env.getCurrentEnv().getJobManager().getStreamingTaskManager().addRunningTask(task); - task.execute(); + + try { + task.execute(); + } catch (Exception e) { + log.error("Failed to execute task, task id: {}, job id: {}", task.getTaskId(), task.getJobId(), e); + } } private void scheduleTaskWithDelay(StreamingInsertTask task, long delayMs) { diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index c9382ed2980d28..6259d6d0da3208 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -1716,8 +1716,8 @@ enum MetaServiceCode { JOB_ALREADY_SUCCESS = 5002; ROUTINE_LOAD_DATA_INCONSISTENT = 5003; ROUTINE_LOAD_PROGRESS_NOT_FOUND = 5004; - STREAMING_JOB_PROGRESS_NOT_FOUND = 5005; - JOB_CHECK_ALTER_VERSION = 5006; + JOB_CHECK_ALTER_VERSION = 5005; + STREAMING_JOB_PROGRESS_NOT_FOUND = 5006; // Rate limit MAX_QPS_LIMIT = 6001; From 71a8baca13a7d562d8ab7faf93f137302889ec5f Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 19 Sep 2025 17:06:37 +0800 Subject: [PATCH 20/25] [Fix](job) fix job bug (#56225) ### What problem does this PR solve? [Fix](job) fix job bug --- .../property/storage/StorageProperties.java | 63 ++++++- .../org/apache/doris/fs/obj/S3ObjStorage.java | 175 +++++++----------- .../apache/doris/job/base/AbstractJob.java | 4 +- .../apache/doris/job/common/JobStatus.java | 7 +- .../org/apache/doris/job/common/TaskType.java | 3 +- .../job/executor/DispatchTaskHandler.java | 5 +- .../insert/streaming/StreamingInsertJob.java | 12 +- .../insert/streaming/StreamingInsertTask.java | 4 +- .../streaming/StreamingJobSchedulerTask.java | 2 +- .../streaming/StreamingJobStatistic.java | 4 +- .../apache/doris/job/manager/JobManager.java | 15 +- .../org/apache/doris/job/offset/Offset.java | 6 +- .../apache/doris/job/offset/s3/S3Offset.java | 18 +- .../org/apache/doris/qe/SessionVariable.java | 2 +- 14 files changed, 159 insertions(+), 161 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java index 422d20a7560293..337b897f256796 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java @@ -146,19 +146,66 @@ public static List createAll(Map origProps) t * @throws RuntimeException if no supported storage type is found */ public static StorageProperties createPrimary(Map origProps) { - for (Function, StorageProperties> func : PROVIDERS) { - StorageProperties p = func.apply(origProps); - if (p != null) { - p.initNormalizeAndCheckProps(); - p.initializeHadoopStorageConfig(); - return p; + StorageProperties p = createPrimaryInternal(origProps); + if (p == null) { + for (Function, StorageProperties> func : PROVIDERS) { + p = func.apply(origProps); + if (p != null) { + break; + } } } + if (p != null) { + p.initNormalizeAndCheckProps(); + p.initializeHadoopStorageConfig(); + return p; + } throw new StoragePropertiesException("No supported storage type found. Please check your configuration."); } + private static StorageProperties createPrimaryInternal(Map origProps) { + String provider = origProps.get(FS_PROVIDER_KEY); + if (provider == null) { + return null; + } + + try { + Type type = Type.valueOf(provider.trim().toUpperCase()); + switch (type) { + case HDFS: + return new HdfsProperties(origProps); + case OSS_HDFS: + return new OSSHdfsProperties(origProps); + case S3: + return new S3Properties(origProps); + case OSS: + return new OSSProperties(origProps); + case OBS: + return new OBSProperties(origProps); + case COS: + return new COSProperties(origProps); + case GCS: + return new GCSProperties(origProps); + case AZURE: + return new AzureProperties(origProps); + case MINIO: + return new MinioProperties(origProps); + case BROKER: + return new BrokerProperties(origProps); + case LOCAL: + return new LocalProperties(origProps); + default: + return null; + } + } catch (Exception e) { + return null; + } + } + private static final List, StorageProperties>> PROVIDERS = Arrays.asList( + props -> (isFsSupport(props, FS_HDFS_SUPPORT) + || HdfsProperties.guessIsMe(props)) ? new HdfsProperties(props) : null, props -> ((isFsSupport(props, FS_OSS_HDFS_SUPPORT) || isFsSupport(props, DEPRECATED_OSS_HDFS_SUPPORT)) || OSSHdfsProperties.guessIsMe(props)) ? new OSSHdfsProperties(props) : null, @@ -179,9 +226,7 @@ public static StorageProperties createPrimary(Map origProps) { props -> (isFsSupport(props, FS_BROKER_SUPPORT) || BrokerProperties.guessIsMe(props)) ? new BrokerProperties(props) : null, props -> (isFsSupport(props, FS_LOCAL_SUPPORT) - || LocalProperties.guessIsMe(props)) ? new LocalProperties(props) : null, - props -> (isFsSupport(props, FS_HDFS_SUPPORT) - || HdfsProperties.guessIsMe(props)) ? new HdfsProperties(props) : null + || LocalProperties.guessIsMe(props)) ? new LocalProperties(props) : null ); protected StorageProperties(Type type, Map origProps) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java index 6f710b74c1b84b..f7cc644017f154 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java @@ -529,109 +529,10 @@ ListObjectsV2Response listObjectsV2(ListObjectsV2Request request) throws UserExc * Copy from `AzureObjStorage.GlobList` */ public Status globList(String remotePath, List result, boolean fileNameOnly) { - long roundCnt = 0; - long elementCnt = 0; - long matchCnt = 0; - long startTime = System.nanoTime(); - try { - S3URI uri = S3URI.create(remotePath, isUsePathStyle, forceParsingByStandardUri); - String bucket = uri.getBucket(); - String globPath = uri.getKey(); // eg: path/to/*.csv - - if (LOG.isDebugEnabled()) { - LOG.debug("globList globPath:{}, remotePath:{}", globPath, remotePath); - } - java.nio.file.Path pathPattern = Paths.get(globPath); - PathMatcher matcher = FileSystems.getDefault().getPathMatcher("glob:" + pathPattern); - HashSet directorySet = new HashSet<>(); - - String listPrefix = S3Util.getLongestPrefix(globPath); // similar to Azure - if (LOG.isDebugEnabled()) { - LOG.debug("globList listPrefix: '{}' (from globPath: '{}')", listPrefix, globPath); - } - - // For Directory Buckets, ensure proper prefix handling using standardized approach - String finalPrefix = listPrefix; - - if (uri.useS3DirectoryBucket()) { - String adjustedPrefix = S3URI.getDirectoryPrefixForGlob(listPrefix); - if (LOG.isDebugEnabled() && !adjustedPrefix.equals(listPrefix)) { - LOG.debug("Directory bucket detected, adjusting prefix from '{}' to '{}'", - listPrefix, adjustedPrefix); - } - finalPrefix = adjustedPrefix; - } - - ListObjectsV2Request request = ListObjectsV2Request.builder() - .bucket(bucket) - .prefix(finalPrefix) - .build(); - - boolean isTruncated = false; - do { - roundCnt++; - ListObjectsV2Response response = listObjectsV2(request); - for (S3Object obj : response.contents()) { - elementCnt++; - java.nio.file.Path objPath = Paths.get(obj.key()); - - boolean isPrefix = false; - while (objPath != null && objPath.normalize().toString().startsWith(listPrefix)) { - if (!matcher.matches(objPath)) { - isPrefix = true; - objPath = objPath.getParent(); - continue; - } - if (directorySet.contains(objPath.normalize().toString())) { - break; - } - if (isPrefix) { - directorySet.add(objPath.normalize().toString()); - } - - matchCnt++; - RemoteFile remoteFile = new RemoteFile( - fileNameOnly ? objPath.getFileName().toString() : - "s3://" + bucket + "/" + objPath.toString(), - !isPrefix, - isPrefix ? -1 : obj.size(), - isPrefix ? -1 : obj.size(), - isPrefix ? 0 : obj.lastModified().toEpochMilli() - ); - result.add(remoteFile); - objPath = objPath.getParent(); - isPrefix = true; - } - } - - isTruncated = response.isTruncated(); - if (isTruncated) { - request = request.toBuilder() - .continuationToken(response.nextContinuationToken()) - .build(); - } - } while (isTruncated); - - if (LOG.isDebugEnabled()) { - LOG.debug("remotePath:{}, result:{}", remotePath, result); - } - return Status.OK; - } catch (Exception e) { - LOG.warn("Errors while getting file status", e); - return new Status(Status.ErrCode.COMMON_ERROR, - "Errors while getting file status " + Util.getRootCauseMessage(e)); - } finally { - long endTime = System.nanoTime(); - long duration = endTime - startTime; - if (LOG.isDebugEnabled()) { - LOG.debug("process {} elements under prefix {} for {} round, match {} elements, take {} ms", - elementCnt, remotePath, roundCnt, matchCnt, - duration / 1000 / 1000); - } - } + GlobListResult globListResult = globListInternal(remotePath, result, fileNameOnly, null, -1, -1); + return globListResult.getStatus(); } - /** * List all files under the given path with glob pattern. * For example, if the path is "s3://bucket/path/to/*.csv", @@ -644,14 +545,31 @@ public Status globList(String remotePath, List result, boolean fileN */ public String globListWithLimit(String remotePath, List result, String startFile, long fileSizeLimit, long fileNumLimit) { + GlobListResult globListResult = globListInternal(remotePath, result, true, startFile, fileSizeLimit, + fileNumLimit); + return globListResult.getMaxFile(); + } + + /** + * List all files under the given path with glob pattern. + * For example, if the path is "s3://bucket/path/to/*.csv", + * it will list all files under "s3://bucket/path/to/" with ".csv" suffix. + *

+ * Copy from `AzureObjStorage.GlobList` + */ + private GlobListResult globListInternal(String remotePath, List result, boolean fileNameOnly, + String startFile, long fileSizeLimit, long fileNumLimit) { long roundCnt = 0; long elementCnt = 0; long matchCnt = 0; long matchFileSize = 0L; long startTime = System.nanoTime(); + String currentMaxFile = ""; + boolean hasLimits = fileSizeLimit > 0 || fileNumLimit > 0; try { S3URI uri = S3URI.create(remotePath, isUsePathStyle, forceParsingByStandardUri); - if (uri.useS3DirectoryBucket()) { + // Directory bucket check for limit scenario + if (hasLimits && uri.useS3DirectoryBucket()) { throw new RuntimeException("Not support glob with limit for directory bucket"); } @@ -670,9 +588,21 @@ public String globListWithLimit(String remotePath, List result, Stri LOG.debug("globList listPrefix: '{}' (from globPath: '{}')", listPrefix, globPath); } - Builder builder = ListObjectsV2Request.builder(); - builder.bucket(bucket) - .prefix(listPrefix); + // For Directory Buckets, ensure proper prefix handling using standardized approach + String finalPrefix = listPrefix; + + if (!hasLimits && uri.useS3DirectoryBucket()) { + String adjustedPrefix = S3URI.getDirectoryPrefixForGlob(listPrefix); + if (LOG.isDebugEnabled() && !adjustedPrefix.equals(listPrefix)) { + LOG.debug("Directory bucket detected, adjusting prefix from '{}' to '{}'", + listPrefix, adjustedPrefix); + } + finalPrefix = adjustedPrefix; + } + + Builder builder = ListObjectsV2Request.builder() + .bucket(bucket) + .prefix(finalPrefix); if (startFile != null) { builder.startAfter(startFile); @@ -680,7 +610,6 @@ public String globListWithLimit(String remotePath, List result, Stri ListObjectsV2Request request = builder.build(); - String currentMaxFile = ""; boolean isTruncated = false; boolean reachLimit = false; do { @@ -705,7 +634,10 @@ public String globListWithLimit(String remotePath, List result, Stri } matchCnt++; - RemoteFile remoteFile = new RemoteFile(objPath.getFileName().toString(), + matchFileSize += obj.size(); + RemoteFile remoteFile = new RemoteFile( + fileNameOnly ? objPath.getFileName().toString() : + "s3://" + bucket + "/" + objPath.toString(), !isPrefix, isPrefix ? -1 : obj.size(), isPrefix ? -1 : obj.size(), @@ -713,10 +645,9 @@ public String globListWithLimit(String remotePath, List result, Stri ); remoteFile.setBucket(bucket); remoteFile.setParentPath(objPath.getParent().toString()); - matchFileSize += obj.size(); result.add(remoteFile); - if (reachLimit(result.size(), matchFileSize, fileSizeLimit, fileNumLimit)) { + if (hasLimits && reachLimit(result.size(), matchFileSize, fileSizeLimit, fileNumLimit)) { reachLimit = true; break; } @@ -728,8 +659,9 @@ public String globListWithLimit(String remotePath, List result, Stri break; } } + + // Record current max file for limit scenario if (!response.contents().isEmpty()) { - //record current last object file name S3Object lastS3Object = response.contents().get(response.contents().size() - 1); currentMaxFile = lastS3Object.key(); } @@ -745,10 +677,11 @@ public String globListWithLimit(String remotePath, List result, Stri if (LOG.isDebugEnabled()) { LOG.debug("remotePath:{}, result:{}", remotePath, result); } - return currentMaxFile; + return new GlobListResult(Status.OK, currentMaxFile); } catch (Exception e) { LOG.warn("Errors while getting file status", e); - throw new RuntimeException(e); + return new GlobListResult(new Status(Status.ErrCode.COMMON_ERROR, + "Errors while getting file status " + Util.getRootCauseMessage(e)), ""); } finally { long endTime = System.nanoTime(); long duration = endTime - startTime; @@ -782,6 +715,24 @@ private static boolean reachLimit(int matchFileCnt, long matchFileSize, long siz return false; } + private static class GlobListResult { + private final Status status; + private final String maxFile; + + public GlobListResult(Status status, String maxFile) { + this.status = status; + this.maxFile = maxFile; + } + + public Status getStatus() { + return status; + } + + public String getMaxFile() { + return maxFile; + } + } + @Override public synchronized void close() throws Exception { if (client != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 24be6358512532..a58dddb55e7a4b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -247,9 +247,11 @@ private boolean canCreateTask(TaskType taskType) { switch (taskType) { case SCHEDULED: - return currentJobStatus.equals(JobStatus.RUNNING) || currentJobStatus.equals(JobStatus.PENDING); + return currentJobStatus.equals(JobStatus.RUNNING); case MANUAL: return currentJobStatus.equals(JobStatus.RUNNING) || currentJobStatus.equals(JobStatus.PAUSED); + case STREAMING: + return !jobStatus.equals(JobStatus.STOPPED) && !jobStatus.equals(JobStatus.FINISHED); default: throw new IllegalArgumentException("Unsupported TaskType: " + taskType); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java index 25d207110d9cb3..c1f47f8ff59a5f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java @@ -42,5 +42,10 @@ public enum JobStatus { /** * When the task is finished, the finished state will be triggered. */ - FINISHED + FINISHED; + + + public static boolean isRunning(JobStatus status) { + return PENDING.equals(status) || RUNNING.equals(status); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskType.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskType.java index 7f782cb4121f9e..138325a21bfc01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskType.java @@ -20,5 +20,6 @@ public enum TaskType { SCHEDULED, - MANUAL; + MANUAL, + STREAMING; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java index 35b1f351f723d5..23710c17659c4b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java @@ -18,6 +18,7 @@ package org.apache.doris.job.executor; import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.common.TaskType; @@ -56,7 +57,9 @@ public void onEvent(TimerJobEvent event) { return; } if (event.getJob().isReadyForScheduling(null) && JobUtils.checkNeedSchedule(event.getJob())) { - List tasks = event.getJob().commonCreateTasks(TaskType.SCHEDULED, null); + TaskType taskType = JobExecuteType.STREAMING.equals(event.getJob().getJobConfig().getExecuteType()) + ? TaskType.STREAMING : TaskType.SCHEDULED; + List tasks = event.getJob().commonCreateTasks(taskType, null); if (CollectionUtils.isEmpty(tasks)) { log.warn("job is ready for scheduling, but create task is empty, skip scheduler," + "job id is {}," + " job name is {}", event.getJob().getJobId(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 7f882757356e63..479698b6f44f19 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -28,7 +28,6 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.job.base.AbstractJob; -import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; import org.apache.doris.job.common.FailureReason; @@ -88,7 +87,7 @@ public class StreamingInsertJob extends AbstractJob properties; @@ -218,6 +217,9 @@ public List createTasks(TaskType taskType, Map Date: Mon, 22 Sep 2025 14:49:48 +0800 Subject: [PATCH 21/25] [fix](streaming job) refactor some logic and add ut (#56292) ### What problem does this PR solve? Refactor some logic and add ut. --- cloud/src/meta-service/meta_service_txn.cpp | 79 ++++---- cloud/src/meta-store/keys.cpp | 16 +- cloud/src/meta-store/keys.h | 8 +- cloud/test/keys_test.cpp | 52 ++++++ cloud/test/meta_service_job_test.cpp | 175 ++++++++++++++++++ .../doris/cloud/transaction/TxnUtil.java | 5 +- .../insert/streaming/StreamingInsertJob.java | 4 +- .../StreamingTaskTxnCommitAttachment.java | 18 +- gensrc/proto/cloud.proto | 11 +- 9 files changed, 294 insertions(+), 74 deletions(-) diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index 22b2a449231f3c..e327e2bf473dc7 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -626,59 +626,54 @@ void update_streaming_job_meta(MetaServiceCode& code, std::string& msg, txn_commit_attachment.streaming_task_txn_commit_attachment(); int64_t job_id = commit_attachment.job_id(); - std::string streaming_meta_key; - std::string streaming_meta_val; - bool prev_meta_existed = true; - StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; - streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); - TxnErrorCode err = txn->get(streaming_meta_key, &streaming_meta_val); - if (err != TxnErrorCode::TXN_OK) { - if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { - prev_meta_existed = false; - } else { - code = cast_as(err); - ss << "failed to get streaming job meta, db_id=" << db_id << " txn_id=" << txn_id - << " err=" << err; - msg = ss.str(); - return; - } + std::string streaming_job_val; + bool prev_existed = true; + std::string streaming_job_key_str = streaming_job_key({instance_id, db_id, job_id}); + TxnErrorCode err = txn->get(streaming_job_key_str, &streaming_job_val); + if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { + prev_existed = false; + } else if (err != TxnErrorCode::TXN_OK) { + code = cast_as(err); + ss << "failed to get streaming job, db_id=" << db_id << " txn_id=" << txn_id + << " err=" << err; + msg = ss.str(); + return; } - StreamingTaskCommitAttachmentPB new_meta_info; - if (prev_meta_existed) { - if (!new_meta_info.ParseFromString(streaming_meta_val)) { + StreamingTaskCommitAttachmentPB new_job_info; + if (prev_existed) { + if (!new_job_info.ParseFromString(streaming_job_val)) { code = MetaServiceCode::PROTOBUF_PARSE_ERR; ss << "failed to parse streaming job meta, db_id=" << db_id << " txn_id=" << txn_id; msg = ss.str(); return; } - new_meta_info.set_scanned_rows(new_meta_info.scanned_rows() + - commit_attachment.scanned_rows()); - new_meta_info.set_load_bytes(new_meta_info.load_bytes() + commit_attachment.load_bytes()); - new_meta_info.set_file_number(new_meta_info.file_number() + - commit_attachment.file_number()); - new_meta_info.set_file_size(new_meta_info.file_size() + commit_attachment.file_size()); + new_job_info.set_scanned_rows(new_job_info.scanned_rows() + + commit_attachment.scanned_rows()); + new_job_info.set_load_bytes(new_job_info.load_bytes() + commit_attachment.load_bytes()); + new_job_info.set_num_files(new_job_info.num_files() + commit_attachment.num_files()); + new_job_info.set_file_bytes(new_job_info.file_bytes() + commit_attachment.file_bytes()); } else { - new_meta_info.set_job_id(commit_attachment.job_id()); - new_meta_info.set_scanned_rows(commit_attachment.scanned_rows()); - new_meta_info.set_load_bytes(commit_attachment.load_bytes()); - new_meta_info.set_file_number(commit_attachment.file_number()); - new_meta_info.set_file_size(commit_attachment.file_size()); + new_job_info.set_job_id(commit_attachment.job_id()); + new_job_info.set_scanned_rows(commit_attachment.scanned_rows()); + new_job_info.set_load_bytes(commit_attachment.load_bytes()); + new_job_info.set_num_files(commit_attachment.num_files()); + new_job_info.set_file_bytes(commit_attachment.file_bytes()); } if (commit_attachment.has_offset()) { - new_meta_info.set_offset(commit_attachment.offset()); + new_job_info.set_offset(commit_attachment.offset()); } - std::string new_meta_val; - if (!new_meta_info.SerializeToString(&new_meta_val)) { + std::string new_job_val; + if (!new_job_info.SerializeToString(&new_job_val)) { code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; - ss << "failed to serialize new streaming meta val, txn_id=" << txn_id; + ss << "failed to serialize new streaming job val, txn_id=" << txn_id; msg = ss.str(); return; } - txn->put(streaming_meta_key, new_meta_val); - LOG(INFO) << "put streaming_meta_key key=" << hex(streaming_meta_key) - << " streaming job new meta: " << new_meta_info.ShortDebugString(); + txn->put(streaming_job_key_str, new_job_val); + LOG(INFO) << "put streaming_job_key key=" << hex(streaming_job_key_str) + << " streaming job new meta: " << new_job_info.ShortDebugString(); } void MetaServiceImpl::get_rl_task_commit_attach(::google::protobuf::RpcController* controller, @@ -780,11 +775,9 @@ void MetaServiceImpl::get_streaming_task_commit_attach( int64_t db_id = request->db_id(); int64_t job_id = request->job_id(); - std::string streaming_meta_key; - std::string streaming_meta_val; - StreamingJobMetaKeyInfo streaming_meta_key_info {instance_id, db_id, job_id}; - streaming_job_meta_key_info(streaming_meta_key_info, &streaming_meta_key); - err = txn->get(streaming_meta_key, &streaming_meta_val); + std::string streaming_job_val; + std::string streaming_job_key_str = streaming_job_key({instance_id, db_id, job_id}); + err = txn->get(streaming_job_key_str, &streaming_job_val); if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) { code = MetaServiceCode::STREAMING_JOB_PROGRESS_NOT_FOUND; ss << "progress info not found, db_id=" << db_id << " job_id=" << job_id << " err=" << err; @@ -799,7 +792,7 @@ void MetaServiceImpl::get_streaming_task_commit_attach( } StreamingTaskCommitAttachmentPB* commit_attach = response->mutable_commit_attach(); - if (!commit_attach->ParseFromString(streaming_meta_val)) { + if (!commit_attach->ParseFromString(streaming_job_val)) { code = MetaServiceCode::PROTOBUF_PARSE_ERR; ss << "failed to parse meta info, db_id=" << db_id << " job_id=" << job_id; msg = ss.str(); diff --git a/cloud/src/meta-store/keys.cpp b/cloud/src/meta-store/keys.cpp index 8a6fed10f283ea..91b9266c815626 100644 --- a/cloud/src/meta-store/keys.cpp +++ b/cloud/src/meta-store/keys.cpp @@ -65,7 +65,7 @@ static const char* STATS_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_TABLET = "tablet"; static const char* JOB_KEY_INFIX_RL_PROGRESS = "routine_load_progress"; -static const char* JOB_KEY_INFIX_STREAMING_JOB_META = "streaming_job_meta"; +static const char* JOB_KEY_INFIX_STREAMING_JOB = "streaming_job"; static const char* JOB_KEY_INFIX_RESTORE_TABLET = "restore_tablet"; static const char* JOB_KEY_INFIX_RESTORE_ROWSET = "restore_rowset"; @@ -146,7 +146,7 @@ static void encode_prefix(const T& t, std::string* key) { MetaDeleteBitmapInfo, MetaDeleteBitmapUpdateLockInfo, MetaPendingDeleteBitmapInfo, PartitionVersionKeyInfo, RecycleIndexKeyInfo, RecyclePartKeyInfo, RecycleRowsetKeyInfo, RecycleTxnKeyInfo, RecycleStageKeyInfo, StatsTabletKeyInfo, TableVersionKeyInfo, JobRestoreTabletKeyInfo, JobRestoreRowsetKeyInfo, - JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, StreamingJobMetaKeyInfo, + JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, StreamingJobKeyInfo, CopyJobKeyInfo, CopyFileKeyInfo, StorageVaultKeyInfo, MetaSchemaPBDictionaryInfo, MowTabletJobInfo>); @@ -184,7 +184,7 @@ static void encode_prefix(const T& t, std::string* key) { } else if constexpr (std::is_same_v || std::is_same_v || std::is_same_v - || std::is_same_v) { + || std::is_same_v) { encode_bytes(JOB_KEY_PREFIX, key); } else if constexpr (std::is_same_v || std::is_same_v) { @@ -466,11 +466,11 @@ void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out) encode_int64(std::get<2>(in), out); // job_id } -void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out) { - encode_prefix(in, out); // 0x01 "job" ${instance_id} - encode_bytes(JOB_KEY_INFIX_STREAMING_JOB_META, out); // "streaming_job_meta" - encode_int64(std::get<1>(in), out); // db_id - encode_int64(std::get<2>(in), out); // job_id +void streaming_job_key(const StreamingJobKeyInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "job" ${instance_id} + encode_bytes(JOB_KEY_INFIX_STREAMING_JOB, out); // "streaming_job" + encode_int64(std::get<1>(in), out); // db_id + encode_int64(std::get<2>(in), out); // job_id } void job_restore_tablet_key(const JobRestoreTabletKeyInfo& in, std::string* out) { diff --git a/cloud/src/meta-store/keys.h b/cloud/src/meta-store/keys.h index 0a7cdccdb6d048..020702982d7da0 100644 --- a/cloud/src/meta-store/keys.h +++ b/cloud/src/meta-store/keys.h @@ -66,6 +66,8 @@ // 0x01 "job" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletJobInfoPB // 0x01 "job" ${instance_id} "recycle" -> JobRecyclePB // 0x01 "job" ${instance_id} "check" -> JobRecyclePB +// 0x01 "job" ${instance_id} "streaming_job" ${db_id} ${job_id} -> StreamingJobPB + // // 0x01 "copy" ${instance_id} "job" ${stage_id} ${table_id} ${copy_id} ${group_id} -> CopyJobPB // 0x01 "copy" ${instance_id} "loading_file" ${stage_id} ${table_id} ${obj_name} ${etag} -> CopyFilePB @@ -219,7 +221,7 @@ using MetaPendingDeleteBitmapInfo = BasicKeyInfo<24 , std::tuple>; // 0:instance_id 1:db_id 2:job_id -using StreamingJobMetaKeyInfo = BasicKeyInfo<52, std::tuple>; +using StreamingJobKeyInfo = BasicKeyInfo<52, std::tuple>; // 0:instance_id 1:vault_id using StorageVaultKeyInfo = BasicKeyInfo<26, std::tuple>; @@ -410,8 +412,8 @@ void job_tablet_key(const JobTabletKeyInfo& in, std::string* out); static inline std::string job_tablet_key(const JobTabletKeyInfo& in) { std::string s; job_tablet_key(in, &s); return s; } void rl_job_progress_key_info(const RLJobProgressKeyInfo& in, std::string* out); static inline std::string rl_job_progress_key_info(const RLJobProgressKeyInfo& in) { std::string s; rl_job_progress_key_info(in, &s); return s; } -void streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in, std::string* out); -static inline std::string streaming_job_meta_key_info(const StreamingJobMetaKeyInfo& in) { std::string s; streaming_job_meta_key_info(in, &s); return s; } +void streaming_job_key(const StreamingJobKeyInfo& in, std::string* out); +static inline std::string streaming_job_key(const StreamingJobKeyInfo& in) { std::string s; streaming_job_key(in, &s); return s; } std::string copy_key_prefix(std::string_view instance_id); void copy_job_key(const CopyJobKeyInfo& in, std::string* out); diff --git a/cloud/test/keys_test.cpp b/cloud/test/keys_test.cpp index 94d331fcf27e91..fee168843c85e0 100644 --- a/cloud/test/keys_test.cpp +++ b/cloud/test/keys_test.cpp @@ -79,6 +79,8 @@ int decode_bytes(std::string_view* in, std::string* out); // 0x01 "trash" ${instacne_id} "table" -> TableTrashPB // // 0x01 "node_status" ${instance_id} "compute" ${backend_id} -> ComputeNodeStatusPB +// +// 0x01 "job" ${instance_id} "streaming_job" ${db_id} ${job_id} -> StreamingJobPB // clang-format on TEST(KeysTest, InstanceKeyTest) { @@ -1963,6 +1965,56 @@ TEST(KeysTest, RestoreJobKeysTest) { } } +TEST(KeysTest, StreamingJobKeysTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_deadbeef"; + + // 0x01 "job" ${instance_id} "streaming_job" ${db_id} ${job_id} -> StreamingJobPB + { + int64_t db_id = 123; + int64_t job_id = 456; + StreamingJobKeyInfo streaming_key {instance_id, db_id, job_id}; + std::string encoded_streaming_key0; + streaming_job_key(streaming_key, &encoded_streaming_key0); + std::cout << hex(encoded_streaming_key0) << std::endl; + + std::string dec_instance_id; + int64_t dec_db_id = 0; + int64_t dec_job_id = 0; + + std::string_view key_sv(encoded_streaming_key0); + std::string dec_job_prefix; + std::string dec_streaming_infix; + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &dec_job_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &dec_streaming_infix), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_db_id), 0); + ASSERT_EQ(decode_int64(&key_sv, &dec_job_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("job", dec_job_prefix); + EXPECT_EQ("streaming_job", dec_streaming_infix); + EXPECT_EQ(instance_id, dec_instance_id); + EXPECT_EQ(db_id, dec_db_id); + EXPECT_EQ(job_id, dec_job_id); + + std::get<2>(streaming_key) = job_id + 1; + std::string encoded_streaming_key1; + streaming_job_key(streaming_key, &encoded_streaming_key1); + std::cout << hex(encoded_streaming_key1) << std::endl; + + ASSERT_GT(encoded_streaming_key1, encoded_streaming_key0); + + std::get<1>(streaming_key) = db_id + 1; + std::string encoded_streaming_key2; + streaming_job_key(streaming_key, &encoded_streaming_key2); + std::cout << hex(encoded_streaming_key2) << std::endl; + + ASSERT_GT(encoded_streaming_key2, encoded_streaming_key0); + } +} + TEST(KeysTest, VersionedKeyPrefixTest) { using namespace doris::cloud; using namespace doris::cloud::versioned; diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp index c30be90e1dad35..9fa021e138f881 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -4990,4 +4990,179 @@ TEST(MetaServiceJobTest, IdempotentCompactionJob) { } } +TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { + auto meta_service = get_meta_service(); + ASSERT_NE(meta_service, nullptr); + + // Test case 1: First time update (job doesn't exist) + { + CommitTxnRequest request; + request.set_cloud_unique_id("test_cloud_unique_id"); + request.set_txn_id(12345); + + TxnCommitAttachmentPB* attachment = request.mutable_commit_attachment(); + attachment->set_type(TxnCommitAttachmentPB::STREAMING_TASK_TXN_COMMIT_ATTACHMENT); + + StreamingTaskCommitAttachmentPB* streaming_attach = + attachment->mutable_streaming_task_txn_commit_attachment(); + streaming_attach->set_job_id(1001); + streaming_attach->set_task_id(2001); + streaming_attach->set_offset("test_offset_1"); + streaming_attach->set_scanned_rows(1000); + streaming_attach->set_load_bytes(5000); + streaming_attach->set_num_files(10); + streaming_attach->set_file_bytes(8000); + + CommitTxnResponse response; + brpc::Controller cntl; + meta_service->commit_txn(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::OK); + } + + // Test case 2: Update existing job (accumulate values) + { + CommitTxnRequest request; + request.set_cloud_unique_id("test_cloud_unique_id"); + request.set_txn_id(12346); + + TxnCommitAttachmentPB* attachment = request.mutable_commit_attachment(); + attachment->set_type(TxnCommitAttachmentPB::STREAMING_TASK_TXN_COMMIT_ATTACHMENT); + + StreamingTaskCommitAttachmentPB* streaming_attach = + attachment->mutable_streaming_task_txn_commit_attachment(); + streaming_attach->set_job_id(1001); // Same job_id as before + streaming_attach->set_task_id(2002); + streaming_attach->set_offset("test_offset_2"); + streaming_attach->set_scanned_rows(500); + streaming_attach->set_load_bytes(2000); + streaming_attach->set_num_files(5); + streaming_attach->set_file_bytes(3000); + + CommitTxnResponse response; + brpc::Controller cntl; + meta_service->commit_txn(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::OK); + } + + // Test case 3: Missing commit attachment + { + CommitTxnRequest request; + request.set_cloud_unique_id("test_cloud_unique_id"); + request.set_txn_id(12347); + // No commit attachment set + + CommitTxnResponse response; + brpc::Controller cntl; + meta_service->commit_txn(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::INVALID_ARGUMENT); + EXPECT_TRUE(response.status().msg().find("missing commit attachment") != std::string::npos); + } +} + +TEST(MetaServiceJobTest, GetStreamingTaskCommitAttachTest) { + auto meta_service = get_meta_service(); + ASSERT_NE(meta_service, nullptr); + + // First, create a streaming job by committing a txn + { + CommitTxnRequest request; + request.set_cloud_unique_id("test_cloud_unique_id"); + request.set_txn_id(12348); + + TxnCommitAttachmentPB* attachment = request.mutable_commit_attachment(); + attachment->set_type(TxnCommitAttachmentPB::STREAMING_TASK_TXN_COMMIT_ATTACHMENT); + + StreamingTaskCommitAttachmentPB* streaming_attach = + attachment->mutable_streaming_task_txn_commit_attachment(); + streaming_attach->set_job_id(1002); + streaming_attach->set_task_id(3001); + streaming_attach->set_offset("test_offset_3"); + streaming_attach->set_scanned_rows(2000); + streaming_attach->set_load_bytes(10000); + streaming_attach->set_num_files(20); + streaming_attach->set_file_bytes(15000); + + CommitTxnResponse response; + brpc::Controller cntl; + meta_service->commit_txn(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::OK); + } + + // Test case 1: Get existing streaming job + { + GetStreamingTaskCommitAttachRequest request; + request.set_cloud_unique_id("test_cloud_unique_id"); + request.set_db_id(1000); // Assuming db_id from the commit + request.set_job_id(1002); + + GetStreamingTaskCommitAttachResponse response; + brpc::Controller cntl; + meta_service->get_streaming_task_commit_attach(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::OK); + EXPECT_TRUE(response.has_commit_attach()); + EXPECT_EQ(response.commit_attach().job_id(), 1002); + EXPECT_EQ(response.commit_attach().scanned_rows(), 2000); + EXPECT_EQ(response.commit_attach().load_bytes(), 10000); + EXPECT_EQ(response.commit_attach().num_files(), 20); + EXPECT_EQ(response.commit_attach().file_bytes(), 15000); + } + + // Test case 2: Get non-existent streaming job + { + GetStreamingTaskCommitAttachRequest request; + request.set_cloud_unique_id("test_cloud_unique_id"); + request.set_db_id(1000); + request.set_job_id(9999); // Non-existent job_id + + GetStreamingTaskCommitAttachResponse response; + brpc::Controller cntl; + meta_service->get_streaming_task_commit_attach(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::STREAMING_JOB_PROGRESS_NOT_FOUND); + EXPECT_TRUE(response.status().msg().find("progress info not found") != std::string::npos); + } + + // Test case 3: Missing required fields + { + GetStreamingTaskCommitAttachRequest request; + request.set_cloud_unique_id("test_cloud_unique_id"); + // Missing db_id and job_id + + GetStreamingTaskCommitAttachResponse response; + brpc::Controller cntl; + meta_service->get_streaming_task_commit_attach(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::INVALID_ARGUMENT); + EXPECT_TRUE(response.status().msg().find("empty db_id or job_id") != std::string::npos); + } + + // Test case 4: Invalid cloud_unique_id + { + GetStreamingTaskCommitAttachRequest request; + request.set_cloud_unique_id("invalid_cloud_unique_id"); + request.set_db_id(1000); + request.set_job_id(1002); + + GetStreamingTaskCommitAttachResponse response; + brpc::Controller cntl; + meta_service->get_streaming_task_commit_attach(&cntl, &request, &response, nullptr); + + EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); + EXPECT_EQ(response.status().code(), MetaServiceCode::INVALID_ARGUMENT); + EXPECT_TRUE(response.status().msg().find("empty instance_id") != std::string::npos); + } +} + } // namespace doris::cloud diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java index 424ce238544410..06cd25d1357418 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java @@ -282,11 +282,10 @@ public static TxnCommitAttachmentPB streamingTaskTxnCommitAttachmentToPb(Streami StreamingTaskCommitAttachmentPB.newBuilder(); builder.setJobId(streamingTaskTxnCommitAttachment.getJobId()) - .setTaskId(streamingTaskTxnCommitAttachment.getTaskId()) .setScannedRows(streamingTaskTxnCommitAttachment.getScannedRows()) .setLoadBytes(streamingTaskTxnCommitAttachment.getLoadBytes()) - .setFileNumber(streamingTaskTxnCommitAttachment.getFileNumber()) - .setFileSize(streamingTaskTxnCommitAttachment.getFileSize()); + .setNumFiles(streamingTaskTxnCommitAttachment.getNumFiles()) + .setFileBytes(streamingTaskTxnCommitAttachment.getFileBytes()); if (streamingTaskTxnCommitAttachment.getOffset() != null) { builder.setOffset(streamingTaskTxnCommitAttachment.getOffset()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 479698b6f44f19..e8fd03c8de4697 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -305,8 +305,8 @@ private void updateJobStatisticAndOffset(StreamingTaskTxnCommitAttachment attach } this.jobStatistic.setScannedRows(this.jobStatistic.getScannedRows() + attachment.getScannedRows()); this.jobStatistic.setLoadBytes(this.jobStatistic.getLoadBytes() + attachment.getLoadBytes()); - this.jobStatistic.setFileNumber(this.jobStatistic.getFileNumber() + attachment.getFileNumber()); - this.jobStatistic.setFileSize(this.jobStatistic.getFileSize() + attachment.getFileSize()); + this.jobStatistic.setFileNumber(this.jobStatistic.getFileNumber() + attachment.getNumFiles()); + this.jobStatistic.setFileSize(this.jobStatistic.getFileSize() + attachment.getFileBytes()); offsetProvider.updateOffset(offsetProvider.deserializeOffset(attachment.getOffset())); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java index 4b7590824b4ee2..b3f6a2c0b1b90a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingTaskTxnCommitAttachment.java @@ -27,14 +27,14 @@ public class StreamingTaskTxnCommitAttachment extends TxnCommitAttachment { public StreamingTaskTxnCommitAttachment(long jobId, long taskId, - long scannedRows, long loadBytes, long fileNumber, long fileSize, String offset) { + long scannedRows, long loadBytes, long numFiles, long fileBytes, String offset) { super(TransactionState.LoadJobSourceType.STREAMING_JOB); this.jobId = jobId; this.taskId = taskId; this.scannedRows = scannedRows; this.loadBytes = loadBytes; - this.fileNumber = fileNumber; - this.fileSize = fileSize; + this.numFiles = numFiles; + this.fileBytes = fileBytes; this.offset = offset; } @@ -42,8 +42,8 @@ public StreamingTaskTxnCommitAttachment(StreamingTaskCommitAttachmentPB pb) { super(TransactionState.LoadJobSourceType.STREAMING_JOB); this.scannedRows = pb.getScannedRows(); this.loadBytes = pb.getLoadBytes(); - this.fileNumber = pb.getFileNumber(); - this.fileSize = pb.getFileSize(); + this.numFiles = pb.getNumFiles(); + this.fileBytes = pb.getFileBytes(); this.offset = pb.getOffset(); } @@ -59,10 +59,10 @@ public StreamingTaskTxnCommitAttachment(StreamingTaskCommitAttachmentPB pb) { private long loadBytes; @SerializedName(value = "fn") @Getter - private long fileNumber; + private long numFiles; @SerializedName(value = "fs") @Getter - private long fileSize; + private long fileBytes; @SerializedName(value = "of") @Getter private String offset; @@ -72,8 +72,8 @@ public String toString() { return "StreamingTaskTxnCommitAttachment: [" + "scannedRows=" + scannedRows + ", loadBytes=" + loadBytes - + ", fileNumber=" + fileNumber - + ", fileSize=" + fileSize + + ", numFiles=" + numFiles + + ", fileBytes=" + fileBytes + ", offset=" + offset + "]"; } diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index 6259d6d0da3208..3428cff6cab212 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -368,12 +368,11 @@ message RoutineLoadJobStatisticPB { message StreamingTaskCommitAttachmentPB { optional int64 job_id = 1; - optional int64 task_id = 2; - optional string offset = 3; - optional int64 scanned_rows = 4; - optional int64 load_bytes = 5; - optional int64 file_number = 6; - optional int64 file_size = 7; + optional string offset = 2; + optional int64 scanned_rows = 3; + optional int64 load_bytes = 4; + optional int64 num_files = 5; + optional int64 file_bytes = 6; } message TxnCommitAttachmentPB { From e0ee446b8d6458cfc495a594d7948a2f464953e2 Mon Sep 17 00:00:00 2001 From: wudi Date: Mon, 22 Sep 2025 18:39:03 +0800 Subject: [PATCH 22/25] [Feature](wip) improve streamjob and delete alterstreamjob op (#56308) ### What problem does this PR solve? improve streamjob and delete alterstreamjob op --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../property/storage/HdfsPropertiesUtils.java | 12 ++- .../property/storage/StorageProperties.java | 57 ++---------- .../java/org/apache/doris/fs/FileSystem.java | 2 +- .../org/apache/doris/fs/GlobListResult.java | 57 ++++++++++++ .../org/apache/doris/fs/obj/S3ObjStorage.java | 33 ++----- .../apache/doris/fs/remote/RemoteFile.java | 18 ---- .../apache/doris/fs/remote/S3FileSystem.java | 3 +- .../apache/doris/job/base/AbstractJob.java | 9 +- .../job/executor/DispatchTaskHandler.java | 5 +- .../insert/streaming/StreamingInsertJob.java | 30 +++---- .../streaming/StreamingJobProperties.java | 31 +++---- .../apache/doris/job/manager/JobManager.java | 23 ++--- .../org/apache/doris/job/offset/Offset.java | 2 + .../job/offset/SourceOffsetProvider.java | 14 +-- .../apache/doris/job/offset/s3/S3Offset.java | 6 ++ .../job/offset/s3/S3SourceOffsetProvider.java | 69 +++++++++------ .../apache/doris/journal/JournalEntity.java | 6 -- .../trees/plans/commands/AlterJobCommand.java | 6 +- .../plans/commands/info/CreateJobInfo.java | 4 +- .../insert/InsertIntoTableCommand.java | 19 +--- .../AlterStreamingJobOperationLog.java | 86 ------------------- .../org/apache/doris/persist/EditLog.java | 9 -- .../apache/doris/persist/OperationType.java | 2 - .../org/apache/doris/qe/SessionVariable.java | 1 + .../test_streaming_insert_job.groovy | 10 +-- 26 files changed, 195 insertions(+), 321 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/fs/GlobListResult.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d4cfe9ead5db19..3062dfef98854f 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -112,7 +112,7 @@ supportedJobStatement commentSpec? DO supportedDmlStatement #createScheduledJob | PAUSE JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #pauseJob - | ALTER JOB FOR (jobName=multipartIdentifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob + | ALTER JOB (jobName=multipartIdentifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob | DROP JOB (IF EXISTS)? WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #dropJob | RESUME JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #resumeJob | CANCEL TASK WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) AND (taskIdKey=identifier) EQ (taskIdValue=INTEGER_VALUE) #cancelJobTask diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java index 7fc091daa648b4..8f2e92d1e6461e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java @@ -23,6 +23,8 @@ import com.google.common.base.Strings; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.UnsupportedEncodingException; import java.net.URI; @@ -38,6 +40,7 @@ import java.util.stream.Collectors; public class HdfsPropertiesUtils { + private static final Logger LOG = LogManager.getLogger(HdfsPropertiesUtils.class); private static final String URI_KEY = "uri"; private static final String STANDARD_HDFS_PREFIX = "hdfs://"; private static final String EMPTY_HDFS_PREFIX = "hdfs:///"; @@ -63,7 +66,14 @@ public static boolean validateUriIsHdfsUri(Map props, if (StringUtils.isBlank(uriStr)) { return false; } - URI uri = URI.create(uriStr); + URI uri; + try { + uri = URI.create(uriStr); + } catch (Exception ex) { + // The glob syntax of s3 contains {, which will cause an error here. + LOG.warn("Failed to validate uri is hdfs uri, {}", ex.getMessage()); + return false; + } String schema = uri.getScheme(); if (StringUtils.isBlank(schema)) { throw new IllegalArgumentException("Invalid uri: " + uriStr + ", extract schema is null"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java index 337b897f256796..2ce87f9ffb9772 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java @@ -146,62 +146,17 @@ public static List createAll(Map origProps) t * @throws RuntimeException if no supported storage type is found */ public static StorageProperties createPrimary(Map origProps) { - StorageProperties p = createPrimaryInternal(origProps); - if (p == null) { - for (Function, StorageProperties> func : PROVIDERS) { - p = func.apply(origProps); - if (p != null) { - break; - } + for (Function, StorageProperties> func : PROVIDERS) { + StorageProperties p = func.apply(origProps); + if (p != null) { + p.initNormalizeAndCheckProps(); + p.initializeHadoopStorageConfig(); + return p; } } - if (p != null) { - p.initNormalizeAndCheckProps(); - p.initializeHadoopStorageConfig(); - return p; - } throw new StoragePropertiesException("No supported storage type found. Please check your configuration."); } - private static StorageProperties createPrimaryInternal(Map origProps) { - String provider = origProps.get(FS_PROVIDER_KEY); - if (provider == null) { - return null; - } - - try { - Type type = Type.valueOf(provider.trim().toUpperCase()); - switch (type) { - case HDFS: - return new HdfsProperties(origProps); - case OSS_HDFS: - return new OSSHdfsProperties(origProps); - case S3: - return new S3Properties(origProps); - case OSS: - return new OSSProperties(origProps); - case OBS: - return new OBSProperties(origProps); - case COS: - return new COSProperties(origProps); - case GCS: - return new GCSProperties(origProps); - case AZURE: - return new AzureProperties(origProps); - case MINIO: - return new MinioProperties(origProps); - case BROKER: - return new BrokerProperties(origProps); - case LOCAL: - return new LocalProperties(origProps); - default: - return null; - } - } catch (Exception e) { - return null; - } - } - private static final List, StorageProperties>> PROVIDERS = Arrays.asList( props -> (isFsSupport(props, FS_HDFS_SUPPORT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java index b2c6957ce382ba..81cdaf5d2a606b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/FileSystem.java @@ -124,7 +124,7 @@ default Status globList(String remotePath, List result) { * @param fileNumLimit limit the total number of files to be listed. * @return */ - default String globListWithLimit(String remotePath, List result, + default GlobListResult globListWithLimit(String remotePath, List result, String startFile, long fileSizeLimit, long fileNumLimit) { throw new UnsupportedOperationException("Unsupported operation glob list with limit on current file system."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/GlobListResult.java b/fe/fe-core/src/main/java/org/apache/doris/fs/GlobListResult.java new file mode 100644 index 00000000000000..af0d2817ffcff1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/GlobListResult.java @@ -0,0 +1,57 @@ +// 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.fs; + +import org.apache.doris.backup.Status; + +public class GlobListResult { + private final Status status; + private final String maxFile; + private final String bucket; + private final String prefix; + + public GlobListResult(Status status, String maxFile, String bucket, String prefix) { + this.status = status; + this.maxFile = maxFile; + this.bucket = bucket; + this.prefix = prefix; + } + + public GlobListResult(Status status) { + this.status = status; + this.maxFile = ""; + this.bucket = ""; + this.prefix = ""; + } + + public Status getStatus() { + return status; + } + + public String getMaxFile() { + return maxFile; + } + + public String getBucket() { + return bucket; + } + + public String getPrefix() { + return prefix; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java index f7cc644017f154..9522449c7d9a68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/obj/S3ObjStorage.java @@ -24,6 +24,7 @@ import org.apache.doris.common.util.S3Util; import org.apache.doris.common.util.Util; import org.apache.doris.datasource.property.storage.AbstractS3CompatibleProperties; +import org.apache.doris.fs.GlobListResult; import org.apache.doris.fs.remote.RemoteFile; import org.apache.commons.lang3.StringUtils; @@ -541,13 +542,11 @@ public Status globList(String remotePath, List result, boolean fileN * Limit: Starting from startFile, until the total file size is greater than fileSizeLimit, * or the number of files is greater than fileNumLimit. * - * @return The largest file name after listObject this time + * @return GlobListResult */ - public String globListWithLimit(String remotePath, List result, String startFile, + public GlobListResult globListWithLimit(String remotePath, List result, String startFile, long fileSizeLimit, long fileNumLimit) { - GlobListResult globListResult = globListInternal(remotePath, result, true, startFile, fileSizeLimit, - fileNumLimit); - return globListResult.getMaxFile(); + return globListInternal(remotePath, result, false, startFile, fileSizeLimit, fileNumLimit); } /** @@ -643,8 +642,6 @@ private GlobListResult globListInternal(String remotePath, List resu isPrefix ? -1 : obj.size(), isPrefix ? 0 : obj.lastModified().toEpochMilli() ); - remoteFile.setBucket(bucket); - remoteFile.setParentPath(objPath.getParent().toString()); result.add(remoteFile); if (hasLimits && reachLimit(result.size(), matchFileSize, fileSizeLimit, fileNumLimit)) { @@ -677,11 +674,11 @@ private GlobListResult globListInternal(String remotePath, List resu if (LOG.isDebugEnabled()) { LOG.debug("remotePath:{}, result:{}", remotePath, result); } - return new GlobListResult(Status.OK, currentMaxFile); + return new GlobListResult(Status.OK, currentMaxFile, bucket, finalPrefix); } catch (Exception e) { LOG.warn("Errors while getting file status", e); return new GlobListResult(new Status(Status.ErrCode.COMMON_ERROR, - "Errors while getting file status " + Util.getRootCauseMessage(e)), ""); + "Errors while getting file status " + Util.getRootCauseMessage(e))); } finally { long endTime = System.nanoTime(); long duration = endTime - startTime; @@ -715,24 +712,6 @@ private static boolean reachLimit(int matchFileCnt, long matchFileSize, long siz return false; } - private static class GlobListResult { - private final Status status; - private final String maxFile; - - public GlobListResult(Status status, String maxFile) { - this.status = status; - this.maxFile = maxFile; - } - - public Status getStatus() { - return status; - } - - public String getMaxFile() { - return maxFile; - } - } - @Override public synchronized void close() throws Exception { if (client != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java index ac1f8add9475f1..1f6f0225278b07 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/RemoteFile.java @@ -36,8 +36,6 @@ public class RemoteFile { private long modificationTime; private Path path; BlockLocation[] blockLocations; - private String parentPath; - private String bucket; public RemoteFile(String name, boolean isFile, long size, long blockSize) { this(name, null, isFile, !isFile, size, blockSize, 0, null); @@ -77,22 +75,6 @@ public void setPath(Path path) { this.path = path; } - public String getBucket() { - return bucket; - } - - public void setBucket(String bucket) { - this.bucket = bucket; - } - - public String getParentPath() { - return parentPath; - } - - public void setParentPath(String parentPath) { - this.parentPath = parentPath; - } - public boolean isFile() { return isFile; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java index d4e5a23c20e4cb..007b67de84dd62 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fs/remote/S3FileSystem.java @@ -23,6 +23,7 @@ import org.apache.doris.common.util.S3URI; import org.apache.doris.datasource.property.storage.AbstractS3CompatibleProperties; import org.apache.doris.datasource.property.storage.StorageProperties; +import org.apache.doris.fs.GlobListResult; import org.apache.doris.fs.obj.S3ObjStorage; import org.apache.logging.log4j.LogManager; @@ -69,7 +70,7 @@ public Status globList(String remotePath, List result, boolean fileN } @Override - public String globListWithLimit(String remotePath, List result, String startFile, + public GlobListResult globListWithLimit(String remotePath, List result, String startFile, long fileSizeLimit, long fileNumLimit) { S3ObjStorage objStorage = (S3ObjStorage) this.objStorage; return objStorage.globListWithLimit(remotePath, result, startFile, fileSizeLimit, fileNumLimit); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index a58dddb55e7a4b..5b03ae6d18b0fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -31,7 +31,6 @@ import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; -import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.thrift.TCell; @@ -218,6 +217,10 @@ public List queryAllTasks() { } public List commonCreateTasks(TaskType taskType, C taskContext) { + if (JobExecuteType.STREAMING.equals(getJobConfig().getExecuteType())) { + taskType = TaskType.STREAMING; + } + if (!canCreateTask(taskType)) { log.info("job is not ready for scheduling, job id is {},job status is {}, taskType is {}", jobId, jobStatus, taskType); @@ -478,10 +481,6 @@ public void onReplayEnd(AbstractJob replayJob) throws JobException { log.info(new LogBuilder(LogKey.SCHEDULER_JOB, getJobId()).add("msg", "replay delete scheduler job").build()); } - public void onReplayUpdateStreaming(AlterStreamingJobOperationLog operationLog) { - log.info(new LogBuilder(LogKey.SCHEDULER_JOB, getJobId()).add("msg", "replay update streaming job").build()); - } - public boolean needPersist() { return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java index 23710c17659c4b..35b1f351f723d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java @@ -18,7 +18,6 @@ package org.apache.doris.job.executor; import org.apache.doris.job.base.AbstractJob; -import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.common.JobType; import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.common.TaskType; @@ -57,9 +56,7 @@ public void onEvent(TimerJobEvent event) { return; } if (event.getJob().isReadyForScheduling(null) && JobUtils.checkNeedSchedule(event.getJob())) { - TaskType taskType = JobExecuteType.STREAMING.equals(event.getJob().getJobConfig().getExecuteType()) - ? TaskType.STREAMING : TaskType.SCHEDULED; - List tasks = event.getJob().commonCreateTasks(taskType, null); + List tasks = event.getJob().commonCreateTasks(TaskType.SCHEDULED, null); if (CollectionUtils.isEmpty(tasks)) { log.warn("job is ready for scheduling, but create task is empty, skip scheduler," + "job id is {}," + " job name is {}", event.getJob().getJobId(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index e8fd03c8de4697..b32ecdcb94ec25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -46,7 +46,6 @@ import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; -import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.qe.ConnectContext; @@ -253,12 +252,6 @@ public boolean hasMoreDataToConsume() { return offsetProvider.hasMoreDataToConsume(); } - @Override - public void logUpdateOperation() { - AlterStreamingJobOperationLog log = - new AlterStreamingJobOperationLog(this.getJobId(), this.getJobStatus(), properties, getExecuteSql()); - Env.getCurrentEnv().getEditLog().logUpdateStreamingJob(log); - } @Override public void onTaskFail(StreamingJobSchedulerTask task) throws JobException { @@ -321,13 +314,16 @@ public void onReplayCreate() throws JobException { super.onReplayCreate(); } - @Override - public void onReplayUpdateStreaming(AlterStreamingJobOperationLog operationLog) { - super.onReplayUpdateStreaming(operationLog); - setJobStatus(operationLog.getStatus()); - this.properties = operationLog.getJobProperties(); + /** + * Because the offset statistics of the streamingInsertJob are all stored in txn, + * only some fields are replayed here. + * @param replayJob + */ + public void replayOnUpdated(StreamingInsertJob replayJob) { + setJobStatus(replayJob.getJobStatus()); + this.properties = replayJob.getProperties(); this.jobProperties = new StreamingJobProperties(properties); - setExecuteSql(operationLog.getExecuteSql()); + setExecuteSql(replayJob.getExecuteSql()); } @Override @@ -358,14 +354,14 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal(properties != null ? GsonUtils.GSON.toJson(properties) : FeConstants.null_string)); - if (offsetProvider != null && offsetProvider.getSyncOffset() != null) { - trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getSyncOffset())); + if (offsetProvider != null && offsetProvider.getConsumedOffset() != null) { + trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getConsumedOffset())); } else { trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); } - if (offsetProvider != null && offsetProvider.getRemoteOffset() != null) { - trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getRemoteOffset())); + if (offsetProvider != null && offsetProvider.getMaxOffset() != null) { + trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getMaxOffset())); } else { trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java index a79bc1b230d6d4..35aa601fceb7eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobProperties.java @@ -32,44 +32,45 @@ @Data public class StreamingJobProperties implements JobProperties { public static final String MAX_INTERVAL_SECOND_PROPERTY = "max_interval"; - public static final String S3_BATCH_FILES_PROPERTY = "s3.batch_files"; - public static final String S3_BATCH_SIZE_PROPERTY = "s3.batch_size"; + public static final String S3_MAX_BATCH_FILES_PROPERTY = "s3.max_batch_files"; + public static final String S3_MAX_BATCH_BYTES_PROPERTY = "s3.max_batch_bytes"; public static final String SESSION_VAR_PREFIX = "session."; public static final long DEFAULT_MAX_INTERVAL_SECOND = 10; - public static final long DEFAULT_S3_BATCH_FILES = 256; - public static final long DEFAULT_S3_BATCH_SIZE = 10 * 1024 * 1024 * 1024L; // 10GB + public static final long DEFAULT_MAX_S3_BATCH_FILES = 256; + public static final long DEFAULT_MAX_S3_BATCH_BYTES = 10 * 1024 * 1024 * 1024L; // 10GB public static final int DEFAULT_INSERT_TIMEOUT = 30 * 60; // 30min private final Map properties; private long maxIntervalSecond; private long s3BatchFiles; - private long s3BatchSize; + private long s3BatchBytes; public StreamingJobProperties(Map jobProperties) { this.properties = jobProperties; if (properties.isEmpty()) { this.maxIntervalSecond = DEFAULT_MAX_INTERVAL_SECOND; - this.s3BatchFiles = DEFAULT_S3_BATCH_FILES; - this.s3BatchSize = DEFAULT_S3_BATCH_SIZE; + this.s3BatchFiles = DEFAULT_MAX_S3_BATCH_FILES; + this.s3BatchBytes = DEFAULT_MAX_S3_BATCH_BYTES; } } public void validate() throws AnalysisException { this.maxIntervalSecond = Util.getLongPropertyOrDefault( - properties.get(StreamingJobProperties.MAX_INTERVAL_SECOND_PROPERTY), - StreamingJobProperties.DEFAULT_MAX_INTERVAL_SECOND, (v) -> v >= 1, + properties.get(StreamingJobProperties.MAX_INTERVAL_SECOND_PROPERTY), + StreamingJobProperties.DEFAULT_MAX_INTERVAL_SECOND, (v) -> v >= 1, StreamingJobProperties.MAX_INTERVAL_SECOND_PROPERTY + " should > 1"); this.s3BatchFiles = Util.getLongPropertyOrDefault( - properties.get(StreamingJobProperties.S3_BATCH_FILES_PROPERTY), - StreamingJobProperties.DEFAULT_S3_BATCH_FILES, (v) -> v >= 1, - StreamingJobProperties.S3_BATCH_FILES_PROPERTY + " should >=1 "); + properties.get(StreamingJobProperties.S3_MAX_BATCH_FILES_PROPERTY), + StreamingJobProperties.DEFAULT_MAX_S3_BATCH_FILES, (v) -> v >= 1, + StreamingJobProperties.S3_MAX_BATCH_FILES_PROPERTY + " should >=1 "); - this.s3BatchSize = Util.getLongPropertyOrDefault(properties.get(StreamingJobProperties.S3_BATCH_SIZE_PROPERTY), - StreamingJobProperties.DEFAULT_S3_BATCH_SIZE, (v) -> v >= 100 * 1024 * 1024 + this.s3BatchBytes = Util.getLongPropertyOrDefault( + properties.get(StreamingJobProperties.S3_MAX_BATCH_BYTES_PROPERTY), + StreamingJobProperties.DEFAULT_MAX_S3_BATCH_BYTES, (v) -> v >= 100 * 1024 * 1024 && v <= (long) (1024 * 1024 * 1024) * 10, - StreamingJobProperties.S3_BATCH_SIZE_PROPERTY + " should between 100MB and 10GB"); + StreamingJobProperties.S3_MAX_BATCH_BYTES_PROPERTY + " should between 100MB and 10GB"); } public SessionVariable getSessionVariable() throws JobException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 14b12f837e0df2..8ce9fb7ea73580 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -38,6 +38,7 @@ import org.apache.doris.job.common.TaskType; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.job.scheduler.JobScheduler; import org.apache.doris.job.scheduler.StreamingTaskScheduler; @@ -45,7 +46,6 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; @@ -368,21 +368,16 @@ public void replayUpdateJob(T job) { LOG.warn("replayUpdateJob not normal, job: {}, jobId: {}, jobMap: {}", job, jobId, jobMap); return; } - jobMap.put(jobId, job); - log.info(new LogBuilder(LogKey.SCHEDULER_JOB, jobId) - .add("msg", "replay update scheduler job").build()); - } - public void replayUpdateStreamingJob(AlterStreamingJobOperationLog log) { - Long jobId = log.getJobId(); - if (!jobMap.containsKey(jobId)) { - LOG.warn("replayUpdateStreamingJob not normal, jobId: {}, jobMap: {}", jobId, log); - return; + if (job instanceof StreamingInsertJob) { + // for streaming job, we only update part properties + StreamingInsertJob currentJob = (StreamingInsertJob) jobMap.get(jobId); + currentJob.replayOnUpdated((StreamingInsertJob) job); + } else { + jobMap.put(jobId, job); } - T job = jobMap.get(jobId); - job.onReplayUpdateStreaming(log); - LOG.info(new LogBuilder(LogKey.SCHEDULER_JOB, jobId) - .add("msg", "replay update streaming job").build()); + log.info(new LogBuilder(LogKey.SCHEDULER_JOB, jobId) + .add("msg", "replay update scheduler job").build()); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java index 03e664ccd3e248..cebff9a39b941f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/Offset.java @@ -19,4 +19,6 @@ public interface Offset { String toSerializedJson(); + + boolean isEmpty(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index d7e1bee4669384..a27b66812c21c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -40,22 +40,16 @@ public interface SourceOffsetProvider { Offset getNextOffset(StreamingJobProperties jobProps, Map properties); /** - * Get current offset + * Get consumered offset to show * @return */ - Offset getCurrentOffset(); + String getConsumedOffset(); /** - * Get sync offset to show + * Get remote datasource max offset * @return */ - String getSyncOffset(); - - /** - * Get remote offset - * @return - */ - String getRemoteOffset(); + String getMaxOffset(); /** * Rewrite the TVF parameters in the SQL based on the current offset. diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index cb783adfbffdd3..b152660873a4c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -23,6 +23,7 @@ import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; +import org.apache.commons.lang3.StringUtils; @Getter @Setter @@ -39,6 +40,11 @@ public String toSerializedJson() { return GsonUtils.GSON.toJson(this); } + @Override + public boolean isEmpty() { + return StringUtils.isEmpty(fileLists); + } + @Override public String toString() { return "{ \"startFile\": \"" + startFile + "\", \"endFile\": \"" + endFile + "\" }"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index d433e42f73ca01..e6a5cf809a523d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -19,6 +19,7 @@ import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.fs.FileSystemFactory; +import org.apache.doris.fs.GlobListResult; import org.apache.doris.fs.remote.RemoteFile; import org.apache.doris.fs.remote.RemoteFileSystem; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; @@ -45,7 +46,7 @@ @Log4j2 public class S3SourceOffsetProvider implements SourceOffsetProvider { S3Offset currentOffset; - String maxRemoteEndFile; + String maxEndFile; @Override public String getSourceType() { @@ -64,18 +65,41 @@ public S3Offset getNextOffset(StreamingJobProperties jobProps, Map= 0) ? prefix.substring(0, lastSlash + 1) : ""; + String filePathBase = bucketBase + basePrefix; + String joined = rfiles.stream() + .filter(name -> !name.equals(filePathBase)) // Single file case + .map(path -> path.getName().replace(filePathBase, "")) + .collect(Collectors.joining(",")); + + if (joined.isEmpty()) { + // base is a single file + offset.setFileLists(filePathBase); + String lastFile = rfiles.get(rfiles.size() - 1).getName().replace(bucketBase, ""); + offset.setEndFile(lastFile); + } else { + // base is dir + String normalizedPrefix = basePrefix.endsWith("/") + ? basePrefix.substring(0, basePrefix.length() - 1) : basePrefix; + String finalFileLists = String.format("s3://%s/%s/{%s}", bucket, normalizedPrefix, joined); + String lastFile = rfiles.get(rfiles.size() - 1).getName().replace(bucketBase, ""); + offset.setFileLists(finalFileLists); + offset.setEndFile(lastFile); + } + maxEndFile = globListResult.getMaxFile(); + } else { + throw new RuntimeException("No new files found in path: " + filePath); } } catch (Exception e) { log.warn("list path exception, path={}", filePath, e); @@ -85,12 +109,7 @@ public S3Offset getNextOffset(StreamingJobProperties jobProps, Map properties) throws Exception { String uri = storageProperties.validateAndGetUri(copiedProps); String filePath = storageProperties.validateAndNormalizeUri(uri); List objects = new ArrayList<>(); - String endFile = fileSystem.globListWithLimit(filePath, objects, startFile, 1, 1); - if (!objects.isEmpty() && StringUtils.isNotEmpty(endFile)) { - maxRemoteEndFile = endFile; + GlobListResult globListResult = fileSystem.globListWithLimit(filePath, objects, startFile, 1, 1); + if (globListResult != null && !objects.isEmpty() && StringUtils.isNotEmpty(globListResult.getMaxFile())) { + maxEndFile = globListResult.getMaxFile(); } else { - maxRemoteEndFile = startFile; + maxEndFile = startFile; } } catch (Exception e) { throw e; @@ -154,7 +173,7 @@ public boolean hasMoreDataToConsume() { if (currentOffset == null) { return true; } - if (currentOffset.endFile.compareTo(maxRemoteEndFile) < 0) { + if (currentOffset.endFile.compareTo(maxEndFile) < 0) { return true; } return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index f0f9d471ed9227..cb6dee000ded01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -67,7 +67,6 @@ import org.apache.doris.persist.AlterLightSchemaChangeInfo; import org.apache.doris.persist.AlterMTMV; import org.apache.doris.persist.AlterRoutineLoadJobOperationLog; -import org.apache.doris.persist.AlterStreamingJobOperationLog; import org.apache.doris.persist.AlterUserOperationLog; import org.apache.doris.persist.AlterViewInfo; import org.apache.doris.persist.AnalyzeDeletionLog; @@ -551,11 +550,6 @@ public void readFields(DataInput in) throws IOException { isRead = true; break; } - case OperationType.OP_UPDATE_STREAMING_JOB: { - data = AlterStreamingJobOperationLog.read(in); - isRead = true; - break; - } case OperationType.OP_CREATE_SCHEDULER_TASK: case OperationType.OP_DELETE_SCHEDULER_TASK: { //todo improve diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index adde81fbfe90ae..8d5bd244df7aeb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -41,9 +41,9 @@ public class AlterJobCommand extends AlterCommand implements ForwardWithSync { // exclude job name prefix, which is used by inner job private static final String excludeJobNamePrefix = "inner_"; - private String jobName; - private Map properties; - private String sql; + private final String jobName; + private final Map properties; + private final String sql; public AlterJobCommand(String jobName, Map properties, String sql) { super(PlanType.ALTER_JOB_COMMAND); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 924f2bd484237a..fecac5e135c58b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -283,8 +283,8 @@ private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String curren throw new AnalysisException(e.getMessage()); } } else { - throw new AnalysisException("Not support this sql : " + sql + " Command class is " - + logicalPlan.getClass().getName() + "."); + throw new AnalysisException("Only " + logicalPlan.getClass().getName() + + " is supported to use with streaming job together"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 6f7cec71857e28..9e5c54099b8db9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -79,7 +79,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -544,24 +543,8 @@ public List getTargetColumns() { } } - // todo: add ut public List getAllTVFRelation() { - List tvfs = new ArrayList<>(); - findAllTVFInPlan(getLogicalQuery(), tvfs); - return tvfs; - } - - private void findAllTVFInPlan(LogicalPlan plan, List tvfs) { - if (plan instanceof UnboundTVFRelation) { - UnboundTVFRelation tvfRelation = (UnboundTVFRelation) plan; - tvfs.add(tvfRelation); - } - - for (Plan child : plan.children()) { - if (child instanceof LogicalPlan) { - findAllTVFInPlan((LogicalPlan) child, tvfs); - } - } + return getLogicalQuery().collectToList(UnboundTVFRelation.class::isInstance); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java deleted file mode 100644 index 785531fa0d14a9..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/AlterStreamingJobOperationLog.java +++ /dev/null @@ -1,86 +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.persist; - -import org.apache.doris.common.io.Text; -import org.apache.doris.common.io.Writable; -import org.apache.doris.job.common.JobStatus; -import org.apache.doris.persist.gson.GsonUtils; - -import com.google.gson.annotations.SerializedName; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.Map; - -public class AlterStreamingJobOperationLog implements Writable { - @SerializedName(value = "jid") - private long jobId; - @SerializedName(value = "js") - private JobStatus status; - @SerializedName(value = "jp") - private Map jobProperties; - @SerializedName(value = "sql") - String executeSql; - - public AlterStreamingJobOperationLog(long jobId, JobStatus status, - Map jobProperties, String executeSql) { - this.jobId = jobId; - this.status = status; - this.jobProperties = jobProperties; - this.executeSql = executeSql; - } - - public long getJobId() { - return jobId; - } - - public Map getJobProperties() { - return jobProperties; - } - - public String getExecuteSql() { - return executeSql; - } - - public JobStatus getStatus() { - return status; - } - - public static AlterStreamingJobOperationLog read(DataInput in) throws IOException { - String json = Text.readString(in); - return GsonUtils.GSON.fromJson(json, AlterStreamingJobOperationLog.class); - } - - @Override - public void write(DataOutput out) throws IOException { - String json = GsonUtils.GSON.toJson(this); - Text.writeString(out, json); - } - - @Override - public String toString() { - return "AlterStreamingJobOperationLog{" - + "jobId=" + jobId - + ", status=" + status - + ", jobProperties=" + jobProperties - + ", executeSql='" + executeSql + '\'' - + '}'; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 74be0f5e4753f8..5abcc4e04646b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -862,11 +862,6 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { Env.getCurrentEnv().getJobManager().replayDeleteJob(job); break; } - case OperationType.OP_UPDATE_STREAMING_JOB: { - AlterStreamingJobOperationLog log = (AlterStreamingJobOperationLog) journal.getData(); - Env.getCurrentEnv().getJobManager().replayUpdateStreamingJob(log); - break; - } /*case OperationType.OP_CREATE_SCHEDULER_TASK: { JobTask task = (JobTask) journal.getData(); Env.getCurrentEnv().getJobTaskManager().replayCreateTask(task); @@ -2043,10 +2038,6 @@ public void logUpdateJob(AbstractJob job) { logEdit(OperationType.OP_UPDATE_SCHEDULER_JOB, job); } - public void logUpdateStreamingJob(AlterStreamingJobOperationLog log) { - logEdit(OperationType.OP_UPDATE_STREAMING_JOB, log); - } - public void logDeleteJob(AbstractJob job) { logEdit(OperationType.OP_DELETE_SCHEDULER_JOB, job); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 12d59d086b9b74..2bbc03d4aeab87 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -414,8 +414,6 @@ public class OperationType { public static final short OP_OPERATE_KEY = 492; - public static final short OP_UPDATE_STREAMING_JOB = 493; - // For cloud. public static final short OP_UPDATE_CLOUD_REPLICA = 1000; @Deprecated diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 6c8506af7d7931..d5262d36a969c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -4761,6 +4761,7 @@ public void readFromJson(String json) throws IOException { field.set(this, root.get(attr.name())); break; case "double": + // root.get(attr.name()) always return Double type, so need to convert it. field.set(this, Double.valueOf(root.get(attr.name()).toString())); break; case "String": diff --git a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy index d89bc8d2f0f221..54fc5faa2e8950 100644 --- a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy @@ -46,7 +46,7 @@ suite("test_streaming_insert_job") { sql """ CREATE JOB ${jobName} PROPERTIES( - "s3.batch_files" = "1" + "s3.max_batch_files" = "1" ) ON STREAMING DO INSERT INTO ${tableName} SELECT * FROM S3 @@ -104,9 +104,9 @@ suite("test_streaming_insert_job") { // alter streaming job sql """ - ALTER JOB FOR ${jobName} + ALTER JOB ${jobName} PROPERTIES( - "session.insert_max_filter_ratio" = "0.5" + "session.insert_max_filter_ratio" = 0.5 ) INSERT INTO ${tableName} SELECT * FROM S3 @@ -126,7 +126,7 @@ suite("test_streaming_insert_job") { select status,properties,ConsumedOffset from jobs("type"="insert") where Name='${jobName}' """ assert alterJobProperties.get(0).get(0) == "PAUSED" - assert alterJobProperties.get(0).get(1) == "{\"s3.batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" + assert alterJobProperties.get(0).get(1) == "{\"s3.max_batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" assert alterJobProperties.get(0).get(2) == "regression/load/data/example_1.csv" sql """ @@ -136,7 +136,7 @@ suite("test_streaming_insert_job") { select status,properties,ConsumedOffset from jobs("type"="insert") where Name='${jobName}' """ assert resumeJobStatus.get(0).get(0) == "RUNNING" || resumeJobStatus.get(0).get(0) == "PENDING" - assert resumeJobStatus.get(0).get(1) == "{\"s3.batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" + assert resumeJobStatus.get(0).get(1) == "{\"s3.max_batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" assert resumeJobStatus.get(0).get(2) == "regression/load/data/example_1.csv" Awaitility.await().atMost(60, SECONDS) From 6d1374279cfba85d19de6501b37aaf3bf2a548c9 Mon Sep 17 00:00:00 2001 From: hui lai Date: Mon, 22 Sep 2025 20:20:00 +0800 Subject: [PATCH 23/25] [fix](streaming job) fix streaming job ut (#56314) ### What problem does this PR solve? [fix](streaming job) fix streaming job ut --- cloud/test/meta_service_job_test.cpp | 93 +++++++++++++++++++++------- 1 file changed, 69 insertions(+), 24 deletions(-) diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp index 9fa021e138f881..5d5758579eb5f8 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -4998,7 +4998,23 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { { CommitTxnRequest request; request.set_cloud_unique_id("test_cloud_unique_id"); - request.set_txn_id(12345); + // Begin a txn to obtain a valid txn_id and db_id mapping + int64_t db_id = 1000; + { + brpc::Controller cntl_bt; + BeginTxnRequest bt_req; + BeginTxnResponse bt_res; + auto* txn_info = bt_req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label("streaming_ut_1"); + txn_info->add_table_ids(1); + txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); + ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK); + request.set_txn_id(bt_res.txn_id()); + request.set_db_id(db_id); + } TxnCommitAttachmentPB* attachment = request.mutable_commit_attachment(); attachment->set_type(TxnCommitAttachmentPB::STREAMING_TASK_TXN_COMMIT_ATTACHMENT); @@ -5006,7 +5022,6 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { StreamingTaskCommitAttachmentPB* streaming_attach = attachment->mutable_streaming_task_txn_commit_attachment(); streaming_attach->set_job_id(1001); - streaming_attach->set_task_id(2001); streaming_attach->set_offset("test_offset_1"); streaming_attach->set_scanned_rows(1000); streaming_attach->set_load_bytes(5000); @@ -5025,7 +5040,23 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { { CommitTxnRequest request; request.set_cloud_unique_id("test_cloud_unique_id"); - request.set_txn_id(12346); + // Begin a txn to obtain a valid txn_id and db_id mapping + int64_t db_id = 1000; + { + brpc::Controller cntl_bt; + BeginTxnRequest bt_req; + BeginTxnResponse bt_res; + auto* txn_info = bt_req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label("streaming_ut_2"); + txn_info->add_table_ids(1); + txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); + ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK); + request.set_txn_id(bt_res.txn_id()); + request.set_db_id(db_id); + } TxnCommitAttachmentPB* attachment = request.mutable_commit_attachment(); attachment->set_type(TxnCommitAttachmentPB::STREAMING_TASK_TXN_COMMIT_ATTACHMENT); @@ -5033,7 +5064,6 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { StreamingTaskCommitAttachmentPB* streaming_attach = attachment->mutable_streaming_task_txn_commit_attachment(); streaming_attach->set_job_id(1001); // Same job_id as before - streaming_attach->set_task_id(2002); streaming_attach->set_offset("test_offset_2"); streaming_attach->set_scanned_rows(500); streaming_attach->set_load_bytes(2000); @@ -5052,7 +5082,23 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { { CommitTxnRequest request; request.set_cloud_unique_id("test_cloud_unique_id"); - request.set_txn_id(12347); + // Begin a txn to obtain a valid txn_id and db_id mapping + int64_t db_id = 1000; + { + brpc::Controller cntl_bt; + BeginTxnRequest bt_req; + BeginTxnResponse bt_res; + auto* txn_info = bt_req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label("streaming_ut_missing_attach"); + txn_info->add_table_ids(1); + txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); + ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK); + request.set_txn_id(bt_res.txn_id()); + request.set_db_id(db_id); + } // No commit attachment set CommitTxnResponse response; @@ -5073,7 +5119,23 @@ TEST(MetaServiceJobTest, GetStreamingTaskCommitAttachTest) { { CommitTxnRequest request; request.set_cloud_unique_id("test_cloud_unique_id"); - request.set_txn_id(12348); + // Begin a txn to obtain a valid txn_id and db_id mapping + int64_t db_id = 1000; + { + brpc::Controller cntl_bt; + BeginTxnRequest bt_req; + BeginTxnResponse bt_res; + auto* txn_info = bt_req.mutable_txn_info(); + txn_info->set_db_id(db_id); + txn_info->set_label("streaming_ut_3"); + txn_info->add_table_ids(1); + txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_timeout_ms(36000); + meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); + ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK); + request.set_txn_id(bt_res.txn_id()); + request.set_db_id(db_id); + } TxnCommitAttachmentPB* attachment = request.mutable_commit_attachment(); attachment->set_type(TxnCommitAttachmentPB::STREAMING_TASK_TXN_COMMIT_ATTACHMENT); @@ -5081,7 +5143,6 @@ TEST(MetaServiceJobTest, GetStreamingTaskCommitAttachTest) { StreamingTaskCommitAttachmentPB* streaming_attach = attachment->mutable_streaming_task_txn_commit_attachment(); streaming_attach->set_job_id(1002); - streaming_attach->set_task_id(3001); streaming_attach->set_offset("test_offset_3"); streaming_attach->set_scanned_rows(2000); streaming_attach->set_load_bytes(10000); @@ -5100,7 +5161,7 @@ TEST(MetaServiceJobTest, GetStreamingTaskCommitAttachTest) { { GetStreamingTaskCommitAttachRequest request; request.set_cloud_unique_id("test_cloud_unique_id"); - request.set_db_id(1000); // Assuming db_id from the commit + request.set_db_id(1000); // Must match db_id used when committing request.set_job_id(1002); GetStreamingTaskCommitAttachResponse response; @@ -5147,22 +5208,6 @@ TEST(MetaServiceJobTest, GetStreamingTaskCommitAttachTest) { EXPECT_EQ(response.status().code(), MetaServiceCode::INVALID_ARGUMENT); EXPECT_TRUE(response.status().msg().find("empty db_id or job_id") != std::string::npos); } - - // Test case 4: Invalid cloud_unique_id - { - GetStreamingTaskCommitAttachRequest request; - request.set_cloud_unique_id("invalid_cloud_unique_id"); - request.set_db_id(1000); - request.set_job_id(1002); - - GetStreamingTaskCommitAttachResponse response; - brpc::Controller cntl; - meta_service->get_streaming_task_commit_attach(&cntl, &request, &response, nullptr); - - EXPECT_FALSE(cntl.Failed()) << "Error: " << cntl.ErrorText(); - EXPECT_EQ(response.status().code(), MetaServiceCode::INVALID_ARGUMENT); - EXPECT_TRUE(response.status().msg().find("empty instance_id") != std::string::npos); - } } } // namespace doris::cloud From d471124ebd71ee72c911d7c6a36628d88fb3a934 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 23 Sep 2025 16:44:41 +0800 Subject: [PATCH 24/25] [Fix](job) improve job api (#56352) ### What problem does this PR solve? improve job api --- .../apache/doris/job/base/AbstractJob.java | 9 +++--- .../java/org/apache/doris/job/base/Job.java | 7 +++++ .../org/apache/doris/job/common/JobUtils.java | 30 ------------------- .../job/executor/DispatchTaskHandler.java | 3 +- .../job/executor/TimerJobSchedulerTask.java | 3 +- .../job/extensions/insert/InsertJob.java | 7 ++--- .../job/extensions/insert/InsertTask.java | 5 +--- .../insert/streaming/StreamingInsertJob.java | 20 +++++++++---- .../streaming/StreamingJobSchedulerTask.java | 2 -- .../job/offset/SourceOffsetProvider.java | 8 ++--- .../apache/doris/job/offset/s3/S3Offset.java | 4 +-- .../job/offset/s3/S3SourceOffsetProvider.java | 12 ++++---- .../doris/job/scheduler/JobScheduler.java | 7 ++--- .../test_streaming_insert_job.groovy | 14 ++++----- 14 files changed, 54 insertions(+), 77 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java index 5b03ae6d18b0fc..6a113d1a042ca9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -178,6 +178,11 @@ protected static long getNextJobId() { return System.nanoTime() + RandomUtils.nextInt(); } + @Override + public boolean isJobRunning() { + return JobStatus.RUNNING.equals(getJobStatus()); + } + @Override public void cancelTaskById(long taskId) throws JobException { if (CollectionUtils.isEmpty(runningTasks)) { @@ -484,8 +489,4 @@ public void onReplayEnd(AbstractJob replayJob) throws JobException { public boolean needPersist() { return true; } - - public boolean isFinalStatus() { - return jobStatus.equals(JobStatus.STOPPED) || jobStatus.equals(JobStatus.FINISHED); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java index 69d1e5e55fb01d..3afcd5eef9b93d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java @@ -68,6 +68,13 @@ public interface Job { */ boolean isReadyForScheduling(C taskContext); + /** + * Checks if the job is running. + * + * @return True if the job is runnning. + */ + boolean isJobRunning(); + /** * Retrieves the metadata for the job, which is used to display job information. * diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java deleted file mode 100644 index 93ff5f41480b38..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobUtils.java +++ /dev/null @@ -1,30 +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.job.common; - -import org.apache.doris.job.base.AbstractJob; -import org.apache.doris.job.base.JobExecuteType; - -public class JobUtils { - public static boolean checkNeedSchedule(AbstractJob job) { - if (job.getJobConfig().getExecuteType() == JobExecuteType.STREAMING) { - return !job.isFinalStatus(); - } - return job.getJobStatus() == JobStatus.RUNNING; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java index 35b1f351f723d5..228b9724863645 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java @@ -19,7 +19,6 @@ import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.common.JobType; -import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.disruptor.TimerJobEvent; import org.apache.doris.job.task.AbstractTask; @@ -55,7 +54,7 @@ public void onEvent(TimerJobEvent event) { log.info("job is null,may be job is deleted, ignore"); return; } - if (event.getJob().isReadyForScheduling(null) && JobUtils.checkNeedSchedule(event.getJob())) { + if (event.getJob().isReadyForScheduling(null) && event.getJob().isJobRunning()) { List tasks = event.getJob().commonCreateTasks(TaskType.SCHEDULED, null); if (CollectionUtils.isEmpty(tasks)) { log.warn("job is ready for scheduling, but create task is empty, skip scheduler," diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java index 0f58452ff7b065..4abf72023a4d27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java @@ -18,7 +18,6 @@ package org.apache.doris.job.executor; import org.apache.doris.job.base.AbstractJob; -import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.disruptor.TaskDisruptor; import io.netty.util.Timeout; @@ -40,7 +39,7 @@ public TimerJobSchedulerTask(TaskDisruptor dispatchDisruptor, T job) { @Override public void run(Timeout timeout) { try { - if (!JobUtils.checkNeedSchedule(job)) { + if (!job.isJobRunning()) { log.info("job status is not running, job id is {}, skip dispatch", this.job.getJobId()); return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java index cc16cbb603d351..305089378a0ae9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -100,8 +100,8 @@ public class InsertJob extends AbstractJob> impl .add(new Column("Comment", ScalarType.createStringType())) // only execute type = streaming need record .add(new Column("Properties", ScalarType.createStringType())) - .add(new Column("ConsumedOffset", ScalarType.createStringType())) - .add(new Column("MaxOffset", ScalarType.createStringType())) + .add(new Column("CurrentOffset", ScalarType.createStringType())) + .add(new Column("EndOffset", ScalarType.createStringType())) .add(new Column("LoadStatistic", ScalarType.createStringType())) .add(new Column("ErrorMsg", ScalarType.createStringType())) .build(); @@ -114,15 +114,12 @@ public class InsertJob extends AbstractJob> impl .addColumn(new Column("EtlInfo", ScalarType.createVarchar(100))) .addColumn(new Column("TaskInfo", ScalarType.createVarchar(100))) .addColumn(new Column("ErrorMsg", ScalarType.createVarchar(100))) - .addColumn(new Column("CreateTimeMs", ScalarType.createVarchar(20))) .addColumn(new Column("FinishTimeMs", ScalarType.createVarchar(20))) .addColumn(new Column("TrackingUrl", ScalarType.createVarchar(200))) .addColumn(new Column("LoadStatistic", ScalarType.createVarchar(200))) .addColumn(new Column("User", ScalarType.createVarchar(50))) .addColumn(new Column("FirstErrorMsg", ScalarType.createVarchar(200))) - // only execute type = streaming need record - .addColumn(new Column("Offset", ScalarType.createStringType())) .build(); public static final ImmutableMap COLUMN_TO_INDEX; diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java index 5875699fb0f50f..aa1ecc02a9f8e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -68,8 +68,7 @@ public class InsertTask extends AbstractTask { new Column("TrackingUrl", ScalarType.createStringType()), new Column("LoadStatistic", ScalarType.createStringType()), new Column("User", ScalarType.createStringType()), - new Column("FirstErrorMsg", ScalarType.createStringType()), - new Column("Offset", ScalarType.createStringType())); + new Column("FirstErrorMsg", ScalarType.createStringType())); public static final ImmutableMap COLUMN_TO_INDEX; @@ -277,7 +276,6 @@ public TRow getTvfInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); } trow.addToColumnValue(new TCell().setStringVal(firstErrorMsg == null ? "" : firstErrorMsg)); - trow.addToColumnValue(new TCell().setStringVal("")); return trow; } @@ -299,7 +297,6 @@ private TRow getPendingTaskTVFInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal("")); trow.addToColumnValue(new TCell().setStringVal(userIdentity.getQualifiedUser())); trow.addToColumnValue(new TCell().setStringVal("")); - trow.addToColumnValue(new TCell().setStringVal("")); return trow; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index b32ecdcb94ec25..f70cbc37fcae07 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -63,6 +63,7 @@ import lombok.Setter; import lombok.extern.log4j.Log4j2; import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import java.io.DataOutput; import java.io.IOException; @@ -203,7 +204,16 @@ protected void checkJobParamsInternal() { @Override public boolean isReadyForScheduling(Map taskContext) { - return CollectionUtils.isEmpty(getRunningTasks()); + return CollectionUtils.isEmpty(getRunningTasks()) && !isFinalStatus(); + } + + @Override + public boolean isJobRunning() { + return !isFinalStatus(); + } + + private boolean isFinalStatus() { + return getJobStatus().equals(JobStatus.STOPPED) || getJobStatus().equals(JobStatus.FINISHED); } @Override @@ -354,14 +364,14 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal(properties != null ? GsonUtils.GSON.toJson(properties) : FeConstants.null_string)); - if (offsetProvider != null && offsetProvider.getConsumedOffset() != null) { - trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getConsumedOffset())); + if (offsetProvider != null && StringUtils.isNotEmpty(offsetProvider.getShowCurrentOffset())) { + trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getShowCurrentOffset())); } else { trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); } - if (offsetProvider != null && offsetProvider.getMaxOffset() != null) { - trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getMaxOffset())); + if (offsetProvider != null && StringUtils.isNotEmpty(offsetProvider.getShowMaxOffset())) { + trow.addToColumnValue(new TCell().setStringVal(offsetProvider.getShowMaxOffset())); } else { trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index a1c8bce649d46a..0cd023b7d1d9ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -154,8 +154,6 @@ public TRow getTvfInfo(String jobName) { trow.addToColumnValue(new TCell().setStringVal(runningTask.getUserIdentity().getQualifiedUser())); } trow.addToColumnValue(new TCell().setStringVal("")); - trow.addToColumnValue(new TCell().setStringVal(runningTask.getRunningOffset() == null ? FeConstants.null_string - : runningTask.getRunningOffset().toString())); return trow; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index a27b66812c21c0..d2cf4e62053927 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -40,16 +40,16 @@ public interface SourceOffsetProvider { Offset getNextOffset(StreamingJobProperties jobProps, Map properties); /** - * Get consumered offset to show + * Get current offset to show * @return */ - String getConsumedOffset(); + String getShowCurrentOffset(); /** - * Get remote datasource max offset + * Get remote datasource max offset to show * @return */ - String getMaxOffset(); + String getShowMaxOffset(); /** * Rewrite the TVF parameters in the SQL based on the current offset. diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java index b152660873a4c4..6257eed8c70056 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3Offset.java @@ -28,8 +28,6 @@ @Getter @Setter public class S3Offset implements Offset { - // path/1.csv - String startFile; @SerializedName("endFile") String endFile; // s3://bucket/path/{1.csv,2.csv} @@ -47,6 +45,6 @@ public boolean isEmpty() { @Override public String toString() { - return "{ \"startFile\": \"" + startFile + "\", \"endFile\": \"" + endFile + "\" }"; + return "{\"endFile\": \"" + endFile + "\" }"; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java index e6a5cf809a523d..74743d093994db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/s3/S3SourceOffsetProvider.java @@ -33,6 +33,7 @@ import org.apache.doris.persist.gson.GsonUtils; import com.google.common.collect.Maps; +import com.google.gson.Gson; import lombok.extern.log4j.Log4j2; import org.apache.commons.lang3.StringUtils; @@ -72,7 +73,6 @@ public S3Offset getNextOffset(StreamingJobProperties jobProps, Map res = new HashMap<>(); + res.put("endFile", maxEndFile); + return new Gson().toJson(res); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java index ab03cce7c9195c..dde5f891efd7e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java @@ -24,7 +24,6 @@ import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.common.JobStatus; -import org.apache.doris.job.common.JobUtils; import org.apache.doris.job.common.TaskType; import org.apache.doris.job.disruptor.TaskDisruptor; import org.apache.doris.job.exception.JobException; @@ -110,7 +109,7 @@ private void batchSchedulerTimerJob() { } public void scheduleOneJob(T job) throws JobException { - if (!JobUtils.checkNeedSchedule(job)) { + if (!job.isJobRunning()) { return; } // not-schedule task @@ -145,7 +144,7 @@ public void scheduleOneJob(T job) throws JobException { } public void cycleTimerJobScheduler(T job) { - if (!JobUtils.checkNeedSchedule(job)) { + if (!job.isJobRunning()) { return; } if (!JobExecuteType.RECURRING.equals(job.getJobConfig().getExecuteType())) { @@ -226,7 +225,7 @@ private void executeTimerJobIdsWithinLastTenMinutesWindow() { clearEndJob(job); continue; } - if (JobUtils.checkNeedSchedule(job) && job.getJobConfig().checkIsTimerJob()) { + if (job.isJobRunning() && job.getJobConfig().checkIsTimerJob()) { cycleTimerJobScheduler(job, lastTimeWindowMs); } } diff --git a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy index 54fc5faa2e8950..6865cedd84241d 100644 --- a/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy +++ b/regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy @@ -97,10 +97,10 @@ suite("test_streaming_insert_job") { def jobOffset = sql """ - select ConsumedOffset, MaxOffset from jobs("type"="insert") where Name='${jobName}' + select currentOffset, endoffset from jobs("type"="insert") where Name='${jobName}' """ - assert jobOffset.get(0).get(0) == "regression/load/data/example_1.csv" - assert jobOffset.get(0).get(1) == "regression/load/data/example_1.csv" + assert jobOffset.get(0).get(0) == "{\"endFile\":\"regression/load/data/example_1.csv\"}"; + assert jobOffset.get(0).get(1) == "{\"endFile\":\"regression/load/data/example_1.csv\"}"; // alter streaming job sql """ @@ -123,21 +123,21 @@ suite("test_streaming_insert_job") { """ def alterJobProperties = sql """ - select status,properties,ConsumedOffset from jobs("type"="insert") where Name='${jobName}' + select status,properties,currentOffset from jobs("type"="insert") where Name='${jobName}' """ assert alterJobProperties.get(0).get(0) == "PAUSED" assert alterJobProperties.get(0).get(1) == "{\"s3.max_batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" - assert alterJobProperties.get(0).get(2) == "regression/load/data/example_1.csv" + assert alterJobProperties.get(0).get(2) == "{\"endFile\":\"regression/load/data/example_1.csv\"}"; sql """ RESUME JOB where jobname = '${jobName}' """ def resumeJobStatus = sql """ - select status,properties,ConsumedOffset from jobs("type"="insert") where Name='${jobName}' + select status,properties,currentOffset from jobs("type"="insert") where Name='${jobName}' """ assert resumeJobStatus.get(0).get(0) == "RUNNING" || resumeJobStatus.get(0).get(0) == "PENDING" assert resumeJobStatus.get(0).get(1) == "{\"s3.max_batch_files\":\"1\",\"session.insert_max_filter_ratio\":\"0.5\"}" - assert resumeJobStatus.get(0).get(2) == "regression/load/data/example_1.csv" + assert resumeJobStatus.get(0).get(2) == "{\"endFile\":\"regression/load/data/example_1.csv\"}"; Awaitility.await().atMost(60, SECONDS) .pollInterval(1, SECONDS).until( From 698fd519e5df4146e61acdd03a8097e959f812e3 Mon Sep 17 00:00:00 2001 From: laihui Date: Mon, 22 Sep 2025 20:53:37 +0800 Subject: [PATCH 25/25] fix format --- cloud/test/meta_service_job_test.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/cloud/test/meta_service_job_test.cpp b/cloud/test/meta_service_job_test.cpp index 5d5758579eb5f8..52359374032e7d 100644 --- a/cloud/test/meta_service_job_test.cpp +++ b/cloud/test/meta_service_job_test.cpp @@ -5008,7 +5008,8 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { txn_info->set_db_id(db_id); txn_info->set_label("streaming_ut_1"); txn_info->add_table_ids(1); - txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_load_job_source_type( + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); txn_info->set_timeout_ms(36000); meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK); @@ -5050,7 +5051,8 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { txn_info->set_db_id(db_id); txn_info->set_label("streaming_ut_2"); txn_info->add_table_ids(1); - txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_load_job_source_type( + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); txn_info->set_timeout_ms(36000); meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK); @@ -5092,7 +5094,8 @@ TEST(MetaServiceJobTest, UpdateStreamingJobMetaTest) { txn_info->set_db_id(db_id); txn_info->set_label("streaming_ut_missing_attach"); txn_info->add_table_ids(1); - txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_load_job_source_type( + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); txn_info->set_timeout_ms(36000); meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK); @@ -5129,7 +5132,8 @@ TEST(MetaServiceJobTest, GetStreamingTaskCommitAttachTest) { txn_info->set_db_id(db_id); txn_info->set_label("streaming_ut_3"); txn_info->add_table_ids(1); - txn_info->set_load_job_source_type(LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); + txn_info->set_load_job_source_type( + LoadJobSourceTypePB::LOAD_JOB_SRC_TYPE_STREAMING_JOB); txn_info->set_timeout_ms(36000); meta_service->begin_txn(&cntl_bt, &bt_req, &bt_res, nullptr); ASSERT_EQ(bt_res.status().code(), MetaServiceCode::OK);