phet commented on code in PR #3776: URL: https://github.com/apache/gobblin/pull/3776#discussion_r1332477363
########## gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagTaskStream.java: ########## @@ -0,0 +1,273 @@ +/* + * 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.gobblin.service.modules.orchestration; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingDeque; + +import com.codahale.metrics.Timer; +import com.google.common.base.Preconditions; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigException; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.config.ConfigBuilder; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.runtime.api.DagActionStore; +import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter; +import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; +import org.apache.gobblin.service.ExecutionStatus; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.processor.KillDagProc; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; +import org.apache.gobblin.service.modules.orchestration.task.KillDagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; +import org.apache.gobblin.service.modules.utils.FlowCompilationValidationHelper; +import org.apache.gobblin.service.monitoring.JobStatus; +import org.apache.gobblin.service.monitoring.JobStatusRetriever; + +import static org.apache.gobblin.service.ExecutionStatus.ORCHESTRATED; +import static org.apache.gobblin.service.ExecutionStatus.valueOf; + + +/** + * Holds a stream of {@link DagTask} that {@link DagManager} would pull from to process, as it is ready for more work. + * It provides an implementation for {@link DagManagement} defines the rules for a flow and job. + * Implements {@link Iterator} to provide the next {@link DagTask} if available to {@link DagManager} + */ + +@Alpha +@Slf4j +@AllArgsConstructor +public class DagTaskStream implements Iterator<Optional<DagTask>>, DagManagement { + + @Getter + private final BlockingDeque<DagActionStore.DagAction> dagActionQueue = new LinkedBlockingDeque<>(); + private FlowTriggerHandler flowTriggerHandler; + private DagManagementStateStore dagManagementStateStore; + private DagManagerMetrics dagManagerMetrics; + + + @Override + public boolean hasNext() { + return true; + } + + + @Override + public Optional<DagTask> next() { + + DagActionStore.DagAction dagAction = dagActionQueue.peek(); + try { + Preconditions.checkArgument(dagAction != null, "No Dag Action found in the queue"); + Properties jobProps = getJobProperties(dagAction); + Optional<MultiActiveLeaseArbiter.LeaseObtainedStatus> leaseObtainedStatus = + flowTriggerHandler.getLeaseOnDagAction(jobProps, dagAction, System.currentTimeMillis()).toJavaUtil(); + if(leaseObtainedStatus.isPresent()) { + DagTask dagTask = createDagTask(dagAction, leaseObtainedStatus.get()); + return Optional.of(dagTask); + } + } catch (Exception ex) { + //TODO: need to handle exceptions gracefully + throw new RuntimeException(ex); + } + return Optional.empty(); + } + + public boolean add(DagActionStore.DagAction dagAction) throws IOException { + return this.dagActionQueue.offer(dagAction); + } + + public DagActionStore.DagAction take() { + return this.dagActionQueue.poll(); + } + + public DagTask createDagTask(DagActionStore.DagAction dagAction, MultiActiveLeaseArbiter.LeaseObtainedStatus leaseObtainedStatus) { + DagActionStore.FlowActionType flowActionType = dagAction.getFlowActionType(); + switch (flowActionType) { + case KILL: + return new KillDagTask(dagAction, leaseObtainedStatus); + case RESUME: + case LAUNCH: + case ADVANCE: + default: + log.warn("It should not reach here. Yet to provide implementation."); + return null; + } + } + + @Override + public void launchFlow(String flowGroup, String flowName, long triggerTimeStamp) { + //TODO: provide implementation after finalizing code flow + throw new UnsupportedOperationException("Currently launch flow is not supported."); + } + + @Override + public void resumeFlow(String flowGroup, String flowName, String flowExecutionId, long triggerTimeStamp) throws IOException, InterruptedException { + //TODO: provide implementation after finalizing code flow + throw new UnsupportedOperationException("Currently launch flow is not supported."); + } + + @Override + public void killFlow(String flowGroup, String flowName, String flowExecutionId, long produceTimestamp) throws IOException { + DagManager.DagId dagId = DagManagerUtils.generateDagId(flowGroup, flowName, flowExecutionId); + if(!this.dagManagementStateStore.getDagIdToDags().containsKey(dagId.toString())) { + log.info("Invalid dag since not present in map. Hence cannot cancel it"); + return; + } + DagActionStore.DagAction killAction = new DagActionStore.DagAction(flowGroup, flowName, flowExecutionId, DagActionStore.FlowActionType.KILL); Review Comment: I would have though our caller would already have a `DagAction`... so could just call `.add(DagAction)`, right? the whole kill-specific pre-check against the `DagMgmtStateStore`... shouldn't that already have been done when the HTTP/REST kill request was handled? i.e. if it made it to the DB as a `DagAction`, doesn't that mean that it at least referred to a real dag that existed at one point in time? -- 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]
