morningman commented on code in PR #26356: URL: https://github.com/apache/doris/pull/26356#discussion_r1424224847
########## fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/load/LoadMgr.java: ########## @@ -0,0 +1,314 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.jobs.load; + +import org.apache.doris.analysis.CompoundPredicate; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.CaseSensibility; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.PatternMatcher; +import org.apache.doris.common.PatternMatcherWrapper; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.manager.JobManager; +import org.apache.doris.load.DataTransFormMgr; +import org.apache.doris.load.FailMsg; +import org.apache.doris.load.loadv2.JobState; +import org.apache.doris.nereids.jobs.load.replay.ReplayLoadLog; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * load manager + */ +public class LoadMgr extends DataTransFormMgr { + private static final Logger LOG = LogManager.getLogger(LoadMgr.class); + private Map<Long, InsertJob> loadIdToJob = new HashMap<>(); + private Map<String, Long> labelToLoadJobId = new HashMap<>(); + private Map<Long, Map<String, List<InsertJob>>> dbIdToLabelToLoadJobs = new ConcurrentHashMap<>(); + + private JobManager<InsertJob, ?> getJobManager() { + return Env.getCurrentEnv().getJobManager(); + } + + /** + * add load job and add tasks + * @param loadJob job + */ + public void addLoadJob(InsertJob loadJob) throws DdlException { + writeLock(); + try { + if (labelToLoadJobId.containsKey(loadJob.getLabel())) { Review Comment: Label is unique within a database. Should use `dbIdToLabelToLoadJobs` ########## fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java: ########## @@ -75,55 +118,147 @@ public class InsertJob extends AbstractJob<InsertTask, Map> { COLUMN_TO_INDEX = builder.build(); } - @SerializedName(value = "lp") - String labelPrefix; + @SerializedName("taskIdList") + ConcurrentLinkedQueue<Long> taskIdList; - InsertIntoTableCommand command; + private final long dbId; + private String labelName; + private List<InsertIntoTableCommand> plans; + private InsertJob.LoadType loadType; + // 0: the job status is pending + // n/100: n is the number of task which has been finished + // 99: all tasks have been finished + // 100: txn status is visible and load has been finished + private int progress; + private long createTimestamp = System.currentTimeMillis(); + private FailMsg failMsg; + private LoadStatistic loadStatistic = new LoadStatistic(); + private Set<Long> finishedTaskIds = new HashSet<>(); + private Set<String> tableNames; + private ConcurrentHashMap<Long, InsertTask> idToTasks = new ConcurrentHashMap<>(); + private Map<String, String> properties; + private AuthorizationInfo authorizationInfo; + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + + private ConnectContext ctx; + private StmtExecutor stmtExecutor; + private List<ErrorTabletInfo> errorTabletInfos = new ArrayList<>(); + private List<TabletCommitInfo> commitInfos = new ArrayList<>(); - StmtExecutor stmtExecutor; + // max save task num, do we need to config it? + private static final int MAX_SAVE_TASK_NUM = 100; - ConnectContext ctx; + /** + * load job type + */ + public enum LoadType { + BULK, + SPARK, + LOCAL_FILE, + UNKNOWN - @SerializedName("tis") - ConcurrentLinkedQueue<Long> taskIdList; + } - // max save task num, do we need to config it? - private static final int MAX_SAVE_TASK_NUM = 100; + public InsertJob(ReplayLoadLog.ReplayCreateLoadLog replayLoadLog) { + super(replayLoadLog.getId()); + setJobId(replayLoadLog.getId()); + this.dbId = replayLoadLog.getDbId(); + } + + public InsertJob(Long jobId, String jobName, + JobStatus jobStatus, + String currentDbName, + String comment, + UserIdentity createUser, + JobExecutionConfiguration jobConfig, + Long createTimeMs, + String executeSql) { + super(jobId, jobName, jobStatus, currentDbName, comment, createUser, Review Comment: You use `label` as job's name. But job's name need to be global unique. So I think job name should be `dbid+label+uuid` ########## fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/load/LoadMgr.java: ########## @@ -0,0 +1,314 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.jobs.load; + +import org.apache.doris.analysis.CompoundPredicate; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.CaseSensibility; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.LabelAlreadyUsedException; +import org.apache.doris.common.PatternMatcher; +import org.apache.doris.common.PatternMatcherWrapper; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; +import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.extensions.insert.InsertJob; +import org.apache.doris.job.manager.JobManager; +import org.apache.doris.load.DataTransFormMgr; +import org.apache.doris.load.FailMsg; +import org.apache.doris.load.loadv2.JobState; +import org.apache.doris.nereids.jobs.load.replay.ReplayLoadLog; + +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * load manager + */ +public class LoadMgr extends DataTransFormMgr { + private static final Logger LOG = LogManager.getLogger(LoadMgr.class); + private Map<Long, InsertJob> loadIdToJob = new HashMap<>(); + private Map<String, Long> labelToLoadJobId = new HashMap<>(); + private Map<Long, Map<String, List<InsertJob>>> dbIdToLabelToLoadJobs = new ConcurrentHashMap<>(); + + private JobManager<InsertJob, ?> getJobManager() { + return Env.getCurrentEnv().getJobManager(); + } + + /** + * add load job and add tasks + * @param loadJob job + */ + public void addLoadJob(InsertJob loadJob) throws DdlException { + writeLock(); + try { + if (labelToLoadJobId.containsKey(loadJob.getLabel())) { + throw new LabelAlreadyUsedException(loadJob.getLabel()); + } + unprotectAddJob(loadJob); + Env.getCurrentEnv().getEditLog().logLoadCreate(ReplayLoadLog.logCreateLoadOperation(loadJob)); Review Comment: In `unprotectAddJob->registerJob()`, there is an edit log. And here is another editlog. I think we should merge these 2 logs ########## fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java: ########## @@ -75,55 +118,147 @@ public class InsertJob extends AbstractJob<InsertTask, Map> { COLUMN_TO_INDEX = builder.build(); } - @SerializedName(value = "lp") - String labelPrefix; + @SerializedName("taskIdList") + ConcurrentLinkedQueue<Long> taskIdList; - InsertIntoTableCommand command; + private final long dbId; + private String labelName; + private List<InsertIntoTableCommand> plans; + private InsertJob.LoadType loadType; + // 0: the job status is pending + // n/100: n is the number of task which has been finished + // 99: all tasks have been finished + // 100: txn status is visible and load has been finished + private int progress; + private long createTimestamp = System.currentTimeMillis(); + private FailMsg failMsg; + private LoadStatistic loadStatistic = new LoadStatistic(); + private Set<Long> finishedTaskIds = new HashSet<>(); + private Set<String> tableNames; + private ConcurrentHashMap<Long, InsertTask> idToTasks = new ConcurrentHashMap<>(); + private Map<String, String> properties; + private AuthorizationInfo authorizationInfo; + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + + private ConnectContext ctx; + private StmtExecutor stmtExecutor; + private List<ErrorTabletInfo> errorTabletInfos = new ArrayList<>(); + private List<TabletCommitInfo> commitInfos = new ArrayList<>(); - StmtExecutor stmtExecutor; + // max save task num, do we need to config it? + private static final int MAX_SAVE_TASK_NUM = 100; - ConnectContext ctx; + /** + * load job type + */ + public enum LoadType { + BULK, + SPARK, + LOCAL_FILE, + UNKNOWN - @SerializedName("tis") - ConcurrentLinkedQueue<Long> taskIdList; + } - // max save task num, do we need to config it? - private static final int MAX_SAVE_TASK_NUM = 100; + public InsertJob(ReplayLoadLog.ReplayCreateLoadLog replayLoadLog) { + super(replayLoadLog.getId()); + setJobId(replayLoadLog.getId()); + this.dbId = replayLoadLog.getDbId(); + } + + public InsertJob(Long jobId, String jobName, + JobStatus jobStatus, + String currentDbName, + String comment, + UserIdentity createUser, + JobExecutionConfiguration jobConfig, + Long createTimeMs, + String executeSql) { + super(jobId, jobName, jobStatus, currentDbName, comment, createUser, + jobConfig, createTimeMs, executeSql, null); + this.dbId = ConnectContext.get().getCurrentDbId(); + } + + public InsertJob(ConnectContext ctx, + StmtExecutor executor, + String labelName, + List<InsertIntoTableCommand> plans, + Set<String> sinkTableNames, + Map<String, String> properties, + String comment, + JobExecutionConfiguration jobConfig) { + super(Env.getCurrentEnv().getNextId(), labelName, JobStatus.RUNNING, null, + comment, ctx.getCurrentUserIdentity(), jobConfig); + this.ctx = ctx; + this.plans = plans; + this.stmtExecutor = executor; + this.dbId = ctx.getCurrentDbId(); + this.labelName = labelName; + this.tableNames = sinkTableNames; + this.properties = properties; + // TODO: not support other type yet + this.loadType = InsertJob.LoadType.BULK; + } @Override public List<InsertTask> createTasks(TaskType taskType, Map taskContext) { - //nothing need to do in insert job - InsertTask task = new InsertTask(null, getCurrentDbName(), getExecuteSql(), getCreateUser()); - task.setJobId(getJobId()); - task.setTaskType(taskType); - task.setTaskId(Env.getCurrentEnv().getNextId()); + if (plans.isEmpty()) { + InsertTask task = new InsertTask(labelName, getCurrentDbName(), getExecuteSql(), getCreateUser()); + task.setJobId(getJobId()); + task.setTaskType(taskType); + task.setTaskId(Env.getCurrentEnv().getNextId()); + ArrayList<InsertTask> tasks = new ArrayList<>(); + tasks.add(task); + super.initTasks(tasks); + addNewTask(task.getTaskId()); + return tasks; + } else { + return createBatchTasks(taskType); + } + } + + private List<InsertTask> createBatchTasks(TaskType taskType) { ArrayList<InsertTask> tasks = new ArrayList<>(); - tasks.add(task); - super.initTasks(tasks); - addNewTask(task.getTaskId()); - return tasks; + for (InsertIntoTableCommand logicalPlan : plans) { + InsertTask task = new InsertTask(logicalPlan, ctx, stmtExecutor, loadStatistic); + task.setJobId(getJobId()); + task.setTaskType(taskType); + idToTasks.put(task.getTaskId(), task); + initTasks(tasks); + } + return new ArrayList<>(idToTasks.values()); } public void addNewTask(long id) { Review Comment: ```suggestion private void addNewTask(long id) { ``` ########## fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java: ########## @@ -101,6 +105,14 @@ public void setJobId(long jobId) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + runInternal(ctx, executor); + } + + public void statefulRun(ConnectContext ctx, StmtExecutor executor) throws Exception { Review Comment: Looks like `statefulRun()` is same as `run()`? ########## fe/fe-core/src/main/java/org/apache/doris/load/DataTransFormMgr.java: ########## @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.load; + +import java.util.concurrent.locks.ReentrantReadWriteLock; + +public abstract class DataTransFormMgr { Review Comment: What is this class for? ########## fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java: ########## @@ -75,55 +118,147 @@ public class InsertJob extends AbstractJob<InsertTask, Map> { COLUMN_TO_INDEX = builder.build(); } - @SerializedName(value = "lp") - String labelPrefix; + @SerializedName("taskIdList") + ConcurrentLinkedQueue<Long> taskIdList; - InsertIntoTableCommand command; + private final long dbId; + private String labelName; + private List<InsertIntoTableCommand> plans; + private InsertJob.LoadType loadType; + // 0: the job status is pending + // n/100: n is the number of task which has been finished + // 99: all tasks have been finished + // 100: txn status is visible and load has been finished + private int progress; + private long createTimestamp = System.currentTimeMillis(); + private FailMsg failMsg; + private LoadStatistic loadStatistic = new LoadStatistic(); + private Set<Long> finishedTaskIds = new HashSet<>(); + private Set<String> tableNames; + private ConcurrentHashMap<Long, InsertTask> idToTasks = new ConcurrentHashMap<>(); + private Map<String, String> properties; + private AuthorizationInfo authorizationInfo; + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + + private ConnectContext ctx; + private StmtExecutor stmtExecutor; + private List<ErrorTabletInfo> errorTabletInfos = new ArrayList<>(); + private List<TabletCommitInfo> commitInfos = new ArrayList<>(); - StmtExecutor stmtExecutor; + // max save task num, do we need to config it? + private static final int MAX_SAVE_TASK_NUM = 100; - ConnectContext ctx; + /** + * load job type + */ + public enum LoadType { + BULK, + SPARK, + LOCAL_FILE, + UNKNOWN - @SerializedName("tis") - ConcurrentLinkedQueue<Long> taskIdList; + } - // max save task num, do we need to config it? - private static final int MAX_SAVE_TASK_NUM = 100; + public InsertJob(ReplayLoadLog.ReplayCreateLoadLog replayLoadLog) { + super(replayLoadLog.getId()); + setJobId(replayLoadLog.getId()); + this.dbId = replayLoadLog.getDbId(); + } + + public InsertJob(Long jobId, String jobName, + JobStatus jobStatus, + String currentDbName, + String comment, + UserIdentity createUser, + JobExecutionConfiguration jobConfig, + Long createTimeMs, + String executeSql) { + super(jobId, jobName, jobStatus, currentDbName, comment, createUser, + jobConfig, createTimeMs, executeSql, null); + this.dbId = ConnectContext.get().getCurrentDbId(); + } + + public InsertJob(ConnectContext ctx, + StmtExecutor executor, + String labelName, + List<InsertIntoTableCommand> plans, + Set<String> sinkTableNames, + Map<String, String> properties, + String comment, + JobExecutionConfiguration jobConfig) { + super(Env.getCurrentEnv().getNextId(), labelName, JobStatus.RUNNING, null, + comment, ctx.getCurrentUserIdentity(), jobConfig); + this.ctx = ctx; + this.plans = plans; + this.stmtExecutor = executor; + this.dbId = ctx.getCurrentDbId(); + this.labelName = labelName; + this.tableNames = sinkTableNames; + this.properties = properties; + // TODO: not support other type yet + this.loadType = InsertJob.LoadType.BULK; + } @Override public List<InsertTask> createTasks(TaskType taskType, Map taskContext) { - //nothing need to do in insert job - InsertTask task = new InsertTask(null, getCurrentDbName(), getExecuteSql(), getCreateUser()); - task.setJobId(getJobId()); - task.setTaskType(taskType); - task.setTaskId(Env.getCurrentEnv().getNextId()); + if (plans.isEmpty()) { + InsertTask task = new InsertTask(labelName, getCurrentDbName(), getExecuteSql(), getCreateUser()); + task.setJobId(getJobId()); + task.setTaskType(taskType); + task.setTaskId(Env.getCurrentEnv().getNextId()); + ArrayList<InsertTask> tasks = new ArrayList<>(); + tasks.add(task); + super.initTasks(tasks); + addNewTask(task.getTaskId()); + return tasks; + } else { + return createBatchTasks(taskType); + } + } + + private List<InsertTask> createBatchTasks(TaskType taskType) { Review Comment: Why not call `addNewTask()` in `createBatchTasks()`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
