[
https://issues.apache.org/jira/browse/GOBBLIN-1910?focusedWorklogId=881069&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-881069
]
ASF GitHub Bot logged work on GOBBLIN-1910:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 21/Sep/23 05:57
Start Date: 21/Sep/23 05:57
Worklog Time Spent: 10m
Work Description: phet commented on code in PR #3776:
URL: https://github.com/apache/gobblin/pull/3776#discussion_r1332438770
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.typesafe.config.Config;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.modules.orchestration.processor.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Alpha
+@Slf4j
+public class DagProcessingEngine {
+
+ public static final String DAG_PROCESSING_ENGINE_PREFIX =
"gobblin.service.dagProcessingEngine.";
Review Comment:
doesn't this need to align w/ what's in `ServiceConfigKeys`? if so, let's
define the prefix over there and bring it in here.
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java:
##########
@@ -0,0 +1,33 @@
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.Map;
+
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+
+
+/**
+ * An interface to provide abstractions for managing {@link Dag} and {@link
org.apache.gobblin.service.modules.flowgraph.Dag.DagNode} states
+ */
+public interface DagManagementStateStore {
+
+ public void addJobState(String dagId, Dag.DagNode<JobExecutionPlan> dagNode);
+
+ public void deleteJobState(String dagId, Dag.DagNode<JobExecutionPlan>
dagNode);
+
+ public boolean hasRunningJobs(String dagId);
+
+ public void removeDagActionFromStore(DagManager.DagId dagId,
DagActionStore.FlowActionType flowActionType) throws IOException;
+
+ public Map<String, LinkedList<Dag.DagNode<JobExecutionPlan>>> getDagToJobs();
+
+ public Map<String, Dag<JobExecutionPlan>> getDagIdToDags();
+
+ public Map<String, Long> getDagToSLA();
Review Comment:
for perf, I doubt we'd want to return such massive collections. is it truly
necessary to have all the mappings at once? couldn't we instead return only
the one related to a particular dag ID?
e.g.:
```
Dag<JobExecutionPlan> getDags(String dagId);
Long getSLA(String dagId);
```
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.typesafe.config.Config;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.modules.orchestration.processor.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Alpha
+@Slf4j
+public class DagProcessingEngine {
+
+ public static final String DAG_PROCESSING_ENGINE_PREFIX =
"gobblin.service.dagProcessingEngine.";
+ public static final String NUM_THREADS_KEY = DAG_PROCESSING_ENGINE_PREFIX +
"numThreads";
+ public static final String JOB_STATUS_POLLING_INTERVAL_KEY =
DAG_PROCESSING_ENGINE_PREFIX + "pollingInterval";
+
+ private static final Integer DEFAULT_NUM_THREADS = 3;
+ private static final Integer DEFAULT_JOB_STATUS_POLLING_INTERVAL = 10;
+
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+ private ScheduledExecutorService scheduledExecutorPool;
+ private Config config;
+ private Integer numThreads;
+ private Integer pollingInterval;
+ private DagProcessingEngine.Thread [] threads;
+ private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
Review Comment:
many of these seem good candidates for `final`. what's your take?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagerUtils.java:
##########
@@ -108,7 +108,7 @@ static DagManager.DagId generateDagId(String flowGroup,
String flowName, long fl
return generateDagId(flowGroup, flowName, String.valueOf(flowExecutionId));
}
- static DagManager.DagId generateDagId(String flowGroup, String flowName,
String flowExecutionId) {
+ public static DagManager.DagId generateDagId(String flowGroup, String
flowName, String flowExecutionId) {
Review Comment:
NBD, but I don't find a compelling reason for this, as:
```
new DagManager.DagId(x, y, z)
```
is not much different than:
```
generateDagId(x, y, z);
```
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.typesafe.config.Config;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.modules.orchestration.processor.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Alpha
+@Slf4j
+public class DagProcessingEngine {
+
+ public static final String DAG_PROCESSING_ENGINE_PREFIX =
"gobblin.service.dagProcessingEngine.";
+ public static final String NUM_THREADS_KEY = DAG_PROCESSING_ENGINE_PREFIX +
"numThreads";
+ public static final String JOB_STATUS_POLLING_INTERVAL_KEY =
DAG_PROCESSING_ENGINE_PREFIX + "pollingInterval";
+
+ private static final Integer DEFAULT_NUM_THREADS = 3;
+ private static final Integer DEFAULT_JOB_STATUS_POLLING_INTERVAL = 10;
+
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+ private ScheduledExecutorService scheduledExecutorPool;
+ private Config config;
+ private Integer numThreads;
+ private Integer pollingInterval;
+ private DagProcessingEngine.Thread [] threads;
+ private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+
+
+ public DagProcessingEngine(Config config, DagTaskStream dagTaskStream,
DagProcFactory dagProcFactory, DagManagementStateStore dagManagementStateStore,
MultiActiveLeaseArbiter multiActiveLeaseArbiter) {
+ this.config = config;
+ this.dagTaskStream = dagTaskStream;
+ this.dagProcFactory = dagProcFactory;
+ this.dagManagementStateStore = dagManagementStateStore;
+ this.multiActiveLeaseArbiter = multiActiveLeaseArbiter;
+ this.numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.pollingInterval = ConfigUtils.getInt(config,
JOB_STATUS_POLLING_INTERVAL_KEY, DEFAULT_JOB_STATUS_POLLING_INTERVAL);
+ this.threads = new DagProcessingEngine.Thread[this.numThreads];
+ for(int i=0; i < this.numThreads; i++) {
+ Thread thread = new Thread(dagTaskStream, dagProcFactory,
dagManagementStateStore, multiActiveLeaseArbiter);
+ this.threads[i] = thread;
+ }
+ }
+
+
+ @AllArgsConstructor
+ private static class Thread implements Runnable {
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+ private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+
+ @Override
+ public void run() {
+ try {
+ while (dagTaskStream.hasNext()) {
+ Optional<DagTask> dagTask = getNextTask();
+ if (dagTask.isPresent()) {
+ DagProc dagProc = (DagProc) dagTask.get().host(dagProcFactory);
+ dagProc.process(dagManagementStateStore);
+ //marks lease success and releases it
+ dagTask.get().conclude(multiActiveLeaseArbiter);
Review Comment:
to encapsulate lease arbitration from this current class, would there be any
downsides to reformulating as:
```
dagTaskStream.complete(dagTask.get());
```
?
that way the MALeaseArbiter need not even be mentioned in this file
##########
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<>();
Review Comment:
why use a deque instead of a queue? the name `dagActionQueue` becomes
especially confusing
##########
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");
Review Comment:
having `Preconditions` throw an exception when the queue/deque is
temporarily empty is definitely not what we want. do you want `take()` rather
than `peek()`?
##########
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.
Review Comment:
actually, looks like it's pulled by `DagProcessingEngine`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.typesafe.config.Config;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.modules.orchestration.processor.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Alpha
+@Slf4j
+public class DagProcessingEngine {
+
+ public static final String DAG_PROCESSING_ENGINE_PREFIX =
"gobblin.service.dagProcessingEngine.";
+ public static final String NUM_THREADS_KEY = DAG_PROCESSING_ENGINE_PREFIX +
"numThreads";
+ public static final String JOB_STATUS_POLLING_INTERVAL_KEY =
DAG_PROCESSING_ENGINE_PREFIX + "pollingInterval";
+
+ private static final Integer DEFAULT_NUM_THREADS = 3;
+ private static final Integer DEFAULT_JOB_STATUS_POLLING_INTERVAL = 10;
+
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+ private ScheduledExecutorService scheduledExecutorPool;
+ private Config config;
+ private Integer numThreads;
+ private Integer pollingInterval;
+ private DagProcessingEngine.Thread [] threads;
+ private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+
+
+ public DagProcessingEngine(Config config, DagTaskStream dagTaskStream,
DagProcFactory dagProcFactory, DagManagementStateStore dagManagementStateStore,
MultiActiveLeaseArbiter multiActiveLeaseArbiter) {
+ this.config = config;
+ this.dagTaskStream = dagTaskStream;
+ this.dagProcFactory = dagProcFactory;
+ this.dagManagementStateStore = dagManagementStateStore;
+ this.multiActiveLeaseArbiter = multiActiveLeaseArbiter;
+ this.numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.pollingInterval = ConfigUtils.getInt(config,
JOB_STATUS_POLLING_INTERVAL_KEY, DEFAULT_JOB_STATUS_POLLING_INTERVAL);
+ this.threads = new DagProcessingEngine.Thread[this.numThreads];
+ for(int i=0; i < this.numThreads; i++) {
+ Thread thread = new Thread(dagTaskStream, dagProcFactory,
dagManagementStateStore, multiActiveLeaseArbiter);
+ this.threads[i] = thread;
+ }
+ }
+
+
+ @AllArgsConstructor
+ private static class Thread implements Runnable {
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+ private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+
+ @Override
+ public void run() {
+ try {
+ while (dagTaskStream.hasNext()) {
Review Comment:
(can't recall if I mentioned this already...)
we expect the task stream to be infinite/forever. given that, a nice way to
write this is:
```
for (DagTask dagTask : dagTaskStream) {
...
}
```
that syntax is possible if we change to
```
class DagTaskStream implements Iterable<DagTask>
```
(rather than `Iterator<DagTask>`)
the `iterator()` method could return an anon inner class realizing
`Iterator<DagTask>`, where `hasNext()` returns always true
##########
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) {
Review Comment:
why is this `public`?
##########
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();
+ }
Review Comment:
I can't think of who should be calling this public method... who do you have
in mind?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/processor/DagProc.java:
##########
Review Comment:
nit: a subpackage seems reasonable, but given all of these are called
"proc", the name "processor" doesn't mesh
##########
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);
+ if(!add(killAction)) {
+ throw new IOException("Could not add kill dag action: " + killAction +
" to the queue.");
+ }
+ }
+ /**
+ * Check if the SLA is configured for the flow this job belongs to.
+ * If it is, this method will try to cancel the job when SLA is reached.
+ *
+ * @param node dag node of the job
+ * @return true if the job is killed because it reached sla
+ * @throws ExecutionException exception
+ * @throws InterruptedException exception
+ */
+
+ @Override
+ public boolean enforceFlowCompletionDeadline(Dag.DagNode<JobExecutionPlan>
node) throws ExecutionException, InterruptedException {
Review Comment:
is this called when the time is up and it's actually time to enforce the
deadline or earlier... perhaps when the first job of the flow is launched?
I don't find any callers ATM, so unclear what you have in mind...
##########
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 request to kill 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?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/KillDagTask.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.task;
+
+import lombok.Getter;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.modules.orchestration.DagManager;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.DagTaskVisitor;
+
+
+/**
+ * A {@link DagTask} responsible to handle kill tasks.
+ */
+@Alpha
+public class KillDagTask extends DagTask {
+
+ @Getter
+ private final DagManager.DagId killDagId;
+ protected final DagActionStore.DagAction killAction;
+
+ public KillDagTask(DagActionStore.DagAction killAction,
MultiActiveLeaseArbiter.LeaseObtainedStatus leaseObtainedStatus) {
+
+ this.killAction = killAction;
+ this.leaseObtainedStatusStatus = leaseObtainedStatus;
+ this.killDagId = DagManagerUtils.generateDagId(killAction.getFlowGroup(),
killAction.getFlowName(), killAction.getFlowExecutionId());
+ }
+
+ @Override
+ public Object host(DagTaskVisitor visitor) throws Exception {
Review Comment:
shouldn't the signature be:
```
public DagProc host(DagTaskVisitor<DagProc> visitor);
```
?
also, what is the exception we thing the `DagTaskVisitor.meet` method would
throw?
##########
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);
+ if(!add(killAction)) {
+ throw new IOException("Could not add kill dag action: " + killAction +
" to the queue.");
+ }
+ }
+ /**
+ * Check if the SLA is configured for the flow this job belongs to.
+ * If it is, this method will try to cancel the job when SLA is reached.
+ *
+ * @param node dag node of the job
+ * @return true if the job is killed because it reached sla
+ * @throws ExecutionException exception
+ * @throws InterruptedException exception
+ */
+
+ @Override
+ public boolean enforceFlowCompletionDeadline(Dag.DagNode<JobExecutionPlan>
node) throws ExecutionException, InterruptedException {
+ //TODO: need to distribute the responsibility outside of this class
+ long flowStartTime = DagManagerUtils.getFlowStartTime(node);
+ long currentTime = System.currentTimeMillis();
+ String dagId = DagManagerUtils.generateDagId(node).toString();
+
+ long flowSla;
+ if (this.dagManagementStateStore.getDagToSLA().containsKey(dagId)) {
+ flowSla = this.dagManagementStateStore.getDagToSLA().get(dagId);
+ } else {
+ try {
+ flowSla = DagManagerUtils.getFlowSLA(node);
+ } catch (ConfigException e) {
+ log.warn("Flow SLA for flowGroup: {}, flowName: {} is given in invalid
format, using default SLA of {}",
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_GROUP_KEY),
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY),
+ DagManagerUtils.DEFAULT_FLOW_SLA_MILLIS);
+ flowSla = DagManagerUtils.DEFAULT_FLOW_SLA_MILLIS;
+ }
+ this.dagManagementStateStore.getDagToSLA().put(dagId, flowSla);
+ }
+
+ if (currentTime > flowStartTime + flowSla) {
+ log.info("Flow {} exceeded the SLA of {} ms. Killing the job {} now...",
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY),
flowSla,
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.JOB_NAME_KEY));
+ dagManagerMetrics.incrementExecutorSlaExceeded(node);
+ KillDagProc.killDagNode(node);
Review Comment:
is this actually performing a kill? if so, how do we coordinate so not all
hosts try to perform it? I expected us to try to acquire a MA lease that would
bestow the exclusive responsibility for handling this
##########
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);
+ if(!add(killAction)) {
+ throw new IOException("Could not add kill dag action: " + killAction +
" to the queue.");
+ }
+ }
+ /**
+ * Check if the SLA is configured for the flow this job belongs to.
+ * If it is, this method will try to cancel the job when SLA is reached.
+ *
+ * @param node dag node of the job
+ * @return true if the job is killed because it reached sla
+ * @throws ExecutionException exception
+ * @throws InterruptedException exception
+ */
+
+ @Override
+ public boolean enforceFlowCompletionDeadline(Dag.DagNode<JobExecutionPlan>
node) throws ExecutionException, InterruptedException {
+ //TODO: need to distribute the responsibility outside of this class
+ long flowStartTime = DagManagerUtils.getFlowStartTime(node);
+ long currentTime = System.currentTimeMillis();
+ String dagId = DagManagerUtils.generateDagId(node).toString();
+
+ long flowSla;
+ if (this.dagManagementStateStore.getDagToSLA().containsKey(dagId)) {
+ flowSla = this.dagManagementStateStore.getDagToSLA().get(dagId);
+ } else {
+ try {
+ flowSla = DagManagerUtils.getFlowSLA(node);
+ } catch (ConfigException e) {
+ log.warn("Flow SLA for flowGroup: {}, flowName: {} is given in invalid
format, using default SLA of {}",
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_GROUP_KEY),
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY),
+ DagManagerUtils.DEFAULT_FLOW_SLA_MILLIS);
+ flowSla = DagManagerUtils.DEFAULT_FLOW_SLA_MILLIS;
+ }
+ this.dagManagementStateStore.getDagToSLA().put(dagId, flowSla);
+ }
+
+ if (currentTime > flowStartTime + flowSla) {
+ log.info("Flow {} exceeded the SLA of {} ms. Killing the job {} now...",
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY),
flowSla,
+
node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.JOB_NAME_KEY));
+ dagManagerMetrics.incrementExecutorSlaExceeded(node);
+ KillDagProc.killDagNode(node);
+
+
this.dagManagementStateStore.getDagIdToDags().get(dagId).setFlowEvent(TimingEvent.FlowTimings.FLOW_RUN_DEADLINE_EXCEEDED);
+
this.dagManagementStateStore.getDagIdToDags().get(dagId).setMessage("Flow
killed due to exceeding SLA of " + flowSla + " ms");
+
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Cancel the job if the job has been "orphaned". A job is orphaned if has
been in ORCHESTRATED
+ * {@link ExecutionStatus} for some specific amount of time.
+ * @param node {@link Dag.DagNode} representing the job
+ * @param jobStatus current {@link JobStatus} of the job
+ * @return true if the total time that the job remains in the ORCHESTRATED
state exceeds
+ * {@value ConfigurationKeys#GOBBLIN_JOB_START_SLA_TIME}.
+ */
+
+ @Override
+ public boolean enforceJobStartDeadline(Dag.DagNode<JobExecutionPlan> node,
JobStatus jobStatus) throws ExecutionException, InterruptedException {
+ //TODO: need to distribute the responsibility outside of this class
+ if (jobStatus == null) {
+ return false;
+ }
+ ExecutionStatus executionStatus = valueOf(jobStatus.getEventName());
+ //TODO: initialize default job sla in millis via configs
+ long timeOutForJobStart = DagManagerUtils.getJobStartSla(node,
System.currentTimeMillis());
+ long jobOrchestratedTime = jobStatus.getOrchestratedTime();
+ if (executionStatus == ORCHESTRATED && System.currentTimeMillis() -
jobOrchestratedTime > timeOutForJobStart) {
+ log.info("Job {} of flow {} exceeded the job start SLA of {} ms. Killing
the job now...",
+ DagManagerUtils.getJobName(node),
+ DagManagerUtils.getFullyQualifiedDagName(node),
+ timeOutForJobStart);
+ dagManagerMetrics.incrementCountsStartSlaExceeded(node);
+ KillDagProc.killDagNode(node);
+
+ String dagId = DagManagerUtils.generateDagId(node).toString();
+
dagManagementStateStore.getDagIdToDags().get(dagId).setFlowEvent(TimingEvent.FlowTimings.FLOW_START_DEADLINE_EXCEEDED);
+ dagManagementStateStore.getDagIdToDags().get(dagId).setMessage("Flow
killed because no update received for " + timeOutForJobStart + " ms after
orchestration");
+ return true;
+ } else {
+ return false;
+ }
+
+ }
+
+ /**
+ * Retrieve the {@link JobStatus} from the {@link JobExecutionPlan}.
+ */
+
+ protected JobStatus retrieveJobStatus(Dag.DagNode<JobExecutionPlan> dagNode)
{
Review Comment:
why should this live in `DagTaskStream`? should it be a method of
`DagManagementStateStore`?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/AdvanceDagTask.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.gobblin.service.modules.orchestration.task;
+
+
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.service.modules.orchestration.DagTaskVisitor;
+
+
+/**
+ * An implementation of {@link DagTask} that is responsible for advancing the
dag to the next node based
+ * on its current flow and job status. It is added to the {@link
org.apache.gobblin.service.modules.orchestration.DagTaskStream}
+ * by the {@link org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor}
after it consumes the appropriate
+ * {@link org.apache.gobblin.metrics.GobblinTrackingEvent} for the {@link
org.apache.gobblin.service.modules.flowgraph.Dag}
+ */
+
+@Alpha
+@Slf4j
+public class AdvanceDagTask extends DagTask {
+
+ @Override
+ public Object host(DagTaskVisitor visitor) throws Exception {
+ throw new UnsupportedOperationException("Not supported");
Review Comment:
given these impls are only one line... why not provide?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/processor/KillDagProc.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.processor;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import com.google.api.client.util.Lists;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.event.EventSubmitter;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.TimingEventUtils;
+import
org.apache.gobblin.service.modules.orchestration.exception.MaybeRetryableException;
+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 static org.apache.gobblin.service.ExecutionStatus.CANCELLED;
+
+
+/**
+ * An implementation of {@link DagProc} for killing {@link DagTask}.
+ */
+@Slf4j
+@Alpha
+public final class KillDagProc extends DagProc {
+
+ private KillDagTask killDagTask;
+ private DagManagementStateStore dagManagementStateStore;
Review Comment:
the `process` method already takes this as a param... do we need to store
one in addition to what will be passed to us?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/processor/KillDagProc.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.processor;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import com.google.api.client.util.Lists;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.event.EventSubmitter;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.TimingEventUtils;
+import
org.apache.gobblin.service.modules.orchestration.exception.MaybeRetryableException;
+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 static org.apache.gobblin.service.ExecutionStatus.CANCELLED;
+
+
+/**
+ * An implementation of {@link DagProc} for killing {@link DagTask}.
+ */
+@Slf4j
+@Alpha
+public final class KillDagProc extends DagProc {
Review Comment:
recall that it's `DagProc<S, R>` decide what those types are and then use
them in the various method signatures (that currently degraded to `Object`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.task;
+
+import java.io.IOException;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagTaskVisitor;
+import org.apache.gobblin.service.modules.orchestration.processor.DagProc;
+
+
+/**
+ * Defines an individual task or job in a Dag.
+ * Upon completion of the {@link DagProc#process(DagManagementStateStore)} it
will mark the lease
+ * acquired by {@link org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter}
as complete
+ * @param <T>
+ */
+
+@Alpha
+public abstract class DagTask<T> {
Review Comment:
let's make `host` a generic method rather than `DagTask` a generic class
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcFactory.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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 com.google.common.base.Optional;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.event.EventSubmitter;
+import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
+import org.apache.gobblin.service.modules.flow.SpecCompiler;
+import org.apache.gobblin.service.modules.orchestration.processor.DagProc;
+import org.apache.gobblin.service.modules.orchestration.processor.KillDagProc;
+import org.apache.gobblin.service.modules.orchestration.task.AdvanceDagTask;
+import org.apache.gobblin.service.modules.orchestration.task.CleanUpDagTask;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.service.modules.orchestration.task.KillDagTask;
+import org.apache.gobblin.service.modules.orchestration.task.LaunchDagTask;
+import org.apache.gobblin.service.modules.orchestration.task.ResumeDagTask;
+import
org.apache.gobblin.service.modules.utils.FlowCompilationValidationHelper;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Factory for creating {@link DagProc} based on the visitor type for a given
{@link DagTask}.
+ */
+
+@Alpha
+@Slf4j
+public class DagProcFactory implements DagTaskVisitor<DagProc> {
+
+ private DagManagementStateStore dagManagementStateStore;
+ private JobStatusRetriever jobStatusRetriever;
+ private FlowStatusGenerator flowStatusGenerator;
+ private UserQuotaManager quotaManager;
+ private SpecCompiler specCompiler;
+ private FlowCatalog flowCatalog;
+ private FlowCompilationValidationHelper flowCompilationValidationHelper;
+ private Config config;
+ private Optional<EventSubmitter> eventSubmitter;
+ private boolean instrumentationEnabled;
+
+ public DagProcFactory(Config config, DagManagementStateStore
dagManagementStateStore, JobStatusRetriever jobStatusRetriever,
+ FlowStatusGenerator flowStatusGenerator, UserQuotaManager quotaManager,
SpecCompiler specCompiler, FlowCatalog flowCatalog,
FlowCompilationValidationHelper flowCompilationValidationHelper, boolean
instrumentationEnabled) {
+
+ this.config = config;
+ this.dagManagementStateStore = dagManagementStateStore;
+ this.jobStatusRetriever = jobStatusRetriever;
+ this.flowStatusGenerator = flowStatusGenerator;
+ this.quotaManager = quotaManager;
+ this.specCompiler = specCompiler;
+ this.flowCatalog = flowCatalog;
+ this.flowCompilationValidationHelper = flowCompilationValidationHelper;
+ this.instrumentationEnabled = instrumentationEnabled;
+ MetricContext metricContext = null;
+ if (instrumentationEnabled) {
+ metricContext =
Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()),
getClass());
+ this.eventSubmitter = Optional.of(new
EventSubmitter.Builder(metricContext, "org.apache.gobblin.service").build());
+ } else {
+ this.eventSubmitter = Optional.absent();
+ }
+ }
+
+
+ @Override
+ public DagProc meet(LaunchDagTask launchDagTask) {
+ throw new UnsupportedOperationException("Currently cannot provide launch
proc");
+ }
+
+ @Override
+ public DagProc meet(KillDagTask killDagTask) {
+ return new KillDagProc(killDagTask);
+ }
+
+ @Override
+ public DagProc meet(ResumeDagTask resumeDagTask) {
+ throw new UnsupportedOperationException("Currently cannot provide resume
proc");
Review Comment:
although `ResumeDagProc` might still throw UOE when `process()` is
invoked... isn't enough scaffolding in place yet to write this as:
```
return new ResumeDagProc(resumeDagTask);
```
?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 com.google.common.base.Optional;
Review Comment:
let's avoid introducing more use of the deprecated guava `Optional`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.util.Optional;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.typesafe.config.Config;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.modules.orchestration.processor.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Alpha
+@Slf4j
+public class DagProcessingEngine {
+
+ public static final String DAG_PROCESSING_ENGINE_PREFIX =
"gobblin.service.dagProcessingEngine.";
+ public static final String NUM_THREADS_KEY = DAG_PROCESSING_ENGINE_PREFIX +
"numThreads";
+ public static final String JOB_STATUS_POLLING_INTERVAL_KEY =
DAG_PROCESSING_ENGINE_PREFIX + "pollingInterval";
+
+ private static final Integer DEFAULT_NUM_THREADS = 3;
+ private static final Integer DEFAULT_JOB_STATUS_POLLING_INTERVAL = 10;
+
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+ private ScheduledExecutorService scheduledExecutorPool;
+ private Config config;
+ private Integer numThreads;
+ private Integer pollingInterval;
+ private DagProcessingEngine.Thread [] threads;
+ private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+
+
+ public DagProcessingEngine(Config config, DagTaskStream dagTaskStream,
DagProcFactory dagProcFactory, DagManagementStateStore dagManagementStateStore,
MultiActiveLeaseArbiter multiActiveLeaseArbiter) {
+ this.config = config;
+ this.dagTaskStream = dagTaskStream;
+ this.dagProcFactory = dagProcFactory;
+ this.dagManagementStateStore = dagManagementStateStore;
+ this.multiActiveLeaseArbiter = multiActiveLeaseArbiter;
+ this.numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.pollingInterval = ConfigUtils.getInt(config,
JOB_STATUS_POLLING_INTERVAL_KEY, DEFAULT_JOB_STATUS_POLLING_INTERVAL);
+ this.threads = new DagProcessingEngine.Thread[this.numThreads];
+ for(int i=0; i < this.numThreads; i++) {
+ Thread thread = new Thread(dagTaskStream, dagProcFactory,
dagManagementStateStore, multiActiveLeaseArbiter);
+ this.threads[i] = thread;
+ }
+ }
+
+
+ @AllArgsConstructor
+ private static class Thread implements Runnable {
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+ private MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+
+ @Override
+ public void run() {
+ try {
+ while (dagTaskStream.hasNext()) {
+ Optional<DagTask> dagTask = getNextTask();
+ if (dagTask.isPresent()) {
+ DagProc dagProc = (DagProc) dagTask.get().host(dagProcFactory);
Review Comment:
I too am puzzled by how we'd ever find an `Optional<DagTask>` on our hands
##########
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}
Review Comment:
I believe you mean "as soon as it's available" (since the stream's `next()`
method ought to block until one's available)
##########
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;
Review Comment:
given this is a stream (indicating infinite) and more work could always
arise from future activity, always returning true does appropriately model that
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/processor/DagProc.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration.processor;
+
+import java.io.IOException;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import
org.apache.gobblin.service.modules.orchestration.exception.MaybeRetryableException;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+
+
+/**
+ * Responsible to performing the actual work for a given {@link DagTask}.
+ * It processes the {@link DagTask} by first initializing its state,
performing actions
+ * based on the type of {@link DagTask} and finally submitting an event to the
executor.
+ * @param <S> current state of the dag node
+ * @param <R> result after processing the dag node
+ */
Review Comment:
it's the `DagProcessingEngine`, who asks the `DagProcFactory` to create one
of these for a given `DagTask`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/processor/KillDagProc.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.processor;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import com.google.api.client.util.Lists;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.event.EventSubmitter;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.TimingEventUtils;
+import
org.apache.gobblin.service.modules.orchestration.exception.MaybeRetryableException;
+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 static org.apache.gobblin.service.ExecutionStatus.CANCELLED;
+
+
+/**
+ * An implementation of {@link DagProc} for killing {@link DagTask}.
+ */
+@Slf4j
+@Alpha
+public final class KillDagProc extends DagProc {
+
+ private KillDagTask killDagTask;
+ private DagManagementStateStore dagManagementStateStore;
+ private MetricContext metricContext;
+ private Optional<EventSubmitter> eventSubmitter;
Review Comment:
perhaps should these two also be args to `process`?
##########
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()) {
Review Comment:
rather than returning `Optional`, this condition ought to guard a `while`
loop that retries lease acquisition on another dag action until one is obtained
and we can construct a `DagTask` to return
Issue Time Tracking
-------------------
Worklog Id: (was: 881069)
Time Spent: 4h 50m (was: 4h 40m)
> Refactor code to move current in-memory references to new design for REST
> calls: Launch, Resume and Kill
> --------------------------------------------------------------------------------------------------------
>
> Key: GOBBLIN-1910
> URL: https://issues.apache.org/jira/browse/GOBBLIN-1910
> Project: Apache Gobblin
> Issue Type: New Feature
> Reporter: Meeth Gala
> Priority: Major
> Time Spent: 4h 50m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)