diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 9775b3c96d5ed0..ee65d926e85bc2 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -2553,17 +2553,17 @@ resource_desc ::= create_job_stmt ::= KW_CREATE KW_JOB job_label:jobLabel KW_ON KW_SCHEDULER KW_EVERY INTEGER_LITERAL:time_interval ident:time_unit opt_job_starts:startsTime opt_job_ends:endsTime opt_comment:comment KW_DO stmt:executeSql {: - CreateJobStmt stmt = new CreateJobStmt(jobLabel,"RECURRING",null,time_interval,time_unit, startsTime, endsTime,comment,executeSql); + CreateJobStmt stmt = new CreateJobStmt(jobLabel,org.apache.doris.job.base.JobExecuteType.RECURRING,null,time_interval,time_unit, startsTime, endsTime,comment,executeSql); RESULT = stmt; :} - | KW_CREATE KW_JOB job_label:jobLabel KW_ON KW_SCHEDULER KW_STREAMING KW_AT STRING_LITERAL:atTime opt_comment:comment KW_DO stmt:executeSql +/* support in future | KW_CREATE KW_JOB job_label:jobLabel KW_ON KW_SCHEDULER KW_STREAMING KW_AT STRING_LITERAL:atTime opt_comment:comment KW_DO stmt:executeSql {: - CreateJobStmt stmt = new CreateJobStmt(jobLabel,"STREAMING",atTime,null,null,null,null,comment,executeSql); + CreateJobStmt stmt = new CreateJobStmt(jobLabel,org.apache.doris.job.base.JobExecuteType.STREAMING,atTime,null,null,null,null,comment,executeSql); RESULT = stmt; - :} + :} */ | KW_CREATE KW_JOB job_label:jobLabel KW_ON KW_SCHEDULER KW_AT STRING_LITERAL:atTime opt_comment:comment KW_DO stmt:executeSql {: - CreateJobStmt stmt = new CreateJobStmt(jobLabel,"ONE_TIME",atTime,null,null,null,null,comment,executeSql); + CreateJobStmt stmt = new CreateJobStmt(jobLabel,org.apache.doris.job.base.JobExecuteType.ONE_TIME,atTime,null,null,null,null,comment,executeSql); RESULT = stmt; :} ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java index 57f976712c9bd7..2c4819895f99dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateJobStmt.java @@ -23,14 +23,14 @@ import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; 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.IntervalUnit; +import org.apache.doris.job.common.JobStatus; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.scheduler.common.IntervalUnit; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.constants.JobStatus; -import org.apache.doris.scheduler.constants.JobType; -import org.apache.doris.scheduler.executor.SqlJobExecutor; -import org.apache.doris.scheduler.job.Job; import com.google.common.collect.ImmutableSet; import lombok.Getter; @@ -66,7 +66,7 @@ public class CreateJobStmt extends DdlStmt { private StatementBase stmt; @Getter - private Job job; + private AbstractJob job; private final LabelName labelName; @@ -88,9 +88,9 @@ public class CreateJobStmt extends DdlStmt { = new ImmutableSet.Builder>().add(InsertStmt.class) .add(UpdateStmt.class).build(); - private static HashSet supportStmtClassNamesCache = new HashSet<>(16); + private static final HashSet supportStmtClassNamesCache = new HashSet<>(16); - public CreateJobStmt(LabelName labelName, String jobTypeName, String onceJobStartTimestamp, + public CreateJobStmt(LabelName labelName, JobExecuteType executeType, String onceJobStartTimestamp, Long interval, String intervalTimeUnit, String startsTimeStamp, String endsTimeStamp, String comment, StatementBase doStmt) { this.labelName = labelName; @@ -101,9 +101,28 @@ public CreateJobStmt(LabelName labelName, String jobTypeName, String onceJobStar this.endsTimeStamp = endsTimeStamp; this.comment = comment; this.stmt = doStmt; - this.job = new Job(); - JobType jobType = JobType.valueOf(jobTypeName.toUpperCase()); - job.setJobType(jobType); + JobExecutionConfiguration jobExecutionConfiguration = new JobExecutionConfiguration(); + jobExecutionConfiguration.setExecuteType(executeType); + + TimerDefinition timerDefinition = new TimerDefinition(); + + if (null != onceJobStartTimestamp) { + timerDefinition.setStartTimeMs(TimeUtils.timeStringToLong(onceJobStartTimestamp)); + } + if (null != interval) { + timerDefinition.setInterval(interval); + } + if (null != intervalTimeUnit) { + timerDefinition.setIntervalUnit(IntervalUnit.valueOf(intervalTimeUnit.toUpperCase())); + } + if (null != startsTimeStamp) { + timerDefinition.setStartTimeMs(TimeUtils.timeStringToLong(startsTimeStamp)); + } + if (null != endsTimeStamp) { + timerDefinition.setEndTimeMs(TimeUtils.timeStringToLong(endsTimeStamp)); + } + jobExecutionConfiguration.setTimerDefinition(timerDefinition); + job.setComment(comment); } private String parseExecuteSql(String sql) throws AnalysisException { @@ -123,23 +142,13 @@ public void analyze(Analyzer analyzer) throws UserException { labelName.analyze(analyzer); String dbName = labelName.getDbName(); Env.getCurrentInternalCatalog().getDbOrAnalysisException(dbName); - job.setDbName(labelName.getDbName()); + job.setCurrentDbName(labelName.getDbName()); job.setJobName(labelName.getLabelName()); - if (StringUtils.isNotBlank(onceJobStartTimestamp)) { - analyzerOnceTimeJob(); - } else { - analyzerCycleJob(); - } - if (ConnectContext.get() != null) { - timezone = ConnectContext.get().getSessionVariable().getTimeZone(); - } - timezone = TimeUtils.checkTimeZoneValidAndStandardize(timezone); - job.setTimezone(timezone); + job.checkJobParams(); job.setComment(comment); //todo support user define - job.setUser(ConnectContext.get().getQualifiedUser()); + job.setCurrentUser(ConnectContext.get().getQualifiedUser()); job.setJobStatus(JobStatus.RUNNING); - job.setJobCategory(JobCategory.SQL); analyzerSqlStmt(); } @@ -165,61 +174,8 @@ private void checkStmtSupport() throws AnalysisException { private void analyzerSqlStmt() throws UserException { checkStmtSupport(); stmt.analyze(analyzer); - String originStmt = getOrigStmt().originStmt; - String executeSql = parseExecuteSql(originStmt); - SqlJobExecutor sqlJobExecutor = new SqlJobExecutor(executeSql); - job.setExecutor(sqlJobExecutor); + //String originStmt = getOrigStmt().originStmt; + //String executeSql = parseExecuteSql(originStmt); } - - private void analyzerCycleJob() throws UserException { - if (null == interval) { - throw new AnalysisException("interval is null"); - } - if (interval <= 0) { - throw new AnalysisException("interval must be greater than 0"); - } - - if (StringUtils.isBlank(intervalTimeUnit)) { - throw new AnalysisException("intervalTimeUnit is null"); - } - try { - IntervalUnit intervalUnit = IntervalUnit.valueOf(intervalTimeUnit.toUpperCase()); - job.setIntervalUnit(intervalUnit); - long intervalTimeMs = intervalUnit.getParameterValue(interval); - job.setIntervalMs(intervalTimeMs); - job.setOriginInterval(interval); - } catch (IllegalArgumentException e) { - throw new AnalysisException("interval time unit is not valid, we only support second,minute,hour,day,week"); - } - if (StringUtils.isNotBlank(startsTimeStamp)) { - long startsTimeMillis = TimeUtils.timeStringToLong(startsTimeStamp); - if (startsTimeMillis < System.currentTimeMillis()) { - throw new AnalysisException("starts time must be greater than current time"); - } - job.setStartTimeMs(startsTimeMillis); - } - if (StringUtils.isNotBlank(endsTimeStamp)) { - long endTimeMillis = TimeUtils.timeStringToLong(endsTimeStamp); - if (endTimeMillis < System.currentTimeMillis()) { - throw new AnalysisException("ends time must be greater than current time"); - } - job.setEndTimeMs(endTimeMillis); - } - if (job.getStartTimeMs() > 0 && job.getEndTimeMs() > 0 - && (job.getEndTimeMs() - job.getStartTimeMs() < job.getIntervalMs())) { - throw new AnalysisException("ends time must be greater than start time and interval time"); - } - } - - - private void analyzerOnceTimeJob() throws UserException { - job.setIntervalMs(0L); - - long executeAtTimeMillis = TimeUtils.timeStringToLong(onceJobStartTimestamp); - if (executeAtTimeMillis < System.currentTimeMillis()) { - throw new AnalysisException("job time stamp must be greater than current time"); - } - job.setStartTimeMs(executeAtTimeMillis); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java index 42fb1c508fc608..85dd5c7f658d0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobStmt.java @@ -26,12 +26,10 @@ import org.apache.doris.common.UserException; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -import org.apache.doris.scheduler.constants.JobCategory; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import lombok.Getter; -import org.apache.commons.lang3.StringUtils; import java.util.List; @@ -65,9 +63,6 @@ public class ShowJobStmt extends ShowStmt { @Getter private String dbFullName; // optional - @Getter - private JobCategory jobCategory; // optional - private String jobCategoryName; // optional @Getter @@ -86,11 +81,6 @@ public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); checkAuth(); checkLabelName(analyzer); - if (StringUtils.isBlank(jobCategoryName)) { - this.jobCategory = JobCategory.SQL; - } else { - this.jobCategory = JobCategory.valueOf(jobCategoryName.toUpperCase()); - } } private void checkAuth() throws AnalysisException { @@ -122,9 +112,6 @@ public ShowResultSetMetaData getMetaData() { ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); for (String title : TITLE_NAMES) { - if (this.jobCategory.equals(JobCategory.MTMV) && title.equals(NAME_TITLE)) { - builder.addColumn(new Column(MTMV_NAME_TITLE, ScalarType.createVarchar(30))); - } builder.addColumn(new Column(title, ScalarType.createVarchar(30))); } return builder.build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java index db3fb2ef3cdd6e..8d5c2b61db2fbf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowJobTaskStmt.java @@ -25,12 +25,10 @@ import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; import org.apache.doris.qe.ShowResultSetMetaData; -import org.apache.doris.scheduler.constants.JobCategory; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import lombok.Getter; -import org.apache.commons.lang3.StringUtils; import java.util.List; @@ -57,8 +55,6 @@ public class ShowJobTaskStmt extends ShowStmt { @Getter private final LabelName labelName; - @Getter - private JobCategory jobCategory; // optional @Getter private String dbFullName; // optional @@ -67,12 +63,6 @@ public class ShowJobTaskStmt extends ShowStmt { public ShowJobTaskStmt(String category, LabelName labelName) { this.labelName = labelName; - String jobCategoryName = category; - if (StringUtils.isBlank(jobCategoryName)) { - this.jobCategory = JobCategory.SQL; - } else { - this.jobCategory = JobCategory.valueOf(jobCategoryName.toUpperCase()); - } } @Override @@ -114,9 +104,6 @@ public ShowResultSetMetaData getMetaData() { @Override public RedirectStatus getRedirectStatus() { - if (jobCategory.isPersistent()) { - return RedirectStatus.FORWARD_NO_SYNC; - } return RedirectStatus.NO_FORWARD; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 8c028bc8fc9199..1ed329134599a1 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -147,6 +147,8 @@ import org.apache.doris.httpv2.entity.ResponseBody; import org.apache.doris.httpv2.meta.MetaBaseAction; import org.apache.doris.httpv2.rest.RestApiStatusCode; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.manager.JobManager; import org.apache.doris.journal.JournalCursor; import org.apache.doris.journal.JournalEntity; import org.apache.doris.journal.bdbje.Timestamp; @@ -217,13 +219,8 @@ import org.apache.doris.qe.VariableMgr; import org.apache.doris.resource.Tag; import org.apache.doris.resource.workloadgroup.WorkloadGroupMgr; -import org.apache.doris.scheduler.disruptor.TaskDisruptor; -import org.apache.doris.scheduler.manager.JobTaskManager; -import org.apache.doris.scheduler.manager.TimerJobManager; import org.apache.doris.scheduler.manager.TransientTaskManager; import org.apache.doris.scheduler.registry.ExportTaskRegister; -import org.apache.doris.scheduler.registry.PersistentJobRegister; -import org.apache.doris.scheduler.registry.TimerJobRegister; import org.apache.doris.service.ExecuteEnv; import org.apache.doris.service.FrontendOptions; import org.apache.doris.statistics.AnalysisManager; @@ -349,13 +346,10 @@ public class Env { private CooldownConfHandler cooldownConfHandler; private MetastoreEventsProcessor metastoreEventsProcessor; - private PersistentJobRegister persistentJobRegister; private ExportTaskRegister exportTaskRegister; - private TimerJobManager timerJobManager; + private JobManager jobManager; private TransientTaskManager transientTaskManager; - private JobTaskManager jobTaskManager; - private TaskDisruptor taskDisruptor; private MasterDaemon labelCleaner; // To clean old LabelInfo, ExportJobInfos private MasterDaemon txnCleaner; // To clean aborted or timeout txns private Daemon feDiskUpdater; // Update fe disk info @@ -625,13 +619,8 @@ private Env(boolean isCheckpointCatalog) { this.cooldownConfHandler = new CooldownConfHandler(); } this.metastoreEventsProcessor = new MetastoreEventsProcessor(); - this.jobTaskManager = new JobTaskManager(); - this.timerJobManager = new TimerJobManager(); + this.jobManager = new JobManager<>(); this.transientTaskManager = new TransientTaskManager(); - this.taskDisruptor = new TaskDisruptor(this.timerJobManager, this.transientTaskManager); - this.timerJobManager.setDisruptor(taskDisruptor); - this.transientTaskManager.setDisruptor(taskDisruptor); - this.persistentJobRegister = new TimerJobRegister(timerJobManager); this.exportTaskRegister = new ExportTaskRegister(transientTaskManager); this.replayedJournalId = new AtomicLong(0L); this.stmtIdCounter = new AtomicLong(0L); @@ -1527,8 +1516,7 @@ private void startMasterOnlyDaemonThreads() { publishVersionDaemon.start(); // Start txn cleaner txnCleaner.start(); - taskDisruptor.start(); - timerJobManager.start(); + jobManager.start(); // Alter getAlterInstance().start(); // Consistency checker @@ -2008,29 +1996,17 @@ public long loadLoadJobsV2(DataInputStream in, long checksum) throws IOException } public long loadAsyncJobManager(DataInputStream in, long checksum) throws IOException { - timerJobManager.readFields(in); + jobManager.readFields(in); LOG.info("finished replay asyncJobMgr from image"); return checksum; } public long saveAsyncJobManager(CountingDataOutputStream out, long checksum) throws IOException { - timerJobManager.write(out); + jobManager.write(out); LOG.info("finished save analysisMgr to image"); return checksum; } - public long loadJobTaskManager(DataInputStream in, long checksum) throws IOException { - jobTaskManager.readFields(in); - LOG.info("finished replay jobTaskMgr from image"); - return checksum; - } - - public long saveJobTaskManager(CountingDataOutputStream out, long checksum) throws IOException { - jobTaskManager.write(out); - LOG.info("finished save jobTaskMgr to image"); - return checksum; - } - public long loadResources(DataInputStream in, long checksum) throws IOException { resourceMgr = ResourceMgr.read(in); LOG.info("finished replay resources from image"); @@ -3818,26 +3794,19 @@ public SyncJobManager getSyncJobManager() { return this.syncJobManager; } - public PersistentJobRegister getJobRegister() { - return persistentJobRegister; - } public ExportTaskRegister getExportTaskRegister() { return exportTaskRegister; } - public TimerJobManager getAsyncJobManager() { - return timerJobManager; + public JobManager getJobManager() { + return jobManager; } public TransientTaskManager getTransientTaskManager() { return transientTaskManager; } - public JobTaskManager getJobTaskManager() { - return jobTaskManager; - } - public SmallFileMgr getSmallFileMgr() { return this.smallFileMgr; } 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 new file mode 100644 index 00000000000000..233bdd2145595c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java @@ -0,0 +1,114 @@ +// 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.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.job.task.Task; +import org.apache.doris.persist.gson.GsonUtils; + +import lombok.Data; +import lombok.Getter; +import org.apache.commons.collections.CollectionUtils; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +@Data +public abstract class AbstractJob implements Job, Writable { + + @Getter + private Long jobId; + + private String jobName; + + private JobStatus jobStatus; + + private String currentDbName; + + private String comment; + + private String currentUser; + + private JobExecutionConfiguration jobConfig; + + private List runningTasks = new ArrayList<>(); + + @Override + public void cancel() throws JobException { + if (CollectionUtils.isEmpty(runningTasks)) { + return; + } + runningTasks.forEach(Task::cancel); + } + + public void checkJobParams() { + if (null == jobConfig) { + throw new IllegalArgumentException("jobConfig cannot be null"); + } + jobConfig.checkParams(); + checkJobParamsInternal(); + } + + public void updateJobStatus(JobStatus newJobStatus) { + if (null == newJobStatus) { + throw new IllegalArgumentException("jobStatus cannot be null"); + } + if (jobStatus == newJobStatus) { + throw new IllegalArgumentException(String.format("Can't update job %s status to the %s status", + jobStatus.name(), this.jobStatus.name())); + } + // check other status + } + + public void resumeJob() { + if (jobStatus != JobStatus.PAUSED) { + throw new IllegalArgumentException(String.format("Can't resume job %s status to the %s status", + jobStatus.name(), this.jobStatus.name())); + } + jobStatus = JobStatus.RUNNING; + } + + protected abstract void checkJobParamsInternal(); + + @Override + public void write(DataOutput out) throws IOException { + String jobData = GsonUtils.GSON.toJson(this); + Text.writeString(out, jobData); + } + + public static AbstractJob readFields(DataInput in) throws IOException { + return GsonUtils.GSON.fromJson(Text.readString(in), AbstractJob.class); + } + + @Override + public void onTaskFail(long taskId) { + // AbstractTask task=runningTasks.stream().findFirst(); + } + + @Override + public void onTaskSuccess(long taskId) { + + } +} 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 new file mode 100644 index 00000000000000..7895eb8d8b16d4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/Job.java @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.base; + +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.qe.ShowResultSetMetaData; + +import java.util.List; + +/** + * Job is the core of the scheduler module, which is used to store the Job information of the job module. + * We can use the job to uniquely identify a Job. + * The jobName is used to identify the job, which is not unique. + * The jobStatus is used to identify the status of the Job, which is used to control the execution of the + * job. + */ +public interface Job { + + List createTasks(); + + void cancel() throws JobException; + + boolean isReadyForScheduling(); + + + ShowResultSetMetaData getJobMetaData(); + + ShowResultSetMetaData getTaskMetaData(); + + JobType getJobType(); + + List queryTasks(); + + void onTaskFail(long taskId); + + void onTaskSuccess(long taskId); + + void onTaskCancel(long taskId); + + void afterTaskRun(long taskId); + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobType.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecuteType.java similarity index 88% rename from fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobType.java rename to fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecuteType.java index 58f681c40614c0..ea9ddb3b0203d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecuteType.java @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.scheduler.constants; +package org.apache.doris.job.base; + +public enum JobExecuteType { -public enum JobType { /** * The job will be executed only once. */ @@ -34,5 +35,9 @@ public enum JobType { /** * The job will be executed manually and need to be triggered by the user. */ - MANUAL + MANUAL, + /** + * The job will be executed immediately. + */ + INSTANT, } 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 new file mode 100644 index 00000000000000..0f40412fc58dd8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/JobExecutionConfiguration.java @@ -0,0 +1,208 @@ +// 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.util.TimeUtils; + +import lombok.Data; + +import java.util.ArrayList; +import java.util.List; + + +@Data +public class JobExecutionConfiguration { + + private TimerDefinition timerDefinition; + private JobExecuteType executeType; + + /** + * Maximum number of concurrent tasks, <= 0 means no limit + * if the number of tasks exceeds the limit, the task will be delayed execution + */ + private Integer maxConcurrentTaskNum = -1; + + public void checkParams() { + if (executeType == null) { + throw new IllegalArgumentException("executeType cannot be null"); + } + + if (executeType == JobExecuteType.INSTANT || executeType == JobExecuteType.MANUAL) { + return; + } + + checkTimerDefinition(); + + if (executeType == JobExecuteType.ONE_TIME) { + validateStartTimeMs(); + return; + } + + if (executeType == JobExecuteType.STREAMING) { + validateStartTimeMs(); + return; + } + + if (executeType == JobExecuteType.RECURRING) { + if (timerDefinition.getInterval() == null) { + throw new IllegalArgumentException("interval cannot be null when executeType is RECURRING"); + } + if (timerDefinition.getIntervalUnit() == null) { + throw new IllegalArgumentException("intervalUnit cannot be null when executeType is RECURRING"); + } + validateStartTimeMs(); + if (timerDefinition.getEndTimeMs() != null && timerDefinition.getEndTimeMs() < System.currentTimeMillis()) { + throw new IllegalArgumentException("endTimeMs cannot be less than current time"); + } + if (timerDefinition.getStartTimeMs() != null && timerDefinition.getEndTimeMs() != null + && timerDefinition.getStartTimeMs() > timerDefinition.getEndTimeMs()) { + throw new IllegalArgumentException("startTimeMs cannot be greater than endTimeMs"); + } + } + } + + private void checkTimerDefinition() { + if (timerDefinition == null) { + throw new IllegalArgumentException( + "timerDefinition cannot be null when executeType is not instant or manual"); + } + } + + private void validateStartTimeMs() { + if (timerDefinition.getStartTimeMs() == null) { + throw new IllegalArgumentException("startTimeMs cannot be null"); + } + if (timerDefinition.getStartTimeMs() < System.currentTimeMillis()) { + throw new IllegalArgumentException("startTimeMs cannot be less than current time"); + } + } + + + // Returns a list of delay times in seconds for triggering the job + public List getTriggerDelayTimes(Long currentTimeMs, Long startTimeMs, Long endTimeMs) { + List delayTimeSeconds = new ArrayList<>(); + + if (JobExecuteType.ONE_TIME.equals(executeType)) { + // If the job is already executed or in the schedule queue, or not within this schedule window + if (null != timerDefinition.getLatestSchedulerTimeMs() || endTimeMs < timerDefinition.getStartTimeMs()) { + return delayTimeSeconds; + } + + delayTimeSeconds.add(queryDelayTimeSecond(currentTimeMs, timerDefinition.getStartTimeMs())); + this.timerDefinition.setLatestSchedulerTimeMs(timerDefinition.getStartTimeMs()); + 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 (timerDefinition.getStartTimeMs() > endTimeMs || null != timerDefinition.getEndTimeMs() + && timerDefinition.getEndTimeMs() < startTimeMs) { + return delayTimeSeconds; + } + + return getExecutionDelaySeconds(startTimeMs, endTimeMs, timerDefinition.getStartTimeMs(), + timerDefinition.getIntervalUnit().getIntervalMs(timerDefinition.getInterval()), currentTimeMs); + } + + return delayTimeSeconds; + } + + // Returns the delay time in seconds between the current time and the specified start time + private Long queryDelayTimeSecond(Long currentTimeMs, Long startTimeMs) { + if (startTimeMs <= currentTimeMs) { + return 0L; + } + + return (startTimeMs - currentTimeMs) / 1000; + } + + // Returns a list of delay times in seconds for executing the job within the specified window + private List getExecutionDelaySeconds(long windowStartTimeMs, long windowEndTimeMs, long startTimeMs, + long intervalMs, long currentTimeMs) { + List timestamps = new ArrayList<>(); + + long windowDuration = windowEndTimeMs - windowStartTimeMs; + + if (windowDuration <= 0 || intervalMs <= 0) { + return timestamps; // Return an empty list if there won't be any trigger time + } + + long firstTriggerTime = windowStartTimeMs + (intervalMs - ((windowStartTimeMs - startTimeMs) + % intervalMs)) % intervalMs; + + if (firstTriggerTime < currentTimeMs) { + firstTriggerTime += intervalMs; + } + + if (firstTriggerTime > windowEndTimeMs) { + return timestamps; // Return an empty list if there won't be any trigger time + } + + // Calculate the trigger time list + for (long triggerTime = firstTriggerTime; triggerTime <= windowEndTimeMs; triggerTime += intervalMs) { + if (triggerTime >= currentTimeMs) { + timestamps.add(queryDelayTimeSecond(currentTimeMs, triggerTime)); + } + } + + return timestamps; + } + + private String convertRecurringStrategyToString() { + switch (executeType) { + case ONE_TIME: + return "AT " + TimeUtils.longToTimeString(timerDefinition.getStartTimeMs()); + case RECURRING: + String result = "EVERY " + timerDefinition.getInterval() + " " + + timerDefinition.getIntervalUnit().name() + " STARTS " + + TimeUtils.longToTimeString(timerDefinition.getStartTimeMs()); + + if (null != timerDefinition.getEndTimeMs()) { + result += " ENDS " + TimeUtils.longToTimeString(timerDefinition.getEndTimeMs()); + } + return result; + /* case STREAMING: + return "STREAMING" + (startTimeMs > 0 ? " AT " + TimeUtils.longToTimeString(startTimeMs) : "");*/ + case MANUAL: + return "MANUAL TRIGGER"; + case INSTANT: + return "INSTANT"; + default: + return "UNKNOWN"; + } + } + + public boolean checkIsTimerJob() { + return null != timerDefinition; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/base/TimerDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/job/base/TimerDefinition.java new file mode 100644 index 00000000000000..25db90ba850dc5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/base/TimerDefinition.java @@ -0,0 +1,60 @@ +// 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.job.common.IntervalUnit; + +import lombok.Data; + +@Data +public class TimerDefinition { + + private Long interval; + + private IntervalUnit intervalUnit; + + private Long startTimeMs; + + private Long endTimeMs; + + private Long createTimeMs; + + private Long latestSchedulerTimeMs; + + + public void checkParams() { + if (null == startTimeMs) { + startTimeMs = createTimeMs + intervalUnit.getIntervalMs(interval); + } + if (startTimeMs < System.currentTimeMillis()) { + throw new IllegalArgumentException("startTimeMs must be greater than current time"); + } + if (null != endTimeMs && endTimeMs < startTimeMs) { + throw new IllegalArgumentException("end time cannot be less than start time"); + } + + if (null != intervalUnit) { + if (null == interval) { + throw new IllegalArgumentException("interval cannot be null when intervalUnit is not null"); + } + if (interval <= 0) { + throw new IllegalArgumentException("interval must be greater than 0"); + } + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/common/IntervalUnit.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/IntervalUnit.java similarity index 92% rename from fe/fe-core/src/main/java/org/apache/doris/scheduler/common/IntervalUnit.java rename to fe/fe-core/src/main/java/org/apache/doris/job/common/IntervalUnit.java index 27f10d8a3f6200..4c576e986fa8e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/common/IntervalUnit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/IntervalUnit.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.scheduler.common; +package org.apache.doris.job.common; import java.util.Arrays; import java.util.concurrent.TimeUnit; @@ -60,7 +60,7 @@ IntervalUnit getByName(String name) { .orElseThrow(() -> new IllegalArgumentException("Unknown configuration " + name)); } - public Long getParameterValue(Long param) { - return (Long) (param != null ? converter.apply(param) : defaultValue); + public Long getIntervalMs(Long interval) { + return (Long) (interval != null ? converter.apply(interval) : defaultValue); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobStatus.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java similarity index 96% rename from fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobStatus.java rename to fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java index f01686a521611c..2df65e4654d414 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobStatus.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobStatus.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.scheduler.constants; +package org.apache.doris.job.common; public enum JobStatus { @@ -42,3 +42,4 @@ public enum JobStatus { */ FINISHED } + diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/JobType.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobType.java new file mode 100644 index 00000000000000..f815e16cecd80f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/JobType.java @@ -0,0 +1,23 @@ +// 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; + +public enum JobType { + INSERT, + MTMV +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskStatus.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskStatus.java new file mode 100644 index 00000000000000..fefdd1eec26e0e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskStatus.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.common; + +public enum TaskStatus { + CANCEL, + RUNNING, + SUCCESS, + FAILD; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/ExecutorResult.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskType.java similarity index 74% rename from fe/fe-core/src/main/java/org/apache/doris/scheduler/job/ExecutorResult.java rename to fe/fe-core/src/main/java/org/apache/doris/job/common/TaskType.java index 99df9c9e787a07..0aab448674ad00 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/ExecutorResult.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/TaskType.java @@ -15,21 +15,11 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.scheduler.job; +package org.apache.doris.job.common; -import lombok.AllArgsConstructor; -import lombok.Data; - -@Data -@AllArgsConstructor -public class ExecutorResult { - - private T result; - - private boolean success; - - private String errorMsg; - - private String executorSql; +public enum TaskType { + SCHDULER, + MANUAL, + INSTANT, } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/ExecuteTaskEvent.java b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/ExecuteTaskEvent.java new file mode 100644 index 00000000000000..3d8f9ed15349ae --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/ExecuteTaskEvent.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.disruptor; + +import org.apache.doris.job.base.JobExecutionConfiguration; +import org.apache.doris.job.task.AbstractTask; + +import com.lmax.disruptor.EventFactory; +import lombok.Data; + +@Data +public class ExecuteTaskEvent { + + private T task; + + private JobExecutionConfiguration jobConfig; + + public static EventFactory> factory() { + return ExecuteTaskEvent::new; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TaskDisruptor.java b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TaskDisruptor.java new file mode 100644 index 00000000000000..0994a1d9211e0c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TaskDisruptor.java @@ -0,0 +1,82 @@ +// 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.disruptor; + +import com.lmax.disruptor.EventFactory; +import com.lmax.disruptor.EventTranslatorVararg; +import com.lmax.disruptor.RingBuffer; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.WorkHandler; +import com.lmax.disruptor.dsl.Disruptor; +import com.lmax.disruptor.dsl.ProducerType; + +import java.util.concurrent.ThreadFactory; + +/** + * Utility class for creating and managing a Disruptor instance. + * + * @param the type of the event handled by the Disruptor + */ +public class TaskDisruptor { + private final Disruptor disruptor; + private final EventTranslatorVararg eventTranslator; + + /** + * Constructs a DisruptorUtil instance. + * + * @param eventFactory the factory for creating events + * @param ringBufferSize the size of the ring buffer + * @param threadFactory the thread factory to create threads for event handling + * @param waitStrategy the wait strategy for the ring buffer + * @param workHandlers the work handlers for processing events + * @param eventTranslator the translator for publishing events with variable arguments + */ + public TaskDisruptor(EventFactory eventFactory, int ringBufferSize, ThreadFactory threadFactory, + WaitStrategy waitStrategy, WorkHandler[] workHandlers, + EventTranslatorVararg eventTranslator) { + disruptor = new Disruptor<>(eventFactory, ringBufferSize, threadFactory, + ProducerType.SINGLE, waitStrategy); + disruptor.handleEventsWithWorkerPool(workHandlers); + this.eventTranslator = eventTranslator; + disruptor.start(); + } + + /** + * Starts the Disruptor. + */ + public void start() { + disruptor.start(); + } + + /** + * Publishes an event with the provided arguments. + * + * @param args the arguments for the event + */ + public void publishEvent(Object... args) { + RingBuffer ringBuffer = disruptor.getRingBuffer(); + ringBuffer.publishEvent(eventTranslator, args); + } + + /** + * Shuts down the Disruptor. + */ + public void shutdown() { + disruptor.shutdown(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TimerJobEvent.java b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TimerJobEvent.java new file mode 100644 index 00000000000000..65654c225feafb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/disruptor/TimerJobEvent.java @@ -0,0 +1,35 @@ +// 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.disruptor; + +import org.apache.doris.job.base.AbstractJob; + +import com.lmax.disruptor.EventFactory; +import lombok.Data; + +@Data +public class TimerJobEvent { + + + private T job; + + public static EventFactory> factory() { + return TimerJobEvent::new; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/exception/JobException.java b/fe/fe-core/src/main/java/org/apache/doris/job/exception/JobException.java new file mode 100644 index 00000000000000..d6e93ead625dfd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/exception/JobException.java @@ -0,0 +1,39 @@ +// 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.exception; + +/** + * This class represents a job exception that can be thrown when a job is executed. + */ +public class JobException extends Exception { + public JobException(String message) { + super(message); + } + + public JobException(String format, Object... msg) { + super(String.format(format, msg)); + } + + public JobException(String message, Throwable cause) { + super(message, cause); + } + + public JobException(Throwable cause) { + super(cause); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/executor/DefaultTaskExecutorHandler.java b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DefaultTaskExecutorHandler.java new file mode 100644 index 00000000000000..c8672a465d5ac5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DefaultTaskExecutorHandler.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.executor; + +import org.apache.doris.job.disruptor.ExecuteTaskEvent; +import org.apache.doris.job.manager.TaskTokenManager; +import org.apache.doris.job.task.AbstractTask; + +import com.lmax.disruptor.WorkHandler; +import lombok.extern.slf4j.Slf4j; + +import java.util.concurrent.Semaphore; + +/** + * DefaultTaskExecutor is an implementation of the TaskExecutor interface. + * if you need to implement your own TaskExecutor, you could refer to this class. and need to register + * it in the TaskExecutorFactory + * It executes a given AbstractTask by acquiring a semaphore token from the TaskTokenManager + * and releasing it after the task execution. + */ +@Slf4j +public class DefaultTaskExecutorHandler implements WorkHandler> { + + + @Override + public void onEvent(ExecuteTaskEvent executeTaskEvent) { + T task = executeTaskEvent.getTask(); + if (null == task) { + log.info("task is null, ignore"); + return; + } + int maxConcurrentTaskNum = executeTaskEvent.getJobConfig().getMaxConcurrentTaskNum(); + if (maxConcurrentTaskNum <= 0) { + task.runTask(); + return; + } + + Semaphore semaphore = null; + // get token + try { + semaphore = TaskTokenManager.tryAcquire(task.getJobId(), maxConcurrentTaskNum); + task.runTask(); + } catch (Exception e) { + task.onFail(); + log.error("execute task error, task id is {}", task.getTaskId(), e); + } finally { + if (null != semaphore) { + semaphore.release(); + } + } + } +} 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 new file mode 100644 index 00000000000000..5949a02d1294f2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/DispatchTaskHandler.java @@ -0,0 +1,61 @@ +// 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.executor; + +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.disruptor.TaskDisruptor; +import org.apache.doris.job.disruptor.TimerJobEvent; +import org.apache.doris.job.task.AbstractTask; + +import com.lmax.disruptor.WorkHandler; +import jline.internal.Log; +import lombok.extern.slf4j.Slf4j; + +import java.util.List; +import java.util.Map; + +@Slf4j +public class DispatchTaskHandler> implements WorkHandler> { + + private final Map> disruptorMap; + + public DispatchTaskHandler(Map> disruptorMap) { + this.disruptorMap = disruptorMap; + } + + + @Override + public void onEvent(TimerJobEvent event) throws Exception { + try { + if (null == event.getJob()) { + log.info("job is null,may be job is deleted, ignore"); + return; + } + if (event.getJob().isReadyForScheduling()) { + List tasks = event.getJob().createTasks(); + JobType jobType = event.getJob().getJobType(); + for (AbstractTask task : tasks) { + disruptorMap.get(jobType).publishEvent(task); + } + } + } catch (Exception e) { + Log.warn("dispatch timer job error, task id is {}", event.getJob().getJobId(), e); + } + } +} 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 new file mode 100644 index 00000000000000..3999ab3a983253 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/executor/TimerJobSchedulerTask.java @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.executor; + +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.disruptor.TaskDisruptor; + +import io.netty.util.Timeout; +import io.netty.util.TimerTask; +import jline.internal.Log; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TimerJobSchedulerTask implements TimerTask { + + private TaskDisruptor dispatchDisruptor; + + private final T job; + + public TimerJobSchedulerTask(TaskDisruptor dispatchDisruptor, T job) { + this.dispatchDisruptor = dispatchDisruptor; + this.job = job; + } + + @Override + public void run(Timeout timeout) { + try { + dispatchDisruptor.publishEvent(this.job); + } catch (Exception e) { + Log.warn("dispatch timer job error, task id is {}", this.job.getJobId(), e); + } + } +} 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 new file mode 100644 index 00000000000000..a79cc8c43f3e6e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java @@ -0,0 +1,101 @@ +// 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; + +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.qe.ShowResultSetMetaData; + +import lombok.Data; + +import java.util.Arrays; +import java.util.List; + +@Data +public class InsertJob extends AbstractJob { + + String labelPrefix; + + @Override + public List createTasks() { + InsertTask task = new InsertTask(); + task.setJobId(getJobId()); + List tasks = Arrays.asList(task); + getRunningTasks().add(task); + return tasks; + } + + + @Override + public void cancel() throws JobException { + + } + + @Override + public boolean isReadyForScheduling() { + return true; + } + + + @Override + protected void checkJobParamsInternal() { + + } + + + @Override + public List queryTasks() { + return null; + } + + @Override + public JobType getJobType() { + return JobType.INSERT; + } + + @Override + public ShowResultSetMetaData getJobMetaData() { + return null; + } + + @Override + public ShowResultSetMetaData getTaskMetaData() { + return null; + } + + @Override + public void onTaskFail(long taskId) { + + } + + @Override + public void onTaskSuccess(long taskId) { + + } + + @Override + public void onTaskCancel(long taskId) { + + } + + @Override + public void afterTaskRun(long taskId) { + + } +} 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 new file mode 100644 index 00000000000000..c448e9cd3128c5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertTask.java @@ -0,0 +1,66 @@ +// 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; + +import org.apache.doris.catalog.Env; +import org.apache.doris.job.base.Job; +import org.apache.doris.job.manager.test; +import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.qe.ShowResultSetMetaData; + +import lombok.extern.slf4j.Slf4j; + + +@Slf4j +public class InsertTask extends AbstractTask { + + //private AtomicInteger test = new AtomicInteger(0); + @Override + public void before() { + + } + + @Override + public void run() { + System.out.println("InsertTask run:" + test.A.incrementAndGet()); + } + + @Override + public void onFail() { + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + job.onTaskFail(getTaskId()); + } + + @Override + public void onSuccess() { + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + job.onTaskSuccess(getTaskId()); + } + + @Override + public void cancel() { + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + job.onTaskCancel(getTaskId()); + } + + @Override + public ShowResultSetMetaData getMetaData() { + 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 new file mode 100644 index 00000000000000..c4d5d92448fa9c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -0,0 +1,171 @@ +// 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.catalog.Env; +import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.scheduler.JobScheduler; +import org.apache.doris.job.task.AbstractTask; + +import lombok.extern.slf4j.Slf4j; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +@Slf4j +public class JobManager> implements Writable { + + + private final ConcurrentHashMap jobMap = new ConcurrentHashMap<>(32); + + private JobScheduler jobScheduler; + + public void start() { + jobScheduler = new JobScheduler(jobMap); + jobScheduler.start(); + } + + Long registerJob(T job) throws JobException { + job.checkJobParams(); + checkJobNameExist(job.getJobName(), job.getJobType()); + long id = Env.getCurrentEnv().getNextId(); + job.setJobId(id); + replayCreateJob(job); + //check name exist + jobMap.put(id, job); + //check its need to scheduler + jobScheduler.scheduleOneJob(job); + return id; + } + + + private void checkJobNameExist(String jobName, JobType type) throws JobException { + if (jobMap.values().stream().anyMatch(a -> a.getJobName().equals(jobName) && a.getJobType().equals(type))) { + throw new JobException("job name exist,jobName:" + jobName); + } + } + + void unregisterJob(Long jobId) throws JobException { + checkJobExist(jobId); + jobMap.get(jobId).setJobStatus(JobStatus.STOPPED); + jobMap.get(jobId).cancel(); + replayDeleteJob(jobMap.get(jobId)); + jobMap.remove(jobId); + } + + void alterJobStatus(Long jobId, JobStatus status) throws JobException { + checkJobExist(jobId); + jobMap.get(jobId).updateJobStatus(status); + } + + void resumeJob(Long jobId) throws JobException { + checkJobExist(jobId); + replayUpdateJob(jobMap.get(jobId)); + jobMap.get(jobId).resumeJob(); + jobScheduler.scheduleOneJob(jobMap.get(jobId)); + } + + private void checkJobExist(Long jobId) throws JobException { + if (null == jobMap.get(jobId)) { + throw new JobException("job not exist,jobId:" + jobId); + } + } + + List queryJobs(JobType type) { + return jobMap.values().stream().filter(a -> a.getJobType().equals(type)) + .collect(java.util.stream.Collectors.toList()); + } + + List queryTasks(Long jobId) throws JobException { + checkJobExist(jobId); + return jobMap.get(jobId).queryTasks(); + } + + public void replayCreateJob(T job) { + if (jobMap.containsKey(job.getJobId())) { + return; + } + jobMap.putIfAbsent(job.getJobId(), job); + jobScheduler.scheduleOneJob(job); + log.info(new LogBuilder(LogKey.SCHEDULER_JOB, job.getJobId()) + .add("msg", "replay create scheduler job").build()); + } + + /** + * Replay update load job. + **/ + public void replayUpdateJob(T job) { + jobMap.put(job.getJobId(), job); + if (JobStatus.RUNNING.equals(job.getJobStatus())) { + jobScheduler.scheduleOneJob(job); + } + log.info(new LogBuilder(LogKey.SCHEDULER_JOB, job.getJobId()) + .add("msg", "replay update scheduler job").build()); + } + + public void replayDeleteJob(T job) { + if (null == jobMap.get(job.getJobId())) { + return; + } + jobMap.remove(job.getJobId()); + log.info(new LogBuilder(LogKey.SCHEDULER_JOB, job.getJobId()) + .add("msg", "replay delete scheduler job").build()); + } + + void cancelTask(Long jobId, Long taskId) throws JobException { + checkJobExist(jobId); + if (null == jobMap.get(jobId).getRunningTasks()) { + throw new JobException("task not exist,taskId:" + taskId); + } + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(jobMap.size()); + for (AbstractJob job : jobMap.values()) { + job.write(out); + } + } + + /** + * read job from data input, and init job + * + * @param in data input + * @throws IOException io exception when read data input error + */ + public void readFields(DataInput in) throws IOException { + int size = in.readInt(); + for (int i = 0; i < size; i++) { + AbstractJob job = AbstractJob.readFields(in); + jobMap.putIfAbsent(job.getJobId(), (T) job); + } + } + + public T getJob(Long jobId) { + return jobMap.get(jobId); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java new file mode 100644 index 00000000000000..2f88b2936d92c7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskDisruptorGroupManager.java @@ -0,0 +1,91 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.manager; + +import org.apache.doris.common.CustomThreadFactory; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.common.JobType; +import org.apache.doris.job.disruptor.ExecuteTaskEvent; +import org.apache.doris.job.disruptor.TaskDisruptor; +import org.apache.doris.job.disruptor.TimerJobEvent; +import org.apache.doris.job.executor.DefaultTaskExecutorHandler; +import org.apache.doris.job.executor.DispatchTaskHandler; +import org.apache.doris.job.extensions.insert.InsertTask; +import org.apache.doris.job.task.AbstractTask; + +import com.lmax.disruptor.BlockingWaitStrategy; +import com.lmax.disruptor.EventFactory; +import com.lmax.disruptor.EventTranslatorVararg; +import com.lmax.disruptor.WorkHandler; +import lombok.Getter; + +import java.util.EnumMap; +import java.util.Map; +import java.util.concurrent.ThreadFactory; + +public class TaskDisruptorGroupManager { + + private final Map> disruptorMap = new EnumMap<>(JobType.class); + + @Getter + private TaskDisruptor>> dispatchDisruptor; + + + public void init() { + registerInsertDisruptor(); + //when all task queue is ready, dispatch task to registered task executor + registerDispatchDisruptor(); + } + + private void registerDispatchDisruptor() { + EventFactory> dispatchEventFactory = TimerJobEvent.factory(); + ThreadFactory dispatchThreadFactory = new CustomThreadFactory("dispatch-task"); + WorkHandler[] dispatchTaskExecutorHandlers = new WorkHandler[5]; + for (int i = 0; i < 5; i++) { + dispatchTaskExecutorHandlers[i] = new DispatchTaskHandler(this.disruptorMap); + } + EventTranslatorVararg> eventTranslator = + (event, sequence, args) -> event.setJob((AbstractJob) args[0]); + this.dispatchDisruptor = new TaskDisruptor<>(dispatchEventFactory, 1024, dispatchThreadFactory, + new BlockingWaitStrategy(), dispatchTaskExecutorHandlers, eventTranslator); + } + + private void registerInsertDisruptor() { + EventFactory> insertEventFactory = ExecuteTaskEvent.factory(); + ThreadFactory insertTaskThreadFactory = new CustomThreadFactory("insert-task-execute"); + WorkHandler[] insertTaskExecutorHandlers = new WorkHandler[5]; + for (int i = 0; i < 5; i++) { + insertTaskExecutorHandlers[i] = new DefaultTaskExecutorHandler(); + } + EventTranslatorVararg> eventTranslator = + (event, sequence, args) -> event.setTask((InsertTask) args[0]); + TaskDisruptor insertDisruptor = new TaskDisruptor<>(insertEventFactory, 1024, + insertTaskThreadFactory, new BlockingWaitStrategy(), insertTaskExecutorHandlers, eventTranslator); + disruptorMap.put(JobType.INSERT, insertDisruptor); + } + + public void dispatchTimerJob(AbstractJob job) { + dispatchDisruptor.publishEvent(job); + } + + public void dispatchInstantTask(AbstractTask task, JobType jobType) { + disruptorMap.get(jobType).publishEvent(task); + } + + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskTokenManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskTokenManager.java new file mode 100644 index 00000000000000..b977e05e340565 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/TaskTokenManager.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.job.manager; + +import jline.internal.Log; +import lombok.experimental.UtilityClass; +import lombok.extern.slf4j.Slf4j; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Semaphore; + +/** + * TaskTokenManager is responsible for managing semaphore tokens for different jobs. + * It provides a method to acquire a semaphore token for a specific job ID with the given maximum concurrency. + * If a semaphore doesn't exist for the job ID, it creates a new one and adds it to the map. + */ +@Slf4j +@UtilityClass +public class TaskTokenManager { + + private static final Map taskTokenMap = new ConcurrentHashMap<>(16); + + /** + * Tries to acquire a semaphore token for the specified job ID with the given maximum concurrency. + * If a semaphore doesn't exist for the job ID, it creates a new one and adds it to the map. + * + * @param jobId the ID of the job + * @param maxConcurrent the maximum concurrency for the job + * @return the acquired semaphore + */ + public static Semaphore tryAcquire(long jobId, long maxConcurrent) { + Semaphore semaphore = taskTokenMap.computeIfAbsent(jobId, id -> new Semaphore((int) maxConcurrent)); + try { + semaphore.acquire(); + } catch (InterruptedException e) { + Log.warn("Interrupted while acquiring semaphore for job id: {} ", jobId, e); + Thread.currentThread().interrupt(); + } + return semaphore; + } +} 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 new file mode 100644 index 00000000000000..fd781d1b88f6e6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/JobScheduler.java @@ -0,0 +1,178 @@ +// 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.common.CustomThreadFactory; +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.common.JobStatus; +import org.apache.doris.job.disruptor.TaskDisruptor; +import org.apache.doris.job.executor.TimerJobSchedulerTask; +import org.apache.doris.job.manager.TaskDisruptorGroupManager; +import org.apache.doris.job.task.AbstractTask; + +import io.netty.util.HashedWheelTimer; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.collections.CollectionUtils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +@Slf4j +public class JobScheduler> implements Closeable { + + /** + * scheduler tasks, it's used to scheduler job + */ + private HashedWheelTimer timerTaskScheduler; + + private TaskDisruptor timerJobDisruptor; + + private TaskDisruptorGroupManager taskDisruptorGroupManager; + + private long latestBatchSchedulerTimerTaskTimeMs = 0L; + + private static final long BATCH_SCHEDULER_INTERVAL_SECONDS = 600; + + private final Map jobMap; + + public JobScheduler(Map jobMap) { + this.jobMap = jobMap; + } + + /** + * batch scheduler interval ms time + */ + private static final long BATCH_SCHEDULER_INTERVAL_MILLI_SECONDS = BATCH_SCHEDULER_INTERVAL_SECONDS * 1000L; + + public void start() { + timerTaskScheduler = new HashedWheelTimer(new CustomThreadFactory("timer-task-scheduler"), 1, + TimeUnit.SECONDS, 660); + timerTaskScheduler.start(); + taskDisruptorGroupManager = new TaskDisruptorGroupManager(); + taskDisruptorGroupManager.init(); + this.timerJobDisruptor = taskDisruptorGroupManager.getDispatchDisruptor(); + } + + /** + * We will cycle system scheduler tasks every 10 minutes. + * Jobs will be re-registered after the task is completed + */ + private void cycleSystemSchedulerTasks() { + log.debug("re-register system scheduler timer tasks" + TimeUtils.longToTimeString(System.currentTimeMillis())); + timerTaskScheduler.newTimeout(timeout -> { + + + cycleSystemSchedulerTasks(); + }, BATCH_SCHEDULER_INTERVAL_SECONDS, TimeUnit.SECONDS); + + } + + private void batchSchedulerTimerJob() { + jobMap.forEach((jobId, job) -> { + if (job.getJobStatus().equals(JobStatus.RUNNING)) { + return; + } + if (!job.getJobConfig().checkIsTimerJob()) { + cycleTimerJobScheduler(job); + } + + }); + } + + public void scheduleOneJob(T job) { + if (job.getJobStatus().equals(JobStatus.RUNNING)) { + return; + } + if (!job.getJobConfig().checkIsTimerJob()) { + //manual job will not scheduler + if (JobExecuteType.MANUAL.equals(job.getJobConfig().getExecuteType())) { + return; + } + //todo skip streaming job,improve in the future + if (JobExecuteType.INSTANT.equals(job.getJobConfig().getExecuteType()) && job.isReadyForScheduling()) { + schedulerImmediateJob(job); + } + } + //if it's timer job and trigger last window already start, we will scheduler it immediately + cycleTimerJobScheduler(job); + } + + @Override + public void close() throws IOException { + + } + + + private void cycleTimerJobScheduler(T job) { + List delaySeconds = job.getJobConfig().getTriggerDelayTimes(System.currentTimeMillis(), + System.currentTimeMillis(), latestBatchSchedulerTimerTaskTimeMs); + if (CollectionUtils.isNotEmpty(delaySeconds)) { + delaySeconds.forEach(delaySecond -> { + TimerJobSchedulerTask timerJobSchedulerTask = new TimerJobSchedulerTask<>(timerJobDisruptor, job); + timerTaskScheduler.newTimeout(timerJobSchedulerTask, delaySecond, TimeUnit.SECONDS); + }); + } + } + + + private void schedulerImmediateJob(T job) { + List tasks = job.createTasks(); + if (CollectionUtils.isEmpty(tasks)) { + return; + } + tasks.forEach(task -> taskDisruptorGroupManager.dispatchInstantTask(task, job.getJobType())); + + } + + private void triggerJob(T job) { + if (!job.isReadyForScheduling()) { + return; + } + List tasks = job.createTasks(); + tasks.forEach(task -> taskDisruptorGroupManager.dispatchInstantTask(task, job.getJobType())); + } + + /** + * We will get the task in the next time window, and then hand it over to the time wheel for timing trigger + */ + private void executeTimerJobIdsWithinLastTenMinutesWindow() { + if (jobMap.isEmpty()) { + return; + } + long currentTimeMs = System.currentTimeMillis(); + long startsWindow = System.currentTimeMillis(); + long endsWindow = 1L; // fixme + for (Map.Entry entry : jobMap.entrySet()) { + T job = entry.getValue(); + List delaySeconds = job.getJobConfig().getTriggerDelayTimes(currentTimeMs, startsWindow, endsWindow); + if (CollectionUtils.isEmpty(delaySeconds)) { + continue; + } + + for (Long timeMs : delaySeconds) { + TimerJobSchedulerTask timerJobSchedulerTask = new TimerJobSchedulerTask(timerJobDisruptor, job); + timerTaskScheduler.newTimeout(timerJobSchedulerTask, timeMs, TimeUnit.SECONDS); + } + } + } +} 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 new file mode 100644 index 00000000000000..7d339e09da3493 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/task/AbstractTask.java @@ -0,0 +1,81 @@ +// 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.task; + +import org.apache.doris.catalog.Env; +import org.apache.doris.job.base.Job; +import org.apache.doris.job.common.TaskStatus; +import org.apache.doris.job.common.TaskType; + +import lombok.Data; +import lombok.extern.slf4j.Slf4j; + +@Data +@Slf4j +public abstract class AbstractTask implements Task { + + private Long jobId; + + private Long taskId; + + private TaskStatus status; + + private Long createTimeMs; + + private Long startTimeMs; + + private Long finishTimeMs; + + private TaskType taskType; + + @Override + public void onFail(String msg) { + Env.getCurrentEnv().getJobManager().getJob(jobId).onTaskFail(taskId); + } + + @Override + public void onFail() { + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + job.onTaskFail(getTaskId()); + } + + @Override + public void onSuccess() { + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + job.onTaskSuccess(getTaskId()); + } + + @Override + public void cancel() { + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + job.onTaskCancel(getTaskId()); + } + + public void runTask() { + before(); + try { + run(); + onSuccess(); + } catch (Exception e) { + onFail(); + log.warn("execute task error, job id is {},task id is {}", jobId, taskId, e); + } + } + + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/task/Task.java b/fe/fe-core/src/main/java/org/apache/doris/job/task/Task.java new file mode 100644 index 00000000000000..dc7a6d81fc63c1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/task/Task.java @@ -0,0 +1,39 @@ +// 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.task; + +import org.apache.doris.qe.ShowResultSetMetaData; + +public interface Task { + + void before(); + + void run(); + + void onFail(); + + void onFail(String msg); + + void onSuccess(); + + void cancel(); + + ShowResultSetMetaData getMetaData(); + + +} 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 ba2fe4becf6a1f..162e8b3cc882c4 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 @@ -44,6 +44,7 @@ import org.apache.doris.datasource.InitDatabaseLog; import org.apache.doris.datasource.InitTableLog; import org.apache.doris.ha.MasterInfo; +import org.apache.doris.job.base.AbstractJob; import org.apache.doris.journal.bdbje.Timestamp; import org.apache.doris.load.DeleteInfo; import org.apache.doris.load.ExportJob; @@ -117,8 +118,6 @@ import org.apache.doris.policy.Policy; import org.apache.doris.policy.StoragePolicy; import org.apache.doris.resource.workloadgroup.WorkloadGroup; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.job.JobTask; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.TableStatsMeta; import org.apache.doris.system.Backend; @@ -538,16 +537,14 @@ public void readFields(DataInput in) throws IOException { case OperationType.OP_UPDATE_SCHEDULER_JOB: case OperationType.OP_DELETE_SCHEDULER_JOB: case OperationType.OP_CREATE_SCHEDULER_JOB: { - Job job = Job.readFields(in); + AbstractJob job = AbstractJob.readFields(in); data = job; isRead = true; break; } case OperationType.OP_CREATE_SCHEDULER_TASK: case OperationType.OP_DELETE_SCHEDULER_TASK: { - JobTask task = JobTask.readFields(in); - data = task; - isRead = true; + //todo improve break; } case OperationType.OP_CREATE_LOAD_JOB: { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSchedule.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSchedule.java index 2f4aa5f96c82fb..17123a3235ec48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSchedule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSchedule.java @@ -18,8 +18,8 @@ package org.apache.doris.mtmv; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.scheduler.common.IntervalUnit; import com.google.gson.annotations.SerializedName; import org.apache.commons.lang3.StringUtils; 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 a8204f1ea97ae1..70cb37b862f08f 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 @@ -28,6 +28,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; +import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.load.loadv2.LoadTask; import org.apache.doris.mtmv.MTMVRefreshEnum.BuildMode; import org.apache.doris.mtmv.MTMVRefreshEnum.RefreshMethod; @@ -374,7 +375,6 @@ import org.apache.doris.policy.PolicyTypeEnum; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SqlModeHelper; -import org.apache.doris.scheduler.common.IntervalUnit; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; 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 d17fbc99c94be8..c2f3fc38b20e3a 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 @@ -52,6 +52,7 @@ import org.apache.doris.datasource.InitCatalogLog; import org.apache.doris.datasource.InitDatabaseLog; import org.apache.doris.ha.MasterInfo; +import org.apache.doris.job.base.AbstractJob; import org.apache.doris.journal.Journal; import org.apache.doris.journal.JournalCursor; import org.apache.doris.journal.JournalEntity; @@ -78,8 +79,6 @@ import org.apache.doris.policy.Policy; import org.apache.doris.policy.StoragePolicy; import org.apache.doris.resource.workloadgroup.WorkloadGroup; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.job.JobTask; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.AnalysisManager; import org.apache.doris.statistics.TableStatsMeta; @@ -661,21 +660,21 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { break; } case OperationType.OP_CREATE_SCHEDULER_JOB: { - Job job = (Job) journal.getData(); - Env.getCurrentEnv().getAsyncJobManager().replayCreateJob(job); + AbstractJob job = (AbstractJob) journal.getData(); + Env.getCurrentEnv().getJobManager().replayCreateJob(job); break; } case OperationType.OP_UPDATE_SCHEDULER_JOB: { - Job job = (Job) journal.getData(); - Env.getCurrentEnv().getAsyncJobManager().replayUpdateJob(job); + AbstractJob job = (AbstractJob) journal.getData(); + Env.getCurrentEnv().getJobManager().replayUpdateJob(job); break; } case OperationType.OP_DELETE_SCHEDULER_JOB: { - Job job = (Job) journal.getData(); - Env.getCurrentEnv().getAsyncJobManager().replayDeleteJob(job); + AbstractJob job = (AbstractJob) journal.getData(); + Env.getCurrentEnv().getJobManager().replayDeleteJob(job); break; } - case OperationType.OP_CREATE_SCHEDULER_TASK: { + /*case OperationType.OP_CREATE_SCHEDULER_TASK: { JobTask task = (JobTask) journal.getData(); Env.getCurrentEnv().getJobTaskManager().replayCreateTask(task); break; @@ -684,7 +683,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { JobTask task = (JobTask) journal.getData(); Env.getCurrentEnv().getJobTaskManager().replayDeleteTask(task); break; - } + }*/ case OperationType.OP_CHANGE_ROUTINE_LOAD_JOB: { RoutineLoadOperation operation = (RoutineLoadOperation) journal.getData(); Env.getCurrentEnv().getRoutineLoadManager().replayChangeRoutineLoadJob(operation); @@ -1602,23 +1601,15 @@ public void logCreateRoutineLoadJob(RoutineLoadJob routineLoadJob) { logEdit(OperationType.OP_CREATE_ROUTINE_LOAD_JOB, routineLoadJob); } - public void logCreateJob(Job job) { + public void logCreateJob(AbstractJob job) { logEdit(OperationType.OP_CREATE_SCHEDULER_JOB, job); } - public void logUpdateJob(Job job) { + public void logUpdateJob(AbstractJob job) { logEdit(OperationType.OP_UPDATE_SCHEDULER_JOB, job); } - public void logCreateJobTask(JobTask jobTask) { - logEdit(OperationType.OP_CREATE_SCHEDULER_TASK, jobTask); - } - - public void logDeleteJobTask(JobTask jobTask) { - logEdit(OperationType.OP_DELETE_SCHEDULER_TASK, jobTask); - } - - public void logDeleteJob(Job job) { + 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/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index 9f4bcd86f78dff..8b6f85d511eef5 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 @@ -77,6 +77,8 @@ import org.apache.doris.datasource.paimon.PaimonExternalCatalog; import org.apache.doris.datasource.paimon.PaimonFileExternalCatalog; import org.apache.doris.datasource.paimon.PaimonHMSExternalCatalog; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.extensions.insert.InsertJob; import org.apache.doris.load.loadv2.LoadJob.LoadJobStateUpdateInfo; import org.apache.doris.load.loadv2.SparkLoadJob.SparkLoadJobStateUpdateInfo; import org.apache.doris.load.routineload.AbstractDataSourceProperties; @@ -86,8 +88,6 @@ import org.apache.doris.policy.Policy; import org.apache.doris.policy.RowPolicy; import org.apache.doris.policy.StoragePolicy; -import org.apache.doris.scheduler.executor.JobExecutor; -import org.apache.doris.scheduler.executor.SqlJobExecutor; import org.apache.doris.system.BackendHbResponse; import org.apache.doris.system.BrokerHbResponse; import org.apache.doris.system.FrontendHbResponse; @@ -221,10 +221,10 @@ public class GsonUtils { RuntimeTypeAdapterFactory.of( AbstractDataSourceProperties.class, "clazz") .registerSubtype(KafkaDataSourceProperties.class, KafkaDataSourceProperties.class.getSimpleName()); - private static RuntimeTypeAdapterFactory jobExecutorRuntimeTypeAdapterFactory = + private static RuntimeTypeAdapterFactory jobExecutorRuntimeTypeAdapterFactory = RuntimeTypeAdapterFactory.of( - JobExecutor.class, "clazz") - .registerSubtype(SqlJobExecutor.class, SqlJobExecutor.class.getSimpleName()); + AbstractJob.class, "clazz") + .registerSubtype(InsertJob.class, InsertJob.class.getSimpleName()); private static RuntimeTypeAdapterFactory dbTypeAdapterFactory = RuntimeTypeAdapterFactory.of( DatabaseIf.class, "clazz") diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaPersistMethod.java b/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaPersistMethod.java index 715cd26796632c..6ed00de22b8ac3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaPersistMethod.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/meta/MetaPersistMethod.java @@ -232,10 +232,6 @@ public static MetaPersistMethod create(String name) throws NoSuchMethodException Env.class.getDeclaredMethod("saveAsyncJobManager", CountingDataOutputStream.class, long.class); break; case "JobTaskManager": - metaPersistMethod.readMethod = - Env.class.getDeclaredMethod("loadJobTaskManager", DataInputStream.class, long.class); - metaPersistMethod.writeMethod = - Env.class.getDeclaredMethod("saveJobTaskManager", CountingDataOutputStream.class, long.class); break; default: break; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java index 483fda4ab51082..b07bdaf538c6a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -123,7 +123,6 @@ import org.apache.doris.common.util.ProfileManager; import org.apache.doris.load.sync.SyncJobManager; import org.apache.doris.persist.CleanQueryStatsInfo; -import org.apache.doris.scheduler.constants.JobCategory; import org.apache.doris.statistics.StatisticsRepository; import org.apache.logging.log4j.LogManager; @@ -184,16 +183,16 @@ public static void execute(Env env, DdlStmt ddlStmt) throws Exception { } else if (ddlStmt instanceof AlterRoutineLoadStmt) { env.getRoutineLoadManager().alterRoutineLoadJob((AlterRoutineLoadStmt) ddlStmt); } else if (ddlStmt instanceof CreateJobStmt) { - env.getJobRegister().registerJob((((CreateJobStmt) ddlStmt).getJob())); + //env.getJobRegister().registerJob((((CreateJobStmt) ddlStmt).getJob()));todo } else if (ddlStmt instanceof StopJobStmt) { - StopJobStmt stmt = (StopJobStmt) ddlStmt; - env.getJobRegister().stopJob(stmt.getDbFullName(), stmt.getName(), JobCategory.SQL); + //StopJobStmt stmt = (StopJobStmt) ddlStmt; + //env.getJobRegister().stopJob(stmt.getDbFullName(), stmt.getName(), JobCategory.SQL); } else if (ddlStmt instanceof PauseJobStmt) { - PauseJobStmt stmt = (PauseJobStmt) ddlStmt; - env.getJobRegister().pauseJob(stmt.getDbFullName(), stmt.getName(), JobCategory.SQL); + //PauseJobStmt stmt = (PauseJobStmt) ddlStmt; + //env.getJobManager().pauseJob(stmt.getDbFullName(), stmt.getName(), JobCategory.SQL); } else if (ddlStmt instanceof ResumeJobStmt) { - ResumeJobStmt stmt = (ResumeJobStmt) ddlStmt; - env.getJobRegister().resumeJob(stmt.getDbFullName(), stmt.getName(), JobCategory.SQL); + //ResumeJobStmt stmt = (ResumeJobStmt) ddlStmt; + //env.getJobRegister().resumeJob(stmt.getDbFullName(), stmt.getName(), JobCategory.SQL); } else if (ddlStmt instanceof CreateUserStmt) { CreateUserStmt stmt = (CreateUserStmt) ddlStmt; env.getAuth().createUser(stmt); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index 2aadab78852349..bedb5174671d68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -189,8 +189,6 @@ import org.apache.doris.load.loadv2.LoadManager; import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.mysql.privilege.PrivPredicate; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.job.JobTask; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.Histogram; @@ -427,9 +425,9 @@ public ShowResultSet execute() throws AnalysisException { } else if (stmt instanceof ShowAnalyzeTaskStatus) { handleShowAnalyzeTaskStatus(); } else if (stmt instanceof ShowJobStmt) { - handleShowJob(); + //handleShowJob(); } else if (stmt instanceof ShowJobTaskStmt) { - handleShowJobTask(); + //handleShowJobTask(); } else if (stmt instanceof ShowConvertLSCStmt) { handleShowConvertLSC(); } else { @@ -1418,7 +1416,7 @@ private void handleShowLoadWarningsFromURL(ShowLoadWarningsStmt showWarningsStmt resultSet = new ShowResultSet(showWarningsStmt.getMetaData(), rows); } - private void handleShowJobTask() { + /*private void handleShowJobTask() { ShowJobTaskStmt showJobTaskStmt = (ShowJobTaskStmt) stmt; List> rows = Lists.newArrayList(); List jobs = Env.getCurrentEnv().getJobRegister() @@ -1439,9 +1437,9 @@ private void handleShowJobTask() { rows.add(jobTask.getShowInfo(job.getJobName())); } resultSet = new ShowResultSet(showJobTaskStmt.getMetaData(), rows); - } + }*/ - private void handleShowJob() throws AnalysisException { + /*private void handleShowJob() throws AnalysisException { ShowJobStmt showJobStmt = (ShowJobStmt) stmt; List> rows = Lists.newArrayList(); // if job exists @@ -1472,7 +1470,7 @@ private void handleShowJob() throws AnalysisException { rows.add(job.getShowInfo()); } resultSet = new ShowResultSet(showJobStmt.getMetaData(), rows); - } + }*/ private void handleShowRoutineLoad() throws AnalysisException { ShowRoutineLoadStmt showRoutineLoadStmt = (ShowRoutineLoadStmt) stmt; diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobCategory.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobCategory.java deleted file mode 100644 index 85783147210bbd..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/constants/JobCategory.java +++ /dev/null @@ -1,58 +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.scheduler.constants; - -import lombok.Getter; - -/** - * The job category is used to distinguish different types of jobs. - */ -public enum JobCategory { - COMMON(1, "common", true), - SQL(2, "sql", true), - MTMV(3, "mtmv", false), - ; - - @Getter - private int code; - - @Getter - private String name; - - /** - * if the job is persistent, it will be saved to the metadata store. - * if the job is not persistent, it will not be saved to the memory. - */ - @Getter - private boolean persistent; - - JobCategory(int code, String name, boolean persistent) { - this.code = code; - this.name = name; - this.persistent = persistent; - } - - public static JobCategory getJobCategoryByName(String name) { - for (JobCategory jobCategory : JobCategory.values()) { - if (jobCategory.name.equalsIgnoreCase(name)) { - return jobCategory; - } - } - throw new IllegalArgumentException("Unknown job category name: " + name); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java index 0e3f8e618d741c..a20f772edaa1e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskDisruptor.java @@ -19,7 +19,6 @@ import org.apache.doris.common.Config; import org.apache.doris.scheduler.constants.TaskType; -import org.apache.doris.scheduler.manager.TimerJobManager; import org.apache.doris.scheduler.manager.TransientTaskManager; import com.lmax.disruptor.BlockingWaitStrategy; @@ -48,8 +47,6 @@ public class TaskDisruptor implements Closeable { private Disruptor disruptor; - - private TimerJobManager timerJobManager; private TransientTaskManager transientTaskManager; private static final int DEFAULT_RING_BUFFER_SIZE = Config.async_task_queen_size; @@ -77,8 +74,7 @@ public class TaskDisruptor implements Closeable { event.setTaskType(taskType); }; - public TaskDisruptor(TimerJobManager timerJobManager, TransientTaskManager transientTaskManager) { - this.timerJobManager = timerJobManager; + public TaskDisruptor(TransientTaskManager transientTaskManager) { this.transientTaskManager = transientTaskManager; } @@ -88,7 +84,7 @@ public void start() { ProducerType.SINGLE, new BlockingWaitStrategy()); WorkHandler[] workers = new TaskHandler[consumerThreadCount]; for (int i = 0; i < consumerThreadCount; i++) { - workers[i] = new TaskHandler(timerJobManager, transientTaskManager); + workers[i] = new TaskHandler(transientTaskManager); } disruptor.handleEventsWithWorkerPool(workers); disruptor.start(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java index 6005fa1bd40ee7..6a0b9f92c5fa02 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java @@ -17,16 +17,8 @@ package org.apache.doris.scheduler.disruptor; -import org.apache.doris.catalog.Env; -import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.scheduler.constants.JobType; import org.apache.doris.scheduler.exception.JobException; import org.apache.doris.scheduler.executor.TransientTaskExecutor; -import org.apache.doris.scheduler.job.ExecutorResult; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.job.JobTask; -import org.apache.doris.scheduler.manager.JobTaskManager; -import org.apache.doris.scheduler.manager.TimerJobManager; import org.apache.doris.scheduler.manager.TransientTaskManager; import com.lmax.disruptor.WorkHandler; @@ -42,22 +34,10 @@ @Slf4j public class TaskHandler implements WorkHandler { - /** - * The event job manager used to retrieve and execute event jobs. - */ - private TimerJobManager timerJobManager; private TransientTaskManager transientTaskManager; - private JobTaskManager jobTaskManager; - - /** - * Constructs a new {@link TaskHandler} instance with the specified event job manager. - * - * @param timerJobManager The event job manager used to retrieve and execute event jobs. - */ - public TaskHandler(TimerJobManager timerJobManager, TransientTaskManager transientTaskManager) { - this.timerJobManager = timerJobManager; + public TaskHandler(TransientTaskManager transientTaskManager) { this.transientTaskManager = transientTaskManager; } @@ -71,10 +51,6 @@ public TaskHandler(TimerJobManager timerJobManager, TransientTaskManager transie @Override public void onEvent(TaskEvent event) { switch (event.getTaskType()) { - case SCHEDULER_JOB_TASK: - case MANUAL_JOB_TASK: - onTimerJobTaskHandle(event); - break; case TRANSIENT_TASK: onTransientTaskHandle(event); break; @@ -84,71 +60,6 @@ public void onEvent(TaskEvent event) { } } - /** - * Processes an event task by retrieving the associated event job and executing it if it is running. - * - * @param taskEvent The event task to be processed. - */ - @SuppressWarnings("checkstyle:UnusedLocalVariable") - public void onTimerJobTaskHandle(TaskEvent taskEvent) { - long jobId = taskEvent.getId(); - long taskId = taskEvent.getTaskId(); - JobTask jobTask = jobTaskManager.pollPrepareTaskByTaskId(jobId, taskId); - if (jobTask == null) { - log.warn("jobTask is null, maybe it's cancel, jobId: {}, taskId: {}", jobId, taskId); - return; - } - Job job = timerJobManager.getJob(jobId); - if (job == null) { - log.info("job is null, jobId: {}", jobId); - return; - } - if (!job.isRunning()) { - log.info("job is not running, eventJobId: {}", jobId); - return; - } - log.debug("job is running, eventJobId: {}", jobId); - - - try { - jobTask.setStartTimeMs(System.currentTimeMillis()); - ExecutorResult result = job.getExecutor().execute(job, jobTask.getContextData()); - job.setLatestCompleteExecuteTimeMs(System.currentTimeMillis()); - if (job.getJobType().equals(JobType.RECURRING)) { - updateJobStatusIfPastEndTime(job); - } else { - // one time job should be finished after execute - updateOnceTimeJobStatus(job); - } - if (null == result) { - log.warn("Job execute failed, jobId: {}, result is null", jobId); - jobTask.setErrorMsg("Job execute failed, result is null"); - jobTask.setIsSuccessful(false); - timerJobManager.setJobLatestStatus(jobId, false); - return; - } - String resultStr = GsonUtils.GSON.toJson(result.getResult()); - jobTask.setExecuteResult(resultStr); - jobTask.setIsSuccessful(result.isSuccess()); - if (!result.isSuccess()) { - log.warn("Job execute failed, jobId: {}, msg : {}", jobId, result.getExecutorSql()); - jobTask.setErrorMsg(result.getErrorMsg()); - } - jobTask.setExecuteSql(result.getExecutorSql()); - } catch (Exception e) { - log.warn("Job execute failed, jobId: {}, msg : {}", jobId, e.getMessage()); - jobTask.setErrorMsg(e.getMessage()); - jobTask.setIsSuccessful(false); - } - jobTask.setEndTimeMs(System.currentTimeMillis()); - if (null == jobTaskManager) { - jobTaskManager = Env.getCurrentEnv().getJobTaskManager(); - } - boolean isPersistent = job.getJobCategory().isPersistent(); - jobTaskManager.addJobTask(jobTask, isPersistent); - timerJobManager.setJobLatestStatus(jobId, jobTask.getIsSuccessful()); - } - public void onTransientTaskHandle(TaskEvent taskEvent) { Long taskId = taskEvent.getId(); TransientTaskExecutor taskExecutor = transientTaskManager.getMemoryTaskExecutor(taskId); @@ -164,18 +75,4 @@ public void onTransientTaskHandle(TaskEvent taskEvent) { } } - private void updateJobStatusIfPastEndTime(Job job) { - if (job.isExpired()) { - timerJobManager.finishJob(job.getJobId()); - } - } - - private void updateOnceTimeJobStatus(Job job) { - if (job.getJobType().equals(JobType.STREAMING)) { - timerJobManager.putOneJobToQueen(job.getJobId()); - return; - } - job.finish(); - } - } diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/AbstractJobExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/AbstractJobExecutor.java deleted file mode 100644 index 7a70c963bd19a6..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/AbstractJobExecutor.java +++ /dev/null @@ -1,54 +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.scheduler.executor; - -import org.apache.doris.analysis.UserIdentity; -import org.apache.doris.catalog.Env; -import org.apache.doris.cluster.ClusterNamespace; -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.thrift.TUniqueId; - -import lombok.Getter; - -import java.util.UUID; - -@Getter -public abstract class AbstractJobExecutor implements JobExecutor { - - protected ConnectContext createContext(Job job) { - ConnectContext ctx = new ConnectContext(); - ctx.setEnv(Env.getCurrentEnv()); - ctx.setCluster(ClusterNamespace.getClusterNameFromFullName(job.getDbName())); - ctx.setDatabase(job.getDbName()); - ctx.setQualifiedUser(job.getUser()); - ctx.setCurrentUserIdentity(UserIdentity.createAnalyzedUserIdentWithIp(job.getUser(), "%")); - ctx.getState().reset(); - ctx.setThreadLocalInfo(); - return ctx; - } - - protected String generateTaskId() { - return UUID.randomUUID().toString(); - } - - protected TUniqueId generateQueryId(String taskIdString) { - UUID taskId = UUID.fromString(taskIdString); - return new TUniqueId(taskId.getMostSignificantBits(), taskId.getLeastSignificantBits()); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/JobExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/JobExecutor.java deleted file mode 100644 index 40aebc8f6adb4b..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/JobExecutor.java +++ /dev/null @@ -1,46 +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.scheduler.executor; - -import org.apache.doris.scheduler.exception.JobException; -import org.apache.doris.scheduler.job.ExecutorResult; -import org.apache.doris.scheduler.job.Job; - -/** - * This interface represents a callback for an event registration. All event registrations - * must implement this interface to provide an execution method. - * We will persist JobExecutor in the database, and then execute it when the scheduler starts. - * We use Gson to serialize and deserialize JobExecutor. so the implementation of JobExecutor needs to be serializable. - * You can see @org.apache.doris.persist.gson.GsonUtils.java for details.When you implement JobExecutor,pls make sure - * you can serialize and deserialize it. - */ -@FunctionalInterface -public interface JobExecutor { - - /** - * Executes the event job and returns the result. - * Exceptions will be caught internally, so there is no need to define or throw them separately. - * - * @param job The event job to execute. - * @param dataContext The data context of the event job. if you need to pass parameters to the event job, - * you can use it. - * @return The result of the event job execution. - */ - ExecutorResult execute(Job job, C dataContext) throws JobException; -} - diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/SqlJobExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/SqlJobExecutor.java deleted file mode 100644 index 546eac9a768029..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/executor/SqlJobExecutor.java +++ /dev/null @@ -1,79 +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.scheduler.executor; - -import org.apache.doris.qe.ConnectContext; -import org.apache.doris.qe.StmtExecutor; -import org.apache.doris.scheduler.exception.JobException; -import org.apache.doris.scheduler.job.ExecutorResult; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.thrift.TUniqueId; - -import com.google.gson.annotations.SerializedName; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; - -import java.util.Map; - -/** - * we use this executor to execute sql job - */ -@Getter -@Slf4j -public class SqlJobExecutor extends AbstractJobExecutor> { - - @Setter - @SerializedName(value = "sql") - private String sql; - - public SqlJobExecutor(String sql) { - this.sql = sql; - } - - @Override - public ExecutorResult execute(Job job, Map dataContext) throws JobException { - ConnectContext ctx = createContext(job); - String taskIdString = generateTaskId(); - TUniqueId queryId = generateQueryId(taskIdString); - try { - StmtExecutor executor = new StmtExecutor(ctx, sql); - executor.execute(queryId); - String result = convertExecuteResult(ctx, taskIdString); - return new ExecutorResult<>(result, true, null, sql); - } catch (Exception e) { - log.warn("execute sql job failed, job id :{}, sql: {}, error: {}", job.getJobId(), sql, e); - return new ExecutorResult<>(null, false, e.getMessage(), sql); - } - - } - - private String convertExecuteResult(ConnectContext ctx, String queryId) throws JobException { - if (null == ctx.getState()) { - throw new JobException("execute sql job failed, sql: " + sql + ", error: response state is null"); - } - if (null != ctx.getState().getErrorCode()) { - throw new JobException("error code: " + ctx.getState().getErrorCode() + ", error msg: " - + ctx.getState().getErrorMessage()); - } - - return "queryId:" + queryId + ",affectedRows : " + ctx.getState().getAffectedRows() + ", warningRows: " - + ctx.getState().getWarningRows() + ",infoMsg" + ctx.getState().getInfoMessage(); - } - -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/Job.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/Job.java deleted file mode 100644 index 7f3fd0884b742e..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/Job.java +++ /dev/null @@ -1,292 +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.scheduler.job; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.io.Text; -import org.apache.doris.common.io.Writable; -import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.scheduler.common.IntervalUnit; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.constants.JobStatus; -import org.apache.doris.scheduler.constants.JobType; -import org.apache.doris.scheduler.executor.JobExecutor; - -import com.google.common.collect.Lists; -import com.google.gson.annotations.SerializedName; -import lombok.Data; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.List; - -/** - * Job is the core of the scheduler module, which is used to store the Job information of the job module. - * We can use the job to uniquely identify a Job. - * The jobName is used to identify the job, which is not unique. - * The jobStatus is used to identify the status of the Job, which is used to control the execution of the - * job. - */ -@Data -public class Job implements Writable { - - public Job(String jobName, Long intervalMilliSeconds, Long startTimeMs, Long endTimeMs, - JobExecutor executor) { - this.jobName = jobName; - this.executor = executor; - this.intervalMs = intervalMilliSeconds; - this.startTimeMs = null == startTimeMs ? 0L : startTimeMs; - this.endTimeMs = null == endTimeMs ? 0L : endTimeMs; - this.jobStatus = JobStatus.RUNNING; - this.jobId = Env.getCurrentEnv().getNextId(); - } - - public Job() { - this.jobId = Env.getCurrentEnv().getNextId(); - } - - @SerializedName("jobId") - private Long jobId; - - @SerializedName("jobName") - private String jobName; - - @SerializedName("dbName") - private String dbName; - - /** - * The status of the job, which is used to control the execution of the job. - * - * @see JobStatus - */ - @SerializedName("jobStatus") - private JobStatus jobStatus; - - @SerializedName("jobType") - private JobType jobType = JobType.RECURRING; - - /** - * The executor of the job. - * - * @see JobExecutor - */ - @SerializedName("executor") - private JobExecutor executor; - @SerializedName("baseName") - private String baseName; - - @SerializedName("user") - private String user; - - @SerializedName("intervalMs") - private Long intervalMs = 0L; - @SerializedName("startTimeMs") - private Long startTimeMs = 0L; - - @SerializedName("endTimeMs") - private Long endTimeMs = 0L; - - @SerializedName("timezone") - private String timezone; - - @SerializedName("jobCategory") - private JobCategory jobCategory; - - - @SerializedName("latestStartExecuteTimeMs") - private Long latestStartExecuteTimeMs = 0L; - @SerializedName("latestCompleteExecuteTimeMs") - private Long latestCompleteExecuteTimeMs = 0L; - - @SerializedName("intervalUnit") - private IntervalUnit intervalUnit; - @SerializedName("originInterval") - private Long originInterval; - @SerializedName("nextExecuteTimeMs") - private Long nextExecuteTimeMs = 0L; - - @SerializedName("createTimeMs") - private Long createTimeMs = System.currentTimeMillis(); - - private Boolean lastExecuteTaskStatus; - - @SerializedName("comment") - private String comment; - - @SerializedName("errMsg") - private String errMsg; - - /** - * if we want to start the job immediately, we can set this flag to true. - * The default value is false. - * when we set this flag to true, the start time will be set to current time. - * we don't need to serialize this field. - */ - private boolean immediatelyStart = false; - - public boolean isRunning() { - return jobStatus == JobStatus.RUNNING; - } - - public boolean isStopped() { - return jobStatus == JobStatus.STOPPED; - } - - public boolean isFinished() { - return jobStatus == JobStatus.FINISHED; - } - - public boolean isExpired(long nextExecuteTimestamp) { - if (endTimeMs == 0L) { - return false; - } - return nextExecuteTimestamp > endTimeMs; - } - - public boolean isTaskTimeExceeded() { - if (endTimeMs == 0L) { - return false; - } - return System.currentTimeMillis() >= endTimeMs || nextExecuteTimeMs > endTimeMs; - } - - public boolean isExpired() { - if (endTimeMs == 0L) { - return false; - } - return System.currentTimeMillis() >= endTimeMs; - } - - public Long getExecuteTimestampAndGeneratorNext() { - this.latestStartExecuteTimeMs = nextExecuteTimeMs; - // todo The problem of delay should be considered. If it is greater than the ten-minute time window, - // should the task be lost or executed on a new time window? - this.nextExecuteTimeMs = latestStartExecuteTimeMs + intervalMs; - return nextExecuteTimeMs; - } - - public void pause() { - this.jobStatus = JobStatus.PAUSED; - } - - public void pause(String errMsg) { - this.jobStatus = JobStatus.PAUSED; - this.errMsg = errMsg; - } - - public void finish() { - this.jobStatus = JobStatus.FINISHED; - } - - public void resume() { - this.jobStatus = JobStatus.RUNNING; - } - - public void stop() { - this.jobStatus = JobStatus.STOPPED; - } - - public void checkJobParam() throws DdlException { - if (null == jobCategory) { - throw new DdlException("jobCategory must be set"); - } - if (null == executor) { - throw new DdlException("Job executor must be set"); - } - if (null == jobType) { - throw new DdlException("Job type must be set"); - } - if (jobType.equals(JobType.MANUAL)) { - return; - } - if (startTimeMs != 0L && startTimeMs < System.currentTimeMillis()) { - throw new DdlException("startTimeMs must be greater than current time"); - } - if (immediatelyStart && startTimeMs != 0L) { - throw new DdlException("immediately start and startTimeMs can't be set at the same time"); - } - if (immediatelyStart) { - startTimeMs = System.currentTimeMillis(); - } - if (endTimeMs != 0L && endTimeMs < System.currentTimeMillis()) { - throw new DdlException("endTimeMs must be greater than current time"); - } - if (null != intervalUnit && null != originInterval) { - this.intervalMs = intervalUnit.getParameterValue(originInterval); - } - if (jobType.equals(JobType.RECURRING) && (intervalMs == null || intervalMs <= 0L)) { - throw new DdlException("cycle job must set intervalMs"); - } - - } - - - @Override - public void write(DataOutput out) throws IOException { - String jobData = GsonUtils.GSON.toJson(this); - Text.writeString(out, jobData); - } - - public static Job readFields(DataInput in) throws IOException { - return GsonUtils.GSON.fromJson(Text.readString(in), Job.class); - } - - public List getShowInfo() { - List row = Lists.newArrayList(); - row.add(String.valueOf(jobId)); - row.add(jobName); - row.add(user); - row.add(jobType.name()); - - row.add(convertRecurringStrategyToString()); - row.add(jobStatus.name()); - row.add(null == lastExecuteTaskStatus ? "null" : lastExecuteTaskStatus.toString()); - row.add(createTimeMs <= 0L ? "null" : TimeUtils.longToTimeString(createTimeMs)); - row.add(comment == null ? "null" : comment); - return row; - } - - private String convertRecurringStrategyToString() { - if (jobType.equals(JobType.MANUAL)) { - return "MANUAL TRIGGER"; - } - switch (jobType) { - case ONE_TIME: - return "AT " + TimeUtils.longToTimeString(startTimeMs); - case RECURRING: - String result = "EVERY " + originInterval + " " + intervalUnit.name(); - if (startTimeMs > 0) { - result += " STARTS " + TimeUtils.longToTimeString(startTimeMs); - } - if (endTimeMs > 0) { - result += " ENDS " + TimeUtils.longToTimeString(endTimeMs); - } - return result; - case STREAMING: - return "STREAMING" + (startTimeMs > 0 ? " AT " + TimeUtils.longToTimeString(startTimeMs) : ""); - case MANUAL: - return "MANUAL TRIGGER"; - default: - return "UNKNOWN"; - } - } - -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/JobTask.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/JobTask.java deleted file mode 100644 index 1f8aac58285803..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/JobTask.java +++ /dev/null @@ -1,136 +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.scheduler.job; - -import org.apache.doris.common.io.Text; -import org.apache.doris.common.io.Writable; -import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.persist.gson.GsonUtils; -import org.apache.doris.scheduler.constants.TaskType; - -import com.google.common.collect.Lists; -import com.google.gson.annotations.SerializedName; -import lombok.Data; -import lombok.extern.slf4j.Slf4j; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.List; - -@Data -@Slf4j -public class JobTask implements Writable { - - @SerializedName("jobId") - private Long jobId; - @SerializedName("taskId") - private Long taskId; - @SerializedName("createTimeMs") - private Long createTimeMs; - @SerializedName("startTimeMs") - private Long startTimeMs; - @SerializedName("endTimeMs") - private Long endTimeMs; - @SerializedName("successful") - private Boolean isSuccessful; - - @SerializedName("executeSql") - private String executeSql; - @SerializedName("executeResult") - private String executeResult; - @SerializedName("errorMsg") - private String errorMsg; - - @SerializedName("contextDataStr") - private String contextDataStr; - - @SerializedName("taskType") - private TaskType taskType = TaskType.SCHEDULER_JOB_TASK; - - /** - * Some parameters specific to the current task that need to be used to execute the task - * eg: sql task, sql it's: select * from table where id = 1 order by id desc limit ${limit} offset ${offset} - * contextData is a map, key1 is limit, value is 10,key2 is offset, value is 1 - * when execute the task, we will replace the ${limit} to 10, ${offset} to 1 - * so to execute sql is: select * from table where id = 1 order by id desc limit 10 offset 1. - */ - private T contextData; - - public JobTask(Long jobId, Long taskId, Long createTimeMs) { - //it's enough to use nanoTime to identify a task - this.taskId = taskId; - this.jobId = jobId; - this.createTimeMs = createTimeMs; - } - - public JobTask(Long jobId, Long taskId, Long createTimeMs, T contextData) { - this(jobId, taskId, createTimeMs); - this.contextData = contextData; - try { - this.contextDataStr = GsonUtils.GSON.toJson(contextData); - } catch (Exception e) { - this.contextDataStr = null; - log.error("contextData serialize failed, jobId: {}, taskId: {}", jobId, taskId, e); - } - } - - public List getShowInfo(String jobName) { - List row = Lists.newArrayList(); - row.add(String.valueOf(taskId)); - row.add(String.valueOf(jobId)); - row.add(jobName); - if (null != createTimeMs) { - row.add(TimeUtils.longToTimeString(createTimeMs)); - } - row.add(TimeUtils.longToTimeString(startTimeMs)); - row.add(null == endTimeMs ? "null" : TimeUtils.longToTimeString(endTimeMs)); - if (endTimeMs == null) { - row.add("RUNNING"); - } else { - row.add(isSuccessful ? "SUCCESS" : "FAILED"); - } - if (null == executeSql) { - row.add("null"); - } else { - row.add(executeSql); - } - if (null == executeResult) { - row.add("null"); - } else { - row.add(executeResult); - } - if (null == errorMsg) { - row.add("null"); - } else { - row.add(errorMsg); - } - row.add(taskType.name()); - return row; - } - - @Override - public void write(DataOutput out) throws IOException { - String jobData = GsonUtils.GSON.toJson(this); - Text.writeString(out, jobData); - } - - public static JobTask readFields(DataInput in) throws IOException { - return GsonUtils.GSON.fromJson(Text.readString(in), JobTask.class); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/TimerJobTask.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/TimerJobTask.java deleted file mode 100644 index b3a199a8a62aaa..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/job/TimerJobTask.java +++ /dev/null @@ -1,57 +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.scheduler.job; - -import org.apache.doris.scheduler.disruptor.TaskDisruptor; - -import io.netty.util.Timeout; -import io.netty.util.TimerTask; -import lombok.Getter; - -/** - * This class represents a timer task that can be scheduled by a Netty timer. - * When the timer task is triggered, it produces a Job task using the Disruptor. - * The Job task contains the ID of the Job and the ID of the task itself. - */ -@Getter -public class TimerJobTask implements TimerTask { - - private final Long jobId; - - // more fields should be added here and record in feature - private final Long taskId; - - private final Long startTimestamp; - - private final TaskDisruptor taskDisruptor; - - public TimerJobTask(Long jobId, Long taskId, Long startTimestamp, TaskDisruptor taskDisruptor) { - this.jobId = jobId; - this.startTimestamp = startTimestamp; - this.taskDisruptor = taskDisruptor; - this.taskId = taskId; - } - - @Override - public void run(Timeout timeout) { - if (timeout.isCancelled()) { - return; - } - taskDisruptor.tryPublish(jobId, taskId); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/JobTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/JobTaskManager.java deleted file mode 100644 index 6f7cd839074e4b..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/JobTaskManager.java +++ /dev/null @@ -1,152 +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.scheduler.manager; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.Config; -import org.apache.doris.common.io.Writable; -import org.apache.doris.common.util.LogBuilder; -import org.apache.doris.common.util.LogKey; -import org.apache.doris.scheduler.job.JobTask; - -import lombok.extern.slf4j.Slf4j; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; - -@Slf4j -public class JobTaskManager implements Writable { - - private static final Integer TASK_MAX_NUM = Config.scheduler_job_task_max_saved_count; - - private ConcurrentHashMap> jobTaskMap = new ConcurrentHashMap<>(16); - - - /** - * taskId -> startTime - * used to record the start time of the task to be executed - * will clear when the task is executed - */ - private static ConcurrentHashMap> prepareTaskCreateMsMap = new ConcurrentHashMap<>(16); - - public static void addPrepareTask(JobTask jobTask) { - long jobId = jobTask.getJobId(); - long taskId = jobTask.getTaskId(); - prepareTaskCreateMsMap.computeIfAbsent(jobId, k -> new HashMap<>()); - prepareTaskCreateMsMap.get(jobId).put(taskId, jobTask); - } - - public static JobTask pollPrepareTaskByTaskId(Long jobId, Long taskId) { - if (!prepareTaskCreateMsMap.containsKey(jobId) || !prepareTaskCreateMsMap.get(jobId).containsKey(taskId)) { - // if the job is not in the map, return new JobTask - // return new JobTask(jobId, taskId, System.currentTimeMillis()); fixme - return null; - } - return prepareTaskCreateMsMap.get(jobId).remove(taskId); - } - - public static void clearPrepareTaskByJobId(Long jobId) { - prepareTaskCreateMsMap.remove(jobId); - } - - public void addJobTask(JobTask jobTask, boolean persist) { - ConcurrentLinkedQueue jobTasks = jobTaskMap - .computeIfAbsent(jobTask.getJobId(), k -> new ConcurrentLinkedQueue<>()); - jobTasks.add(jobTask); - if (jobTasks.size() > TASK_MAX_NUM) { - JobTask oldTask = jobTasks.poll(); - if (persist) { - Env.getCurrentEnv().getEditLog().logDeleteJobTask(oldTask); - } - } - if (persist) { - Env.getCurrentEnv().getEditLog().logCreateJobTask(jobTask); - } - } - - public List getJobTasks(Long jobId) { - if (jobTaskMap.containsKey(jobId)) { - ConcurrentLinkedQueue jobTasks = jobTaskMap.get(jobId); - List jobTaskList = new LinkedList<>(jobTasks); - Collections.reverse(jobTaskList); - return jobTaskList; - } - return new ArrayList<>(); - } - - public void replayCreateTask(JobTask task) { - ConcurrentLinkedQueue jobTasks = jobTaskMap - .computeIfAbsent(task.getJobId(), k -> new ConcurrentLinkedQueue<>()); - jobTasks.add(task); - log.info(new LogBuilder(LogKey.SCHEDULER_TASK, task.getTaskId()) - .add("msg", "replay create scheduler task").build()); - } - - public void replayDeleteTask(JobTask task) { - ConcurrentLinkedQueue jobTasks = jobTaskMap.get(task.getJobId()); - if (jobTasks != null) { - jobTasks.remove(task); - } - log.info(new LogBuilder(LogKey.SCHEDULER_TASK, task.getTaskId()) - .add("msg", "replay delete scheduler task").build()); - } - - public void deleteJobTasks(Long jobId) { - ConcurrentLinkedQueue jobTasks = jobTaskMap.get(jobId); - if (null != jobTasks) { - jobTaskMap.remove(jobId); - } - clearPrepareTaskByJobId(jobId); - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(jobTaskMap.size()); - for (Map.Entry> entry : jobTaskMap.entrySet()) { - out.writeLong(entry.getKey()); - out.writeInt(entry.getValue().size()); - for (JobTask jobTask : entry.getValue()) { - jobTask.write(out); - } - } - - } - - public void readFields(DataInput in) throws IOException { - int size = in.readInt(); - for (int i = 0; i < size; i++) { - Long jobId = in.readLong(); - int taskSize = in.readInt(); - ConcurrentLinkedQueue jobTasks = new ConcurrentLinkedQueue<>(); - for (int j = 0; j < taskSize; j++) { - JobTask jobTask = JobTask.readFields(in); - jobTasks.add(jobTask); - } - jobTaskMap.put(jobId, jobTasks); - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TimerJobManager.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TimerJobManager.java deleted file mode 100644 index 33ac7d5f940a3c..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TimerJobManager.java +++ /dev/null @@ -1,573 +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.scheduler.manager; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.Config; -import org.apache.doris.common.DdlException; -import org.apache.doris.common.PatternMatcher; -import org.apache.doris.common.io.Writable; -import org.apache.doris.common.util.LogBuilder; -import org.apache.doris.common.util.LogKey; -import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.constants.JobStatus; -import org.apache.doris.scheduler.constants.JobType; -import org.apache.doris.scheduler.constants.TaskType; -import org.apache.doris.scheduler.disruptor.TaskDisruptor; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.job.JobTask; -import org.apache.doris.scheduler.job.TimerJobTask; - -import io.netty.util.HashedWheelTimer; -import io.netty.util.Timeout; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; - -import java.io.Closeable; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; - -@Slf4j -public class TimerJobManager implements Closeable, Writable { - - private final ConcurrentHashMap jobMap = new ConcurrentHashMap<>(128); - private long lastBatchSchedulerTimestamp; - private static final long BATCH_SCHEDULER_INTERVAL_SECONDS = 600; - - /** - * batch scheduler interval ms time - */ - private static final long BATCH_SCHEDULER_INTERVAL_MILLI_SECONDS = BATCH_SCHEDULER_INTERVAL_SECONDS * 1000L; - - private boolean isClosed = false; - - /** - * key: jobid - * value: timeout list for one job - * it's used to cancel task, if task has started, it can't be canceled - */ - private final ConcurrentHashMap> jobTimeoutMap = new ConcurrentHashMap<>(128); - - /** - * scheduler tasks, it's used to scheduler job - */ - private HashedWheelTimer dorisTimer; - - /** - * Producer and Consumer model - * disruptor is used to handle task - * disruptor will start a thread pool to handle task - */ - @Setter - private TaskDisruptor disruptor; - - public TimerJobManager() { - this.lastBatchSchedulerTimestamp = System.currentTimeMillis(); - } - - public void start() { - dorisTimer = new HashedWheelTimer(1, TimeUnit.SECONDS, 660); - dorisTimer.start(); - Long currentTimeMs = System.currentTimeMillis(); - jobMap.forEach((jobId, job) -> { - Long nextExecuteTimeMs = findFistExecuteTime(currentTimeMs, job.getStartTimeMs(), - job.getIntervalMs(), job.getJobType()); - job.setNextExecuteTimeMs(nextExecuteTimeMs); - }); - batchSchedulerTasks(); - cycleSystemSchedulerTasks(); - } - - public Long registerJob(Job job) throws DdlException { - job.checkJobParam(); - checkIsJobNameUsed(job.getDbName(), job.getJobName(), job.getJobCategory()); - jobMap.putIfAbsent(job.getJobId(), job); - initAndSchedulerJob(job); - Env.getCurrentEnv().getEditLog().logCreateJob(job); - return job.getJobId(); - } - - public void replayCreateJob(Job job) { - if (jobMap.containsKey(job.getJobId())) { - return; - } - jobMap.putIfAbsent(job.getJobId(), job); - initAndSchedulerJob(job); - log.info(new LogBuilder(LogKey.SCHEDULER_JOB, job.getJobId()) - .add("msg", "replay create scheduler job").build()); - } - - /** - * Replay update load job. - **/ - public void replayUpdateJob(Job job) { - jobMap.put(job.getJobId(), job); - if (JobStatus.RUNNING.equals(job.getJobStatus())) { - initAndSchedulerJob(job); - } - log.info(new LogBuilder(LogKey.SCHEDULER_JOB, job.getJobId()) - .add("msg", "replay update scheduler job").build()); - } - - public void replayDeleteJob(Job job) { - if (null == jobMap.get(job.getJobId())) { - return; - } - jobMap.remove(job.getJobId()); - log.info(new LogBuilder(LogKey.SCHEDULER_JOB, job.getJobId()) - .add("msg", "replay delete scheduler job").build()); - Env.getCurrentEnv().getJobTaskManager().deleteJobTasks(job.getJobId()); - } - - private void checkIsJobNameUsed(String dbName, String jobName, JobCategory jobCategory) throws DdlException { - Optional optionalJob = jobMap.values().stream().filter(job -> job.getJobCategory().equals(jobCategory)) - .filter(job -> job.getDbName().equals(dbName)) - .filter(job -> job.getJobName().equals(jobName)).findFirst(); - if (optionalJob.isPresent()) { - throw new DdlException("Name " + jobName + " already used in db " + dbName); - } - } - - private void initAndSchedulerJob(Job job) { - if (!job.getJobStatus().equals(JobStatus.RUNNING) || job.getJobType().equals(JobType.MANUAL)) { - return; - } - - Long currentTimeMs = System.currentTimeMillis(); - Long nextExecuteTimeMs = findFistExecuteTime(currentTimeMs, job.getStartTimeMs(), - job.getIntervalMs(), job.getJobType()); - job.setNextExecuteTimeMs(nextExecuteTimeMs); - if (job.getNextExecuteTimeMs() < lastBatchSchedulerTimestamp) { - List executeTimestamp = findTasksBetweenTime(job, - lastBatchSchedulerTimestamp, - job.getNextExecuteTimeMs(), job.getJobType()); - if (!executeTimestamp.isEmpty()) { - for (Long timestamp : executeTimestamp) { - putOneTask(job.getJobId(), timestamp); - } - } - } - } - - private Long findFistExecuteTime(Long currentTimeMs, Long startTimeMs, Long intervalMs, JobType jobType) { - // if job not delay, first execute time is start time - if (startTimeMs != 0L && startTimeMs > currentTimeMs) { - return startTimeMs; - } - // if job already delay, first execute time is current time - if (startTimeMs != 0L && startTimeMs < currentTimeMs) { - return currentTimeMs; - } - // if it's cycle job and not set start tine, first execute time is current time + interval - if (jobType.equals(JobType.RECURRING) && startTimeMs == 0L) { - return currentTimeMs + intervalMs; - } - // if it's not cycle job and already delay, first execute time is current time - return currentTimeMs; - } - - public boolean immediateExecuteTask(Long jobId, T taskContextData) throws DdlException { - Job job = jobMap.get(jobId); - if (job == null) { - log.warn("immediateExecuteTask failed, jobId: {} not exist", jobId); - return false; - } - if (!job.getJobStatus().equals(JobStatus.RUNNING)) { - log.warn("immediateExecuteTask failed, jobId: {} is not running", jobId); - return false; - } - JobTask jobTask = createInitialTask(jobId, taskContextData); - jobTask.setTaskType(TaskType.MANUAL_JOB_TASK); - JobTaskManager.addPrepareTask(jobTask); - disruptor.tryPublish(jobId, jobTask.getTaskId(), TaskType.MANUAL_JOB_TASK); - return true; - } - - public void unregisterJob(Long jobId) { - jobMap.remove(jobId); - } - - public void pauseJob(Long jobId) { - Job job = jobMap.get(jobId); - if (jobMap.get(jobId) == null) { - log.warn("pauseJob failed, jobId: {} not exist", jobId); - } - if (jobMap.get(jobId).getJobStatus().equals(JobStatus.PAUSED)) { - log.warn("pauseJob failed, jobId: {} is already paused", jobId); - } - pauseJob(job); - } - - public void setJobLatestStatus(long jobId, boolean status) { - Job job = jobMap.get(jobId); - if (jobMap.get(jobId) == null) { - log.warn("pauseJob failed, jobId: {} not exist", jobId); - } - job.setLastExecuteTaskStatus(status); - } - - public void stopJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException { - Optional optionalJob = findJob(dbName, jobName, jobCategory); - - if (!optionalJob.isPresent()) { - throw new DdlException("Job " + jobName + " not exist in db " + dbName); - } - Job job = optionalJob.get(); - if (job.getJobStatus().equals(JobStatus.STOPPED)) { - throw new DdlException("Job " + jobName + " is already stopped"); - } - stopJob(optionalJob.get()); - Env.getCurrentEnv().getEditLog().logDeleteJob(optionalJob.get()); - } - - private void stopJob(Job job) { - if (JobStatus.RUNNING.equals(job.getJobStatus())) { - cancelJobAllTask(job.getJobId()); - } - job.setJobStatus(JobStatus.STOPPED); - jobMap.get(job.getJobId()).stop(); - Env.getCurrentEnv().getEditLog().logDeleteJob(job); - Env.getCurrentEnv().getJobTaskManager().deleteJobTasks(job.getJobId()); - } - - - public void resumeJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException { - Optional optionalJob = findJob(dbName, jobName, jobCategory); - if (!optionalJob.isPresent()) { - throw new DdlException("Job " + jobName + " not exist in db " + dbName); - } - Job job = optionalJob.get(); - if (!job.getJobStatus().equals(JobStatus.PAUSED)) { - throw new DdlException("Job " + jobName + " is not paused"); - } - resumeJob(job); - } - - private void resumeJob(Job job) { - cancelJobAllTask(job.getJobId()); - job.setJobStatus(JobStatus.RUNNING); - jobMap.get(job.getJobId()).resume(); - initAndSchedulerJob(job); - Env.getCurrentEnv().getEditLog().logUpdateJob(job); - } - - public void pauseJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException { - Optional optionalJob = findJob(dbName, jobName, jobCategory); - if (!optionalJob.isPresent()) { - throw new DdlException("Job " + jobName + " not exist in db " + dbName); - } - Job job = optionalJob.get(); - if (!job.getJobStatus().equals(JobStatus.RUNNING)) { - throw new DdlException("Job " + jobName + " is not running"); - } - pauseJob(job); - } - - private void pauseJob(Job job) { - cancelJobAllTask(job.getJobId()); - job.setJobStatus(JobStatus.PAUSED); - jobMap.get(job.getJobId()).pause(); - Env.getCurrentEnv().getEditLog().logUpdateJob(job); - } - - public void finishJob(long jobId) { - Job job = jobMap.get(jobId); - if (jobMap.get(jobId) == null) { - log.warn("update job status failed, jobId: {} not exist", jobId); - } - if (jobMap.get(jobId).getJobStatus().equals(JobStatus.FINISHED)) { - return; - } - job.setLatestCompleteExecuteTimeMs(System.currentTimeMillis()); - cancelJobAllTask(job.getJobId()); - job.setJobStatus(JobStatus.FINISHED); - Env.getCurrentEnv().getEditLog().logUpdateJob(job); - } - - private Optional findJob(String dbName, String jobName, JobCategory jobCategory) { - return jobMap.values().stream().filter(job -> checkJobMatch(job, dbName, jobName, jobCategory)).findFirst(); - } - - private boolean checkJobMatch(Job job, String dbName, String jobName, JobCategory jobCategory) { - return job.getDbName().equals(dbName) && job.getJobName().equals(jobName) - && job.getJobCategory().equals(jobCategory); - } - - - public void resumeJob(Long jobId) { - if (jobMap.get(jobId) == null) { - log.warn("resumeJob failed, jobId: {} not exist", jobId); - return; - } - Job job = jobMap.get(jobId); - resumeJob(job); - } - - public void stopJob(Long jobId) { - Job job = jobMap.get(jobId); - if (null == job) { - log.warn("stopJob failed, jobId: {} not exist", jobId); - return; - } - if (job.getJobStatus().equals(JobStatus.STOPPED)) { - log.warn("stopJob failed, jobId: {} is already stopped", jobId); - return; - } - stopJob(job); - } - - public Job getJob(Long jobId) { - return jobMap.get(jobId); - } - - public Map getAllJob() { - return jobMap; - } - - public void batchSchedulerTasks() { - executeJobIdsWithinLastTenMinutesWindow(); - } - - private List findTasksBetweenTime(Job job, Long endTimeEndWindow, Long nextExecuteTime, JobType jobType) { - - List jobExecuteTimes = new ArrayList<>(); - if (!jobType.equals(JobType.RECURRING) && (nextExecuteTime < endTimeEndWindow)) { - jobExecuteTimes.add(nextExecuteTime); - return jobExecuteTimes; - } - if (jobType.equals(JobType.RECURRING) && (nextExecuteTime > endTimeEndWindow)) { - return new ArrayList<>(); - } - while (endTimeEndWindow >= nextExecuteTime) { - if (job.isTaskTimeExceeded()) { - break; - } - jobExecuteTimes.add(nextExecuteTime); - nextExecuteTime = job.getExecuteTimestampAndGeneratorNext(); - } - return jobExecuteTimes; - } - - /** - * We will get the task in the next time window, and then hand it over to the time wheel for timing trigger - */ - private void executeJobIdsWithinLastTenMinutesWindow() { - // if the task executes for more than 10 minutes, it will be delay, so, - // set lastBatchSchedulerTimestamp to current time - if (lastBatchSchedulerTimestamp < System.currentTimeMillis()) { - this.lastBatchSchedulerTimestamp = System.currentTimeMillis(); - } - this.lastBatchSchedulerTimestamp += BATCH_SCHEDULER_INTERVAL_MILLI_SECONDS; - if (jobMap.isEmpty()) { - return; - } - jobMap.forEach((k, v) -> { - if (!v.getJobType().equals(JobType.MANUAL) && v.isRunning() && (v.getNextExecuteTimeMs() - + v.getIntervalMs() < lastBatchSchedulerTimestamp)) { - List executeTimes = findTasksBetweenTime( - v, lastBatchSchedulerTimestamp, - v.getNextExecuteTimeMs(), v.getJobType()); - if (!executeTimes.isEmpty()) { - for (Long executeTime : executeTimes) { - putOneTask(v.getJobId(), executeTime); - } - } - } - }); - } - - /** - * We will cycle system scheduler tasks every 10 minutes. - * Jobs will be re-registered after the task is completed - */ - private void cycleSystemSchedulerTasks() { - log.info("re-register system scheduler tasks" + TimeUtils.longToTimeString(System.currentTimeMillis())); - dorisTimer.newTimeout(timeout -> { - batchSchedulerTasks(); - clearFinishJob(); - cycleSystemSchedulerTasks(); - }, BATCH_SCHEDULER_INTERVAL_SECONDS, TimeUnit.SECONDS); - - } - - /** - * put one task to time wheel,it's well be trigger after delay milliseconds - * if the scheduler is closed, the task will not be put into the time wheel - * if delay is less than 0, the task will be trigger immediately - * - * @param jobId job id, we will use it to find the job - * @param startExecuteTime the task will be trigger in this time, unit is millisecond,and we will convert it to - * delay seconds, we just can be second precision - */ - public void putOneTask(Long jobId, Long startExecuteTime) { - if (isClosed) { - log.info("putOneTask failed, scheduler is closed, jobId: {}", jobId); - return; - } - JobTask jobTask = createAsyncInitialTask(jobId, startExecuteTime); - long taskId = jobTask.getTaskId(); - TimerJobTask task = new TimerJobTask(jobId, taskId, startExecuteTime, disruptor); - long delay = getDelaySecond(task.getStartTimestamp()); - Timeout timeout = dorisTimer.newTimeout(task, delay, TimeUnit.SECONDS); - if (timeout == null) { - log.error("putOneTask failed, jobId: {}", task.getJobId()); - return; - } - if (jobTimeoutMap.containsKey(task.getJobId())) { - jobTimeoutMap.get(task.getJobId()).put(task.getTaskId(), timeout); - JobTaskManager.addPrepareTask(jobTask); - return; - } - Map timeoutMap = new ConcurrentHashMap<>(); - timeoutMap.put(task.getTaskId(), timeout); - jobTimeoutMap.put(task.getJobId(), timeoutMap); - JobTaskManager.addPrepareTask(jobTask); - } - - // cancel all task for one job - // if task has started, it can't be canceled - public void cancelJobAllTask(Long jobId) { - if (!jobTimeoutMap.containsKey(jobId)) { - return; - } - - jobTimeoutMap.get(jobId).values().forEach(timeout -> { - if (!timeout.isExpired() || timeout.isCancelled()) { - timeout.cancel(); - } - }); - JobTaskManager.clearPrepareTaskByJobId(jobId); - } - - // get delay time, if startTimestamp is less than now, return 0 - private long getDelaySecond(long startTimestamp) { - long delay = 0; - long now = System.currentTimeMillis(); - if (startTimestamp > now) { - delay = startTimestamp - now; - } else { - //if execute time is less than now, return 0,immediately execute - log.warn("startTimestamp is less than now, startTimestamp: {}, now: {}", startTimestamp, now); - return delay; - } - return delay / 1000; - } - - @Override - public void close() throws IOException { - isClosed = true; - dorisTimer.stop(); - disruptor.close(); - } - - /** - * sort by job id - * - * @param dbFullName database name - * @param category job category - * @param matcher job name matcher - */ - public List queryJob(String dbFullName, String jobName, JobCategory category, PatternMatcher matcher) { - List jobs = new ArrayList<>(); - jobMap.values().forEach(job -> { - if (matchJob(job, dbFullName, jobName, category, matcher)) { - jobs.add(job); - } - }); - return jobs; - } - - private boolean matchJob(Job job, String dbFullName, String jobName, JobCategory category, PatternMatcher matcher) { - if (StringUtils.isNotBlank(dbFullName) && !job.getDbName().equalsIgnoreCase(dbFullName)) { - return false; - } - if (StringUtils.isNotBlank(jobName) && !job.getJobName().equalsIgnoreCase(jobName)) { - return false; - } - if (category != null && !job.getJobCategory().equals(category)) { - return false; - } - return null == matcher || matcher.match(job.getJobName()); - } - - public void putOneJobToQueen(Long jobId) { - JobTask jobTask = createInitialTask(jobId, null); - JobTaskManager.addPrepareTask(jobTask); - disruptor.tryPublish(jobId, jobTask.getTaskId()); - } - - private JobTask createAsyncInitialTask(long jobId, long createTimeMs) { - long taskId = System.nanoTime(); - return new JobTask(jobId, taskId, createTimeMs); - } - - private JobTask createInitialTask(long jobId, T context) { - long taskId = System.nanoTime(); - return new JobTask(jobId, taskId, System.currentTimeMillis(), context); - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(jobMap.size()); - for (Job job : jobMap.values()) { - job.write(out); - } - } - - /** - * read job from data input, and init job - * - * @param in data input - * @throws IOException io exception when read data input error - */ - public void readFields(DataInput in) throws IOException { - int size = in.readInt(); - for (int i = 0; i < size; i++) { - Job job = Job.readFields(in); - jobMap.putIfAbsent(job.getJobId(), job); - } - } - - /** - * clear finish job,if job finish time is more than @Config.finish_job_max_saved_second, we will delete it - * this method will be called every 10 minutes, therefore, the actual maximum - * deletion time is Config.finish_job_max_saved_second + 10 min. - * we could to delete job in time, but it's not make sense.start - */ - private void clearFinishJob() { - Long now = System.currentTimeMillis(); - jobMap.values().forEach(job -> { - if (job.isFinished() && now - job.getLatestCompleteExecuteTimeMs() > Config.finish_job_max_saved_second) { - jobMap.remove(job.getJobId()); - Env.getCurrentEnv().getEditLog().logDeleteJob(job); - Env.getCurrentEnv().getJobTaskManager().deleteJobTasks(job.getJobId()); - log.debug("delete finish job:{}", job.getJobId()); - } - }); - - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java index bc72689622ec58..9602b19ca2ae6b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java @@ -43,6 +43,8 @@ public class TransientTaskManager { private TaskDisruptor disruptor; public TransientTaskManager() { + disruptor = new TaskDisruptor(this); + disruptor.start(); } public TransientTaskExecutor getMemoryTaskExecutor(Long taskId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/PersistentJobRegister.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/PersistentJobRegister.java deleted file mode 100644 index f1a901299f754e..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/PersistentJobRegister.java +++ /dev/null @@ -1,136 +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.scheduler.registry; - -import org.apache.doris.common.DdlException; -import org.apache.doris.common.PatternMatcher; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.executor.JobExecutor; -import org.apache.doris.scheduler.job.Job; - -import java.io.IOException; -import java.util.List; - -/** - * This interface provides a contract for registering timed scheduling events. - * The implementation should trigger events in a timely manner using a specific algorithm. - * The execution of the events may be asynchronous and not guarantee strict timing accuracy. - */ -public interface PersistentJobRegister { - - /** - * Register a job - * - * @param name job name,it's not unique - * @param intervalMs job interval, unit: ms - * @param executor job executor @See {@link JobExecutor} - * @return event job id - */ - Long registerJob(String name, Long intervalMs, JobExecutor executor) throws DdlException; - - /** - * Register a job - * - * @param name job name,it's not unique - * @param intervalMs job interval, unit: ms - * @param startTimeStamp job start time stamp, unit: ms - * if startTimeStamp is null, event job will start immediately in the next cycle - * startTimeStamp should be greater than current time - * @param executor event job executor @See {@link JobExecutor} - * @return job id - */ - Long registerJob(String name, Long intervalMs, Long startTimeStamp, JobExecutor executor) throws DdlException; - - - /** - * Register a event job - * - * @param name job name,it's not unique - * @param intervalMs job interval, unit: ms - * @param startTimeStamp job start time stamp, unit: ms - * if startTimeStamp is null, job will start immediately in the next cycle - * startTimeStamp should be greater than current time - * @param endTimeStamp job end time stamp, unit: ms - * if endTimeStamp is null, job will never stop - * endTimeStamp must be greater than startTimeStamp and endTimeStamp should be greater - * than current time - * @param executor event job executor @See {@link JobExecutor} - * @return event job id - */ - Long registerJob(String name, Long intervalMs, Long startTimeStamp, Long endTimeStamp, - JobExecutor executor) throws DdlException; - - /** - * if job is running, pause it - * pause means event job will not be executed in the next cycle,but current cycle will not be interrupted - * we can resume it by {@link #resumeJob(Long)} - * - * @param jodId job id - * if jobId not exist, return false - */ - void pauseJob(Long jodId); - - void pauseJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException; - - void resumeJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException; - - /** - * if job is running, stop it - * stop means event job will not be executed in the next cycle and current cycle will be interrupted - * stop not can be resumed, if you want to resume it, you should register it again - * we will delete stopped event job - * - * @param jobId event job id - */ - void stopJob(Long jobId); - - void stopJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException; - - /** - * if job is paused, resume it - * - * @param jobId job id - */ - void resumeJob(Long jobId); - - Long registerJob(Job job) throws DdlException; - - - /** - * execute job task immediately,this method will not change job status and don't affect scheduler job - * this task type should set to {@link org.apache.doris.scheduler.constants.TaskType#MANUAL_JOB_TASK} - * - * @param jobId job id - * @param contextData if you need to pass parameters to the task, - * @param context data type - * @return true if execute success, false if execute failed, - * if job is not exist or job is not running, or job not support manual execute, return false - */ - boolean immediateExecuteTask(Long jobId, T contextData) throws DdlException; - - List getJobs(String dbFullName, String jobName, JobCategory jobCategory, PatternMatcher matcher); - - /** - * close job scheduler register - * close means job scheduler register will not accept new job - * Jobs that have not reached the trigger time will not be executed. Jobs that have reached the trigger time will - * have an execution time of 5 seconds, and will not be executed if the time exceeds - */ - void close() throws IOException; - -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/TimerJobRegister.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/TimerJobRegister.java deleted file mode 100644 index f8ab59e5d54519..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/TimerJobRegister.java +++ /dev/null @@ -1,115 +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.scheduler.registry; - -import org.apache.doris.common.DdlException; -import org.apache.doris.common.PatternMatcher; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.executor.JobExecutor; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.manager.TimerJobManager; - -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.util.List; - -/** - * This class registers timed scheduling events using the Netty time wheel algorithm to trigger events in a timely - * manner. - * After the event is triggered, it is produced by the Disruptor producer and consumed by the consumer, which is an - * asynchronous - * consumption model that does not guarantee strict timing accuracy. - */ -@Slf4j -public class TimerJobRegister implements PersistentJobRegister { - - private final TimerJobManager timerJobManager; - - public TimerJobRegister(TimerJobManager timerJobManager) { - this.timerJobManager = timerJobManager; - } - - @Override - public Long registerJob(String name, Long intervalMs, JobExecutor executor) throws DdlException { - return this.registerJob(name, intervalMs, null, null, executor); - } - - @Override - public Long registerJob(String name, Long intervalMs, Long startTimeMs, JobExecutor executor) throws DdlException { - return this.registerJob(name, intervalMs, startTimeMs, null, executor); - } - - @Override - public Long registerJob(String name, Long intervalMs, Long startTimeMs, Long endTimeStamp, - JobExecutor executor) throws DdlException { - - Job job = new Job(name, intervalMs, startTimeMs, endTimeStamp, executor); - return timerJobManager.registerJob(job); - } - - @Override - public Long registerJob(Job job) throws DdlException { - return timerJobManager.registerJob(job); - } - - @Override - public boolean immediateExecuteTask(Long jobId, T data) throws DdlException { - return timerJobManager.immediateExecuteTask(jobId, data); - } - - @Override - public void pauseJob(Long jobId) { - timerJobManager.pauseJob(jobId); - } - - @Override - public void pauseJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException { - timerJobManager.pauseJob(dbName, jobName, jobCategory); - } - - @Override - public void resumeJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException { - timerJobManager.resumeJob(dbName, jobName, jobCategory); - } - - @Override - public void stopJob(Long jobId) { - timerJobManager.stopJob(jobId); - } - - @Override - public void stopJob(String dbName, String jobName, JobCategory jobCategory) throws DdlException { - timerJobManager.stopJob(dbName, jobName, jobCategory); - } - - @Override - public void resumeJob(Long jobId) { - timerJobManager.resumeJob(jobId); - } - - @Override - public List getJobs(String dbFullName, String jobName, JobCategory jobCategory, PatternMatcher matcher) { - return timerJobManager.queryJob(dbFullName, jobName, jobCategory, matcher); - } - - @Override - public void close() throws IOException { - timerJobManager.close(); - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/job/base/JobExecutionConfigurationTest.java b/fe/fe-core/src/test/java/org/apache/doris/job/base/JobExecutionConfigurationTest.java new file mode 100644 index 00000000000000..eadc6c567d3776 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/job/base/JobExecutionConfigurationTest.java @@ -0,0 +1,70 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.base; + +import org.apache.doris.job.common.IntervalUnit; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +public class JobExecutionConfigurationTest { + + @Test + public void testGetTriggerDelayTimesOneTime() { + JobExecutionConfiguration configuration = new JobExecutionConfiguration(); + configuration.setExecuteType(JobExecuteType.ONE_TIME); + + TimerDefinition timerDefinition = new TimerDefinition(); + timerDefinition.setStartTimeMs(System.currentTimeMillis() + 1000); // Start time set to 1 second in the future + configuration.setTimerDefinition(timerDefinition); + + List delayTimes = configuration.getTriggerDelayTimes( + System.currentTimeMillis(), System.currentTimeMillis(), System.currentTimeMillis() + 5000); + + Assertions.assertEquals(1, delayTimes.size()); + Assertions.assertEquals(1, delayTimes.get(0).longValue()); + } + + @Test + public void testGetTriggerDelayTimesRecurring() { + JobExecutionConfiguration configuration = new JobExecutionConfiguration(); + configuration.setExecuteType(JobExecuteType.RECURRING); + + TimerDefinition timerDefinition = new TimerDefinition(); + timerDefinition.setStartTimeMs(1000L); // Start time set to 1 second in the future + timerDefinition.setInterval(10L); // Interval set to 10 milliseconds + timerDefinition.setIntervalUnit(IntervalUnit.SECOND); + configuration.setTimerDefinition(timerDefinition); + + List delayTimes = configuration.getTriggerDelayTimes( + 0L, 0L, 11000L); + + Assertions.assertEquals(2, delayTimes.size()); + Assertions.assertArrayEquals(new Long[]{1L, 11L}, delayTimes.toArray()); + delayTimes = configuration.getTriggerDelayTimes( + 2000L, 0L, 11000L); + Assertions.assertEquals(1, delayTimes.size()); + Assertions.assertArrayEquals(new Long[]{ 9L}, delayTimes.toArray()); + delayTimes = configuration.getTriggerDelayTimes( + 1001L, 0L, 10000L); + Assertions.assertEquals(0, delayTimes.size()); + } + +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/JobTest.java b/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/JobTest.java deleted file mode 100644 index 57fbfda6cf933e..00000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/JobTest.java +++ /dev/null @@ -1,77 +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.scheduler.disruptor; - -import org.apache.doris.scheduler.common.IntervalUnit; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.constants.JobType; -import org.apache.doris.scheduler.executor.SqlJobExecutor; -import org.apache.doris.scheduler.job.Job; - -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; - -public class JobTest { - - private static Job job; - - @BeforeAll - public static void init() { - SqlJobExecutor sqlJobExecutor = new SqlJobExecutor("insert into test values(1);"); - job = new Job("insertTest", 1000L, System.currentTimeMillis(), System.currentTimeMillis() + 100000, sqlJobExecutor); - job.setJobType(JobType.RECURRING); - job.setComment("test"); - job.setOriginInterval(10L); - job.setIntervalUnit(IntervalUnit.SECOND); - job.setUser("root"); - job.setDbName("test"); - job.setTimezone("Asia/Shanghai"); - job.setJobCategory(JobCategory.SQL); - } - - @Test - public void testSerialization() throws IOException { - Path path = Paths.get("./scheduler-jobs"); - Files.deleteIfExists(path); - Files.createFile(path); - DataOutputStream dos = new DataOutputStream(Files.newOutputStream(path)); - job.write(dos); - dos.flush(); - dos.close(); - DataInputStream dis = new DataInputStream(Files.newInputStream(path)); - Job readJob = Job.readFields(dis); - Assertions.assertEquals(job.getJobName(), readJob.getJobName()); - Assertions.assertEquals(job.getTimezone(), readJob.getTimezone()); - - } - - @AfterAll - public static void clean() throws IOException { - Path path = Paths.get("./scheduler-jobs"); - Files.deleteIfExists(path); - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/TaskDisruptorTest.java b/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/TaskDisruptorTest.java deleted file mode 100644 index b8482b11fffcc1..00000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/TaskDisruptorTest.java +++ /dev/null @@ -1,95 +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.scheduler.disruptor; - -import org.apache.doris.catalog.Env; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.exception.JobException; -import org.apache.doris.scheduler.executor.JobExecutor; -import org.apache.doris.scheduler.job.ExecutorResult; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.job.JobTask; -import org.apache.doris.scheduler.manager.JobTaskManager; -import org.apache.doris.scheduler.manager.TimerJobManager; -import org.apache.doris.scheduler.manager.TransientTaskManager; - -import mockit.Expectations; -import mockit.Injectable; -import mockit.Mocked; -import mockit.Tested; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.concurrent.TimeUnit; - -public class TaskDisruptorTest { - - @Tested - private TaskDisruptor taskDisruptor; - - @Injectable - private TimerJobManager timerJobManager; - - @Injectable - private TransientTaskManager transientTaskManager; - - private static boolean testEventExecuteFlag = false; - - @Mocked - Env env; - - @BeforeEach - public void init() { - taskDisruptor = new TaskDisruptor(timerJobManager, transientTaskManager); - taskDisruptor.start(); - } - - @Test - void testPublishEventAndConsumer() { - Job job = new Job("test", 6000L, null, - null, new TestExecutor()); - JobTask jobTask = new JobTask(job.getJobId(), 1L, System.currentTimeMillis()); - JobTaskManager.addPrepareTask(jobTask); - job.setJobCategory(JobCategory.COMMON); - new Expectations() {{ - timerJobManager.getJob(anyLong); - result = job; - }}; - taskDisruptor.tryPublish(job.getJobId(), 1L); - Awaitility.await().atMost(3, TimeUnit.SECONDS).until(() -> testEventExecuteFlag); - Assertions.assertTrue(testEventExecuteFlag); - } - - - class TestExecutor implements JobExecutor { - - @Override - public ExecutorResult execute(Job job, String dataContext) throws JobException { - testEventExecuteFlag = true; - return new ExecutorResult(true, true, null, "null"); - } - } - - @AfterEach - public void after() { - taskDisruptor.close(); - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/TimerJobManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/TimerJobManagerTest.java deleted file mode 100644 index 5fcf242fd13566..00000000000000 --- a/fe/fe-core/src/test/java/org/apache/doris/scheduler/disruptor/TimerJobManagerTest.java +++ /dev/null @@ -1,182 +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.scheduler.disruptor; - -import org.apache.doris.catalog.Env; -import org.apache.doris.common.DdlException; -import org.apache.doris.persist.EditLog; -import org.apache.doris.scheduler.constants.JobCategory; -import org.apache.doris.scheduler.constants.JobType; -import org.apache.doris.scheduler.exception.JobException; -import org.apache.doris.scheduler.executor.JobExecutor; -import org.apache.doris.scheduler.job.ExecutorResult; -import org.apache.doris.scheduler.job.Job; -import org.apache.doris.scheduler.job.JobTask; -import org.apache.doris.scheduler.manager.TimerJobManager; -import org.apache.doris.scheduler.manager.TransientTaskManager; - -import lombok.extern.slf4j.Slf4j; -import mockit.Expectations; -import mockit.Mocked; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -@Slf4j -public class TimerJobManagerTest { - - TimerJobManager timerJobManager; - - @Mocked - EditLog editLog; - - private static AtomicInteger testExecuteCount = new AtomicInteger(0); - Job job = new Job("test", 6000L, null, - null, new TestExecutor()); - JobTask jobTask = new JobTask(job.getJobId(), 1L, System.currentTimeMillis()); - - @BeforeEach - public void init() { - job.setJobType(JobType.RECURRING); - job.setJobCategory(JobCategory.COMMON); - testExecuteCount.set(0); - timerJobManager = new TimerJobManager(); - TransientTaskManager transientTaskManager = new TransientTaskManager(); - TaskDisruptor taskDisruptor = new TaskDisruptor(this.timerJobManager, transientTaskManager); - this.timerJobManager.setDisruptor(taskDisruptor); - taskDisruptor.start(); - timerJobManager.start(); - } - - @Test - public void testCycleScheduler(@Mocked Env env) throws DdlException { - setContext(env); - timerJobManager.registerJob(job); - //consider the time of the first execution and give some buffer time - Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> testExecuteCount.get() >= 3); - } - - private void setContext(Env env) { - new Expectations() { - { - env.getEditLog(); - result = editLog; - editLog.logCreateJob((Job) any); - } - }; - } - - @Test - public void testCycleSchedulerAndStop(@Mocked Env env) throws DdlException { - setContext(env); - timerJobManager.registerJob(job); - long startTime = System.currentTimeMillis(); - Awaitility.await().atMost(8, TimeUnit.SECONDS).until(() -> testExecuteCount.get() >= 1); - timerJobManager.unregisterJob(job.getJobId()); - //consider the time of the first execution and give some buffer time - Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> System.currentTimeMillis() >= startTime + 13000L); - Assertions.assertEquals(1, testExecuteCount.get()); - } - - @Test - public void testCycleSchedulerWithIncludeStartTimeAndEndTime(@Mocked Env env) throws DdlException { - setContext(env); - job.setStartTimeMs(System.currentTimeMillis() + 6000L); - long endTimestamp = System.currentTimeMillis() + 19000L; - job.setEndTimeMs(endTimestamp); - timerJobManager.registerJob(job); - //consider the time of the first execution and give some buffer time - - Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> System.currentTimeMillis() - >= endTimestamp + 12000L); - Assertions.assertEquals(3, testExecuteCount.get()); - } - - @Test - public void testCycleSchedulerWithIncludeEndTime(@Mocked Env env) throws DdlException { - setContext(env); - long endTimestamp = System.currentTimeMillis() + 13000; - job.setEndTimeMs(endTimestamp); - timerJobManager.registerJob(job); - - //consider the time of the first execution and give some buffer time - Awaitility.await().atMost(36, TimeUnit.SECONDS).until(() -> System.currentTimeMillis() - >= endTimestamp + 12000L); - Assertions.assertEquals(2, testExecuteCount.get()); - } - - @Test - public void testCycleSchedulerWithIncludeStartTime(@Mocked Env env) throws DdlException { - setContext(env); - - long startTimestamp = System.currentTimeMillis() + 6000L; - job.setStartTimeMs(startTimestamp); - timerJobManager.registerJob(job); - //consider the time of the first execution and give some buffer time - Awaitility.await().atMost(14, TimeUnit.SECONDS).until(() -> System.currentTimeMillis() - >= startTimestamp + 7000L); - Assertions.assertEquals(2, testExecuteCount.get()); - } - - @Test - public void testCycleSchedulerWithImmediatelyStart(@Mocked Env env) throws DdlException { - setContext(env); - long startTimestamp = System.currentTimeMillis(); - job.setImmediatelyStart(true); - timerJobManager.registerJob(job); - //consider the time of the first execution and give some buffer time - Awaitility.await().atMost(16, TimeUnit.SECONDS).until(() -> System.currentTimeMillis() - >= startTimestamp + 15000L); - Assertions.assertEquals(3, testExecuteCount.get()); - } - - @Test - public void testOneTimeJob(@Mocked Env env) throws DdlException { - setContext(env); - - long startTimestamp = System.currentTimeMillis() + 3000L; - job.setIntervalMs(0L); - job.setStartTimeMs(startTimestamp); - job.setJobType(JobType.ONE_TIME); - timerJobManager.registerJob(job); - //consider the time of the first execution and give some buffer time - Awaitility.await().atMost(14, TimeUnit.SECONDS).until(() -> System.currentTimeMillis() - >= startTimestamp + 7000L); - Assertions.assertEquals(1, testExecuteCount.get()); - } - - @AfterEach - public void after() throws IOException { - timerJobManager.close(); - } - - class TestExecutor implements JobExecutor { - - @Override - public ExecutorResult execute(Job job, String dataContext) throws JobException { - log.info("test execute count:{}", testExecuteCount.incrementAndGet()); - return new ExecutorResult<>(true, true, null, ""); - } - } -}