[
https://issues.apache.org/jira/browse/GOBBLIN-1910?focusedWorklogId=904497&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-904497
]
ASF GitHub Bot logged work on GOBBLIN-1910:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 10/Feb/24 01:50
Start Date: 10/Feb/24 01:50
Worklog Time Spent: 10m
Work Description: phet commented on code in PR #3858:
URL: https://github.com/apache/gobblin/pull/3858#discussion_r1484923266
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+
+import javax.naming.OperationNotSupportedException;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.orchestration.proc.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Responsible for polling {@link DagTask}s from {@link DagTaskStream} and
processing the {@link org.apache.gobblin.service.modules.flowgraph.Dag}
+ * based on the type of {@link DagTask} which is determined by the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * Each {@link DagTask} acquires a lease for the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * The {@link DagProcFactory} then provides the appropriate {@link DagProc}
associated with the {@link DagTask}.
+ * The actual work or processing is done by the {@link
DagProc#process(DagManagementStateStore)}
+ */
+
+@Alpha
+@Slf4j
+@Singleton
+public class DagProcessingEngine {
+ public static final String GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX =
ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "dagProcessingEngine.";
+ public static final String DAG_PROCESSING_ENGINE_ENABLED =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "enabled";
+ public static final String NUM_THREADS_KEY =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "numThreads";
+ private static final Integer DEFAULT_NUM_THREADS = 3;
+
+ @Getter private final DagTaskStream dagTaskStream;
+ @Getter DagManagementStateStore dagManagementStateStore;
+
+ @Inject
+ public DagProcessingEngine(Config config, DagTaskStream dagTaskStream,
DagProcFactory dagProcFactory,
+ DagManagementStateStore dagManagementStateStore,
Optional<MultiActiveLeaseArbiter> multiActiveLeaseArbiter) {
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ ScheduledExecutorService scheduledExecutorPool =
+ Executors.newScheduledThreadPool(numThreads, new
NamedThreadFactory("DagProcessingEngineThread"));
+ this.dagTaskStream = dagTaskStream;
+ this.dagManagementStateStore = dagManagementStateStore;
+
+ for (int i=0; i < numThreads; i++) {
+ DagProcEngineThread dagProcEngineThread = new
DagProcEngineThread(dagTaskStream, dagProcFactory, dagManagementStateStore);
+ scheduledExecutorPool.submit(dagProcEngineThread);
+ }
+ }
+
+ public void addDagNodeToRetry(Dag.DagNode<JobExecutionPlan> dagNode) throws
OperationNotSupportedException {
Review Comment:
I'm unclear on the semantics here. until I learn more, I'm skeptical it
belongs here... but details might help change my mind. (javadoc would really
help)
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+
+import javax.naming.OperationNotSupportedException;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.orchestration.proc.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Responsible for polling {@link DagTask}s from {@link DagTaskStream} and
processing the {@link org.apache.gobblin.service.modules.flowgraph.Dag}
+ * based on the type of {@link DagTask} which is determined by the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
Review Comment:
I'd suggest omitting the part starting with `which is...`, and continuing to
the end of this javadoc comment.
it all happens behind the scenes in `DagTaskStream`, so would belong in
javadoc over there.
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java:
##########
@@ -0,0 +1,430 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.Data;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+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.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.TopologySpec;
+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.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.JobStatus;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+import org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor;
+import org.apache.gobblin.service.monitoring.KillFlowEvent;
+import org.apache.gobblin.service.monitoring.ResumeFlowEvent;
+import org.apache.gobblin.service.monitoring.event.JobStatusEvent;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+
+import static org.apache.gobblin.service.ExecutionStatus.*;
+
+
+/**
+ * NewDagManager has these functionalities :
+ * a) manages {@link Dag}s through {@link DagManagementStateStore}.
+ * b) subscribes to {@link JobStatusEvent} sent by {@link
KafkaJobStatusMonitor}
+ * c) spawns a {@link DagManager.FailedDagRetentionThread} that cleans failed
dags.
+ * d) load {@link Dag}s on service-start / set-active.
+ */
+@Slf4j
+@Singleton
+@Data
+public class NewDagManager {
+ public static final String DAG_MANAGER_PREFIX =
"gobblin.service.dagManager.";
+ public static final int DEFAULT_NUM_THREADS = 3;
+ public static final String NUM_THREADS_KEY = DAG_MANAGER_PREFIX +
"numThreads";
+ private static final int TERMINATION_TIMEOUT = 30;
+ private static final String DAG_STATESTORE_CLASS_KEY = DAG_MANAGER_PREFIX +
"dagStateStoreClass";
+ private static final String FAILED_DAG_STATESTORE_PREFIX =
"failedDagStateStore";
+ private static final String FAILED_DAG_RETENTION_TIME_UNIT =
FAILED_DAG_STATESTORE_PREFIX + ".retention.timeUnit";
+ private static final String DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT = "DAYS";
+ private static final String FAILED_DAG_RETENTION_TIME =
FAILED_DAG_STATESTORE_PREFIX + ".retention.time";
+ private static final long DEFAULT_FAILED_DAG_RETENTION_TIME = 7L;
+ // Re-emit the final flow status if not detected within 5 minutes
+ public static final String FAILED_DAG_POLLING_INTERVAL =
FAILED_DAG_STATESTORE_PREFIX + ".retention.pollingIntervalMinutes";
+ public static final int DEFAULT_FAILED_DAG_POLLING_INTERVAL = 60;
+ private static final int INITIAL_HOUSEKEEPING_THREAD_DELAY = 2;
+ private final Config config;
+ private final int retentionPollingInterval;
+
+ private final ScheduledExecutorService scheduledExecutorPool;
+ @Inject UserQuotaManager userQuotaManager;
+ @Inject private FlowCatalog flowCatalog;
+ @Getter private DagStateStore failedDagStateStore;
+ private final boolean dagProcessingEngineEnabled;
+ @Getter private Set<String> failedDagIds;
+ private Map<URI, TopologySpec> topologySpecMap = new HashMap<>();
+ @Getter private DagStateStore dagStateStore;
+ @Getter private final JobStatusRetriever jobStatusRetriever;
+ @Getter private final Timer jobStatusPolledTimer;
+ @Getter private final EventSubmitter eventSubmitter;
+ private final long failedDagRetentionTime;
+ @Getter private static final DagManagerMetrics dagManagerMetrics = new
DagManagerMetrics();
+ private ScheduledExecutorService houseKeepingThreadPool;
+ private volatile boolean isActive = false;
+
+ @Inject(optional=true)
+ protected Optional<DagActionStore> dagActionStore;
+ @Inject(optional=true)
+ @Getter DagManagementStateStore dagManagementStateStore;
+ private static final int MAX_HOUSEKEEPING_THREAD_DELAY = 180;
+ protected final EventBus eventBus;
+ DagTaskStream dagTaskStream;
+
+ @Inject
+ public NewDagManager(Config config, JobStatusRetriever jobStatusRetriever,
Optional<DagActionStore> dagActionStore,
+ DagTaskStream dagTaskStream, DagManagementStateStore
dagManagementStateStore) {
+ this.config = config;
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.dagActionStore = dagActionStore;
+ this.dagTaskStream = dagTaskStream;
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.retentionPollingInterval = ConfigUtils.getInt(config,
FAILED_DAG_POLLING_INTERVAL, DEFAULT_FAILED_DAG_POLLING_INTERVAL);
+ this.eventBus = KafkaJobStatusMonitor.getEventBus();
+ this.eventBus.register(this);
+ this.dagProcessingEngineEnabled = ConfigUtils.getBoolean(config,
DagProcessingEngine.DAG_PROCESSING_ENGINE_ENABLED, false);
+ this.dagManagementStateStore = dagManagementStateStore;
+ MetricContext metricContext =
Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()),
getClass());
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.eventSubmitter = new EventSubmitter.Builder(metricContext,
"org.apache.gobblin.service").build();
+ this.jobStatusRetriever = jobStatusRetriever;
+ TimeUnit timeUnit = TimeUnit.valueOf(ConfigUtils.getString(config,
FAILED_DAG_RETENTION_TIME_UNIT, DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT));
+ this.failedDagRetentionTime =
timeUnit.toMillis(ConfigUtils.getLong(config, FAILED_DAG_RETENTION_TIME,
DEFAULT_FAILED_DAG_RETENTION_TIME));
+ }
+
+ public void setActive(boolean active) throws IOException {
+ if (this.isActive == active) {
+ log.info("DagManager already {}, skipping further actions.", (!active) ?
"inactive" : "active");
+ }
+ this.isActive = active;
+ try {
+ if (this.isActive) {
+ log.info("Activating NewDagManager.");
+ //Initializing state store for persisting Dags.
+ this.dagStateStore = createDagStateStore(config, topologySpecMap);
+ // todo - implement as a kill dag action
+ this.failedDagStateStore =
createDagStateStore(ConfigUtils.getConfigOrEmpty(config,
FAILED_DAG_STATESTORE_PREFIX).withFallback(config),
+ topologySpecMap);
+ this.failedDagIds =
Collections.synchronizedSet(failedDagStateStore.getDagIds());
+ dagManagerMetrics.activate();
+ userQuotaManager.init(dagStateStore.getDags());
+ DagManager.FailedDagRetentionThread failedDagRetentionThread =
+ new DagManager.FailedDagRetentionThread(failedDagStateStore,
failedDagIds, failedDagRetentionTime);
+
this.scheduledExecutorPool.scheduleAtFixedRate(failedDagRetentionThread, 0,
retentionPollingInterval, TimeUnit.MINUTES);
+ loadDagFromDagStateStore();
+ this.houseKeepingThreadPool =
Executors.newSingleThreadScheduledExecutor(new
NamedThreadFactory("LoadDagsThread"));
+ for (int delay = INITIAL_HOUSEKEEPING_THREAD_DELAY; delay <
MAX_HOUSEKEEPING_THREAD_DELAY; delay *= 2) {
+ this.houseKeepingThreadPool.schedule(() -> {
+ try {
+ loadDagFromDagStateStore();
+ } catch (Exception e) {
+ log.error("failed to sync dag state store due to ", e);
+ }
+ }, delay, TimeUnit.MINUTES);
+ }
+ } else { //Mark the DagManager inactive.
+ log.info("Inactivating the DagManager. Shutting down all DagManager
threads");
+ this.scheduledExecutorPool.shutdown();
+ dagManagerMetrics.cleanup();
+ this.houseKeepingThreadPool.shutdown();
+ try {
+ this.scheduledExecutorPool.awaitTermination(TERMINATION_TIMEOUT,
TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ log.error("Exception encountered when shutting down DagManager
threads.", e);
+ }
+ }
+ } catch (IOException e) {
+ log.error("Exception encountered when activating the new DagManager",
e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static DagStateStore createDagStateStore(Config config, Map<URI,
TopologySpec> topologySpecMap) {
+ try {
+ Class<?> dagStateStoreClass =
Class.forName(ConfigUtils.getString(config, DAG_STATESTORE_CLASS_KEY,
FSDagStateStore.class.getName()));
+ return (DagStateStore)
GobblinConstructorUtils.invokeLongestConstructor(dagStateStoreClass, config,
topologySpecMap);
+ } catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void loadDagFromDagStateStore() throws IOException {
+ List<Dag<JobExecutionPlan>> dags = this.dagStateStore.getDags();
+ log.info("Loading " + dags.size() + " dags from dag state store");
+ for (Dag<JobExecutionPlan> dag : dags) {
+ addDag(dag, false, false);
+ }
+ }
+
+ public synchronized void addDagAndRemoveAdhocFlowSpec(FlowSpec flowSpec,
Dag<JobExecutionPlan> dag, boolean persist, boolean setStatus)
+ throws IOException {
+ addDag(dag, persist, setStatus);
+ // Only the active newDagManager should delete the flowSpec
+ if (isActive) {
+ deleteSpecFromCatalogIfAdhoc(flowSpec);
+ }
+ }
+
+ private synchronized void addDag(Dag<JobExecutionPlan> dag, boolean persist,
boolean setStatus) throws IOException {
+ // TODO: Used to track missing dag issue, remove later as needed
+ log.info("Add dag (persist: {}, setStatus: {}): {}", persist, setStatus,
dag);
+ if (!isActive) {
+ log.warn("Skipping add dag because this instance of DagManager is not
active for dag: {}", dag);
+ return;
+ }
+
+ DagManager.DagId dagId = DagManagerUtils.generateDagId(dag);
+ String dagIdString = dagId.toString();
+ if (persist) {
+ // Persist the dag
+ this.dagStateStore.writeCheckpoint(dag);
+ // After persisting the dag, its status will be tracked by active
dagManagers so the action should be deleted
+ // to avoid duplicate executions upon leadership change
+ if (this.dagActionStore.isPresent()) {
+
this.dagActionStore.get().deleteDagAction(dagId.toDagAction(DagActionStore.FlowActionType.LAUNCH));
+ }
+ }
+
+ DagActionStore.DagAction dagAction = new DagActionStore.DagAction(
+ dagId.getFlowGroup(), dagId.getFlowName(), dagId.getFlowExecutionId(),
DagActionStore.FlowActionType.LAUNCH);
+
+ if (dagManagementStateStore.containsDag(dagIdString)) {
+ log.warn("Already tracking a dag with dagId {}, skipping.", dagIdString);
+ return;
+ }
+
+ dagManagementStateStore.addDag(dagIdString, dag);
+ this.dagTaskStream.addDagAction(dagAction);
+
+ if (setStatus) {
+ DagManagerUtils.submitPendingExecStatus(dag, this.eventSubmitter);
+ }
+ }
+
+ private void deleteSpecFromCatalogIfAdhoc(FlowSpec flowSpec) {
+ if (!flowSpec.isScheduled()) {
+ this.flowCatalog.remove(flowSpec.getUri(), new Properties(), false);
+ }
+ }
+
+ public void removeDagActionFromStore(DagActionStore.DagAction dagAction)
throws IOException {
+ if (this.dagActionStore.isPresent()) {
+ this.dagActionStore.get().deleteDagAction(dagAction);
+ }
+ }
+
+ //@Subscribe todo uncomment it when new dag manager is ready for kill
functionality
+ public void handleKillFlowEvent(KillFlowEvent killFlowEvent) {
Review Comment:
why not have the `DagTaskStream` subscribe to this and resume as well? what
pros/cons do you find?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java:
##########
@@ -0,0 +1,430 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.Data;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+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.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.TopologySpec;
+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.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.JobStatus;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+import org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor;
+import org.apache.gobblin.service.monitoring.KillFlowEvent;
+import org.apache.gobblin.service.monitoring.ResumeFlowEvent;
+import org.apache.gobblin.service.monitoring.event.JobStatusEvent;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+
+import static org.apache.gobblin.service.ExecutionStatus.*;
+
+
+/**
+ * NewDagManager has these functionalities :
+ * a) manages {@link Dag}s through {@link DagManagementStateStore}.
+ * b) subscribes to {@link JobStatusEvent} sent by {@link
KafkaJobStatusMonitor}
+ * c) spawns a {@link DagManager.FailedDagRetentionThread} that cleans failed
dags.
+ * d) load {@link Dag}s on service-start / set-active.
+ */
+@Slf4j
+@Singleton
+@Data
+public class NewDagManager {
+ public static final String DAG_MANAGER_PREFIX =
"gobblin.service.dagManager.";
+ public static final int DEFAULT_NUM_THREADS = 3;
+ public static final String NUM_THREADS_KEY = DAG_MANAGER_PREFIX +
"numThreads";
+ private static final int TERMINATION_TIMEOUT = 30;
+ private static final String DAG_STATESTORE_CLASS_KEY = DAG_MANAGER_PREFIX +
"dagStateStoreClass";
+ private static final String FAILED_DAG_STATESTORE_PREFIX =
"failedDagStateStore";
+ private static final String FAILED_DAG_RETENTION_TIME_UNIT =
FAILED_DAG_STATESTORE_PREFIX + ".retention.timeUnit";
+ private static final String DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT = "DAYS";
+ private static final String FAILED_DAG_RETENTION_TIME =
FAILED_DAG_STATESTORE_PREFIX + ".retention.time";
+ private static final long DEFAULT_FAILED_DAG_RETENTION_TIME = 7L;
+ // Re-emit the final flow status if not detected within 5 minutes
+ public static final String FAILED_DAG_POLLING_INTERVAL =
FAILED_DAG_STATESTORE_PREFIX + ".retention.pollingIntervalMinutes";
+ public static final int DEFAULT_FAILED_DAG_POLLING_INTERVAL = 60;
+ private static final int INITIAL_HOUSEKEEPING_THREAD_DELAY = 2;
+ private final Config config;
+ private final int retentionPollingInterval;
+
+ private final ScheduledExecutorService scheduledExecutorPool;
+ @Inject UserQuotaManager userQuotaManager;
+ @Inject private FlowCatalog flowCatalog;
+ @Getter private DagStateStore failedDagStateStore;
+ private final boolean dagProcessingEngineEnabled;
+ @Getter private Set<String> failedDagIds;
+ private Map<URI, TopologySpec> topologySpecMap = new HashMap<>();
+ @Getter private DagStateStore dagStateStore;
+ @Getter private final JobStatusRetriever jobStatusRetriever;
+ @Getter private final Timer jobStatusPolledTimer;
+ @Getter private final EventSubmitter eventSubmitter;
+ private final long failedDagRetentionTime;
+ @Getter private static final DagManagerMetrics dagManagerMetrics = new
DagManagerMetrics();
+ private ScheduledExecutorService houseKeepingThreadPool;
+ private volatile boolean isActive = false;
+
+ @Inject(optional=true)
+ protected Optional<DagActionStore> dagActionStore;
+ @Inject(optional=true)
+ @Getter DagManagementStateStore dagManagementStateStore;
+ private static final int MAX_HOUSEKEEPING_THREAD_DELAY = 180;
+ protected final EventBus eventBus;
+ DagTaskStream dagTaskStream;
+
+ @Inject
+ public NewDagManager(Config config, JobStatusRetriever jobStatusRetriever,
Optional<DagActionStore> dagActionStore,
+ DagTaskStream dagTaskStream, DagManagementStateStore
dagManagementStateStore) {
+ this.config = config;
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.dagActionStore = dagActionStore;
+ this.dagTaskStream = dagTaskStream;
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.retentionPollingInterval = ConfigUtils.getInt(config,
FAILED_DAG_POLLING_INTERVAL, DEFAULT_FAILED_DAG_POLLING_INTERVAL);
+ this.eventBus = KafkaJobStatusMonitor.getEventBus();
+ this.eventBus.register(this);
+ this.dagProcessingEngineEnabled = ConfigUtils.getBoolean(config,
DagProcessingEngine.DAG_PROCESSING_ENGINE_ENABLED, false);
+ this.dagManagementStateStore = dagManagementStateStore;
+ MetricContext metricContext =
Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()),
getClass());
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.eventSubmitter = new EventSubmitter.Builder(metricContext,
"org.apache.gobblin.service").build();
+ this.jobStatusRetriever = jobStatusRetriever;
+ TimeUnit timeUnit = TimeUnit.valueOf(ConfigUtils.getString(config,
FAILED_DAG_RETENTION_TIME_UNIT, DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT));
+ this.failedDagRetentionTime =
timeUnit.toMillis(ConfigUtils.getLong(config, FAILED_DAG_RETENTION_TIME,
DEFAULT_FAILED_DAG_RETENTION_TIME));
+ }
+
+ public void setActive(boolean active) throws IOException {
+ if (this.isActive == active) {
+ log.info("DagManager already {}, skipping further actions.", (!active) ?
"inactive" : "active");
+ }
+ this.isActive = active;
+ try {
+ if (this.isActive) {
+ log.info("Activating NewDagManager.");
+ //Initializing state store for persisting Dags.
+ this.dagStateStore = createDagStateStore(config, topologySpecMap);
+ // todo - implement as a kill dag action
+ this.failedDagStateStore =
createDagStateStore(ConfigUtils.getConfigOrEmpty(config,
FAILED_DAG_STATESTORE_PREFIX).withFallback(config),
+ topologySpecMap);
+ this.failedDagIds =
Collections.synchronizedSet(failedDagStateStore.getDagIds());
+ dagManagerMetrics.activate();
+ userQuotaManager.init(dagStateStore.getDags());
+ DagManager.FailedDagRetentionThread failedDagRetentionThread =
+ new DagManager.FailedDagRetentionThread(failedDagStateStore,
failedDagIds, failedDagRetentionTime);
+
this.scheduledExecutorPool.scheduleAtFixedRate(failedDagRetentionThread, 0,
retentionPollingInterval, TimeUnit.MINUTES);
+ loadDagFromDagStateStore();
+ this.houseKeepingThreadPool =
Executors.newSingleThreadScheduledExecutor(new
NamedThreadFactory("LoadDagsThread"));
+ for (int delay = INITIAL_HOUSEKEEPING_THREAD_DELAY; delay <
MAX_HOUSEKEEPING_THREAD_DELAY; delay *= 2) {
+ this.houseKeepingThreadPool.schedule(() -> {
+ try {
+ loadDagFromDagStateStore();
+ } catch (Exception e) {
+ log.error("failed to sync dag state store due to ", e);
+ }
+ }, delay, TimeUnit.MINUTES);
+ }
+ } else { //Mark the DagManager inactive.
+ log.info("Inactivating the DagManager. Shutting down all DagManager
threads");
+ this.scheduledExecutorPool.shutdown();
+ dagManagerMetrics.cleanup();
+ this.houseKeepingThreadPool.shutdown();
+ try {
+ this.scheduledExecutorPool.awaitTermination(TERMINATION_TIMEOUT,
TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ log.error("Exception encountered when shutting down DagManager
threads.", e);
+ }
+ }
+ } catch (IOException e) {
+ log.error("Exception encountered when activating the new DagManager",
e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static DagStateStore createDagStateStore(Config config, Map<URI,
TopologySpec> topologySpecMap) {
+ try {
+ Class<?> dagStateStoreClass =
Class.forName(ConfigUtils.getString(config, DAG_STATESTORE_CLASS_KEY,
FSDagStateStore.class.getName()));
+ return (DagStateStore)
GobblinConstructorUtils.invokeLongestConstructor(dagStateStoreClass, config,
topologySpecMap);
+ } catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void loadDagFromDagStateStore() throws IOException {
+ List<Dag<JobExecutionPlan>> dags = this.dagStateStore.getDags();
+ log.info("Loading " + dags.size() + " dags from dag state store");
+ for (Dag<JobExecutionPlan> dag : dags) {
+ addDag(dag, false, false);
+ }
+ }
+
+ public synchronized void addDagAndRemoveAdhocFlowSpec(FlowSpec flowSpec,
Dag<JobExecutionPlan> dag, boolean persist, boolean setStatus)
+ throws IOException {
+ addDag(dag, persist, setStatus);
+ // Only the active newDagManager should delete the flowSpec
+ if (isActive) {
+ deleteSpecFromCatalogIfAdhoc(flowSpec);
+ }
+ }
+
+ private synchronized void addDag(Dag<JobExecutionPlan> dag, boolean persist,
boolean setStatus) throws IOException {
+ // TODO: Used to track missing dag issue, remove later as needed
+ log.info("Add dag (persist: {}, setStatus: {}): {}", persist, setStatus,
dag);
+ if (!isActive) {
+ log.warn("Skipping add dag because this instance of DagManager is not
active for dag: {}", dag);
+ return;
+ }
+
+ DagManager.DagId dagId = DagManagerUtils.generateDagId(dag);
+ String dagIdString = dagId.toString();
+ if (persist) {
+ // Persist the dag
+ this.dagStateStore.writeCheckpoint(dag);
+ // After persisting the dag, its status will be tracked by active
dagManagers so the action should be deleted
+ // to avoid duplicate executions upon leadership change
+ if (this.dagActionStore.isPresent()) {
+
this.dagActionStore.get().deleteDagAction(dagId.toDagAction(DagActionStore.FlowActionType.LAUNCH));
+ }
+ }
+
+ DagActionStore.DagAction dagAction = new DagActionStore.DagAction(
+ dagId.getFlowGroup(), dagId.getFlowName(), dagId.getFlowExecutionId(),
DagActionStore.FlowActionType.LAUNCH);
+
+ if (dagManagementStateStore.containsDag(dagIdString)) {
+ log.warn("Already tracking a dag with dagId {}, skipping.", dagIdString);
+ return;
+ }
+
+ dagManagementStateStore.addDag(dagIdString, dag);
+ this.dagTaskStream.addDagAction(dagAction);
+
+ if (setStatus) {
+ DagManagerUtils.submitPendingExecStatus(dag, this.eventSubmitter);
+ }
+ }
+
+ private void deleteSpecFromCatalogIfAdhoc(FlowSpec flowSpec) {
+ if (!flowSpec.isScheduled()) {
+ this.flowCatalog.remove(flowSpec.getUri(), new Properties(), false);
+ }
+ }
+
+ public void removeDagActionFromStore(DagActionStore.DagAction dagAction)
throws IOException {
+ if (this.dagActionStore.isPresent()) {
+ this.dagActionStore.get().deleteDagAction(dagAction);
+ }
+ }
+
+ //@Subscribe todo uncomment it when new dag manager is ready for kill
functionality
+ public void handleKillFlowEvent(KillFlowEvent killFlowEvent) {
+ String flowGroup = killFlowEvent.getFlowGroup();
+ String flowName = killFlowEvent.getFlowName();
+ if (killFlowEvent.getFlowExecutionId() != null) {
+ long flowExecutionId = killFlowEvent.getFlowExecutionId();
+ log.info("Received kill request for flow ({}, {}, {})", flowGroup,
flowName, flowExecutionId);
+ // todo - should this be added to the dac action store instead?
+ this.dagTaskStream.addDagAction(new DagActionStore.DagAction(flowGroup,
flowName, String.valueOf(flowExecutionId), DagActionStore.FlowActionType.KILL));
+
+ } else {
+ List<Long> flowExecutionIds =
getJobStatusRetriever().getLatestExecutionIdsForFlow(flowName, flowGroup, 10);
+ log.info("Found {} flows to cancel.", flowExecutionIds.size());
+ for (long flowExecutionId : flowExecutionIds) {
+ handleKillFlowEvent(new KillFlowEvent(flowGroup, flowName,
flowExecutionId));
+ }
+ }
+ }
+
+ @Subscribe
+ public void handleResumeFlowEvent(ResumeFlowEvent resumeFlowEvent) {
Review Comment:
why not formulate this instead as its own `DagTask` and `DagProc`?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+
+import javax.naming.OperationNotSupportedException;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.orchestration.proc.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Responsible for polling {@link DagTask}s from {@link DagTaskStream} and
processing the {@link org.apache.gobblin.service.modules.flowgraph.Dag}
+ * based on the type of {@link DagTask} which is determined by the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * Each {@link DagTask} acquires a lease for the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * The {@link DagProcFactory} then provides the appropriate {@link DagProc}
associated with the {@link DagTask}.
+ * The actual work or processing is done by the {@link
DagProc#process(DagManagementStateStore)}
+ */
+
+@Alpha
+@Slf4j
+@Singleton
+public class DagProcessingEngine {
+ public static final String GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX =
ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "dagProcessingEngine.";
+ public static final String DAG_PROCESSING_ENGINE_ENABLED =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "enabled";
+ public static final String NUM_THREADS_KEY =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "numThreads";
+ private static final Integer DEFAULT_NUM_THREADS = 3;
+
+ @Getter private final DagTaskStream dagTaskStream;
+ @Getter DagManagementStateStore dagManagementStateStore;
+
+ @Inject
+ public DagProcessingEngine(Config config, DagTaskStream dagTaskStream,
DagProcFactory dagProcFactory,
+ DagManagementStateStore dagManagementStateStore,
Optional<MultiActiveLeaseArbiter> multiActiveLeaseArbiter) {
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ ScheduledExecutorService scheduledExecutorPool =
+ Executors.newScheduledThreadPool(numThreads, new
NamedThreadFactory("DagProcessingEngineThread"));
+ this.dagTaskStream = dagTaskStream;
+ this.dagManagementStateStore = dagManagementStateStore;
+
+ for (int i=0; i < numThreads; i++) {
+ DagProcEngineThread dagProcEngineThread = new
DagProcEngineThread(dagTaskStream, dagProcFactory, dagManagementStateStore);
+ scheduledExecutorPool.submit(dagProcEngineThread);
+ }
+ }
+
+ public void addDagNodeToRetry(Dag.DagNode<JobExecutionPlan> dagNode) throws
OperationNotSupportedException {
+ // todo - how to add dag action for a dag node? should we create a dag
node action? right now dag action is synonymous to flow action
+ // this.dagTaskStream.addDagTask(new RetryDagTask(dagNode));
+ throw new OperationNotSupportedException();
+ }
+
+ @AllArgsConstructor
+ private static class DagProcEngineThread implements Runnable {
+
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+
+ @Override
+ public void run() {
+ while (true) {
+ DagTask dagTask = dagTaskStream.next(); // blocking call
+ if (dagTask == null) {
+ continue;
+ }
+ DagProc dagProc = dagTask.host(dagProcFactory);
+ try {
+ // todo - add retries
+ dagProc.process(dagManagementStateStore);
+ } catch (Throwable t) {
+ log.error("DagProcEngineThread encountered error ", t);
+ }
+ // todo mark lease success and releases it
+ //dagTaskStream.complete(dagTask);
+ }
Review Comment:
I agree w/ this TODO. it probably belongs in the `try` block.
at the same time, let's firm up the contract w/ the `DagTaskStream`. above
all, we don't want to "leak" tasks due to failures. e.g. if we don't complete
the task is a reminder automatically set within the `DagTaskStream`? or should
we be interpreting the exception here and setting the reminder?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java:
##########
@@ -0,0 +1,430 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.Data;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+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.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.TopologySpec;
+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.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.JobStatus;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+import org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor;
+import org.apache.gobblin.service.monitoring.KillFlowEvent;
+import org.apache.gobblin.service.monitoring.ResumeFlowEvent;
+import org.apache.gobblin.service.monitoring.event.JobStatusEvent;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+
+import static org.apache.gobblin.service.ExecutionStatus.*;
+
+
+/**
+ * NewDagManager has these functionalities :
+ * a) manages {@link Dag}s through {@link DagManagementStateStore}.
+ * b) subscribes to {@link JobStatusEvent} sent by {@link
KafkaJobStatusMonitor}
+ * c) spawns a {@link DagManager.FailedDagRetentionThread} that cleans failed
dags.
+ * d) load {@link Dag}s on service-start / set-active.
+ */
+@Slf4j
+@Singleton
+@Data
+public class NewDagManager {
+ public static final String DAG_MANAGER_PREFIX =
"gobblin.service.dagManager.";
+ public static final int DEFAULT_NUM_THREADS = 3;
+ public static final String NUM_THREADS_KEY = DAG_MANAGER_PREFIX +
"numThreads";
+ private static final int TERMINATION_TIMEOUT = 30;
+ private static final String DAG_STATESTORE_CLASS_KEY = DAG_MANAGER_PREFIX +
"dagStateStoreClass";
+ private static final String FAILED_DAG_STATESTORE_PREFIX =
"failedDagStateStore";
+ private static final String FAILED_DAG_RETENTION_TIME_UNIT =
FAILED_DAG_STATESTORE_PREFIX + ".retention.timeUnit";
+ private static final String DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT = "DAYS";
+ private static final String FAILED_DAG_RETENTION_TIME =
FAILED_DAG_STATESTORE_PREFIX + ".retention.time";
+ private static final long DEFAULT_FAILED_DAG_RETENTION_TIME = 7L;
+ // Re-emit the final flow status if not detected within 5 minutes
+ public static final String FAILED_DAG_POLLING_INTERVAL =
FAILED_DAG_STATESTORE_PREFIX + ".retention.pollingIntervalMinutes";
+ public static final int DEFAULT_FAILED_DAG_POLLING_INTERVAL = 60;
+ private static final int INITIAL_HOUSEKEEPING_THREAD_DELAY = 2;
+ private final Config config;
+ private final int retentionPollingInterval;
+
+ private final ScheduledExecutorService scheduledExecutorPool;
+ @Inject UserQuotaManager userQuotaManager;
+ @Inject private FlowCatalog flowCatalog;
+ @Getter private DagStateStore failedDagStateStore;
+ private final boolean dagProcessingEngineEnabled;
+ @Getter private Set<String> failedDagIds;
+ private Map<URI, TopologySpec> topologySpecMap = new HashMap<>();
+ @Getter private DagStateStore dagStateStore;
+ @Getter private final JobStatusRetriever jobStatusRetriever;
+ @Getter private final Timer jobStatusPolledTimer;
+ @Getter private final EventSubmitter eventSubmitter;
+ private final long failedDagRetentionTime;
+ @Getter private static final DagManagerMetrics dagManagerMetrics = new
DagManagerMetrics();
+ private ScheduledExecutorService houseKeepingThreadPool;
+ private volatile boolean isActive = false;
+
+ @Inject(optional=true)
+ protected Optional<DagActionStore> dagActionStore;
+ @Inject(optional=true)
+ @Getter DagManagementStateStore dagManagementStateStore;
+ private static final int MAX_HOUSEKEEPING_THREAD_DELAY = 180;
+ protected final EventBus eventBus;
+ DagTaskStream dagTaskStream;
+
+ @Inject
+ public NewDagManager(Config config, JobStatusRetriever jobStatusRetriever,
Optional<DagActionStore> dagActionStore,
+ DagTaskStream dagTaskStream, DagManagementStateStore
dagManagementStateStore) {
+ this.config = config;
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.dagActionStore = dagActionStore;
+ this.dagTaskStream = dagTaskStream;
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.retentionPollingInterval = ConfigUtils.getInt(config,
FAILED_DAG_POLLING_INTERVAL, DEFAULT_FAILED_DAG_POLLING_INTERVAL);
+ this.eventBus = KafkaJobStatusMonitor.getEventBus();
+ this.eventBus.register(this);
+ this.dagProcessingEngineEnabled = ConfigUtils.getBoolean(config,
DagProcessingEngine.DAG_PROCESSING_ENGINE_ENABLED, false);
+ this.dagManagementStateStore = dagManagementStateStore;
+ MetricContext metricContext =
Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()),
getClass());
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.eventSubmitter = new EventSubmitter.Builder(metricContext,
"org.apache.gobblin.service").build();
+ this.jobStatusRetriever = jobStatusRetriever;
+ TimeUnit timeUnit = TimeUnit.valueOf(ConfigUtils.getString(config,
FAILED_DAG_RETENTION_TIME_UNIT, DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT));
+ this.failedDagRetentionTime =
timeUnit.toMillis(ConfigUtils.getLong(config, FAILED_DAG_RETENTION_TIME,
DEFAULT_FAILED_DAG_RETENTION_TIME));
+ }
+
+ public void setActive(boolean active) throws IOException {
+ if (this.isActive == active) {
+ log.info("DagManager already {}, skipping further actions.", (!active) ?
"inactive" : "active");
+ }
+ this.isActive = active;
+ try {
+ if (this.isActive) {
+ log.info("Activating NewDagManager.");
+ //Initializing state store for persisting Dags.
+ this.dagStateStore = createDagStateStore(config, topologySpecMap);
+ // todo - implement as a kill dag action
+ this.failedDagStateStore =
createDagStateStore(ConfigUtils.getConfigOrEmpty(config,
FAILED_DAG_STATESTORE_PREFIX).withFallback(config),
+ topologySpecMap);
+ this.failedDagIds =
Collections.synchronizedSet(failedDagStateStore.getDagIds());
+ dagManagerMetrics.activate();
+ userQuotaManager.init(dagStateStore.getDags());
+ DagManager.FailedDagRetentionThread failedDagRetentionThread =
+ new DagManager.FailedDagRetentionThread(failedDagStateStore,
failedDagIds, failedDagRetentionTime);
+
this.scheduledExecutorPool.scheduleAtFixedRate(failedDagRetentionThread, 0,
retentionPollingInterval, TimeUnit.MINUTES);
+ loadDagFromDagStateStore();
+ this.houseKeepingThreadPool =
Executors.newSingleThreadScheduledExecutor(new
NamedThreadFactory("LoadDagsThread"));
+ for (int delay = INITIAL_HOUSEKEEPING_THREAD_DELAY; delay <
MAX_HOUSEKEEPING_THREAD_DELAY; delay *= 2) {
+ this.houseKeepingThreadPool.schedule(() -> {
+ try {
+ loadDagFromDagStateStore();
+ } catch (Exception e) {
+ log.error("failed to sync dag state store due to ", e);
+ }
+ }, delay, TimeUnit.MINUTES);
+ }
+ } else { //Mark the DagManager inactive.
+ log.info("Inactivating the DagManager. Shutting down all DagManager
threads");
+ this.scheduledExecutorPool.shutdown();
+ dagManagerMetrics.cleanup();
+ this.houseKeepingThreadPool.shutdown();
+ try {
+ this.scheduledExecutorPool.awaitTermination(TERMINATION_TIMEOUT,
TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ log.error("Exception encountered when shutting down DagManager
threads.", e);
+ }
+ }
+ } catch (IOException e) {
+ log.error("Exception encountered when activating the new DagManager",
e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static DagStateStore createDagStateStore(Config config, Map<URI,
TopologySpec> topologySpecMap) {
Review Comment:
my first instinct is that this belongs within `DagManagementStateStore`
...along w/ the various CRUD ops that follow
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+
+import javax.naming.OperationNotSupportedException;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.orchestration.proc.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Responsible for polling {@link DagTask}s from {@link DagTaskStream} and
processing the {@link org.apache.gobblin.service.modules.flowgraph.Dag}
+ * based on the type of {@link DagTask} which is determined by the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * Each {@link DagTask} acquires a lease for the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * The {@link DagProcFactory} then provides the appropriate {@link DagProc}
associated with the {@link DagTask}.
+ * The actual work or processing is done by the {@link
DagProc#process(DagManagementStateStore)}
+ */
+
+@Alpha
+@Slf4j
+@Singleton
+public class DagProcessingEngine {
+ public static final String GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX =
ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "dagProcessingEngine.";
+ public static final String DAG_PROCESSING_ENGINE_ENABLED =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "enabled";
+ public static final String NUM_THREADS_KEY =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "numThreads";
+ private static final Integer DEFAULT_NUM_THREADS = 3;
+
+ @Getter private final DagTaskStream dagTaskStream;
+ @Getter DagManagementStateStore dagManagementStateStore;
+
+ @Inject
+ public DagProcessingEngine(Config config, DagTaskStream dagTaskStream,
DagProcFactory dagProcFactory,
+ DagManagementStateStore dagManagementStateStore,
Optional<MultiActiveLeaseArbiter> multiActiveLeaseArbiter) {
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ ScheduledExecutorService scheduledExecutorPool =
+ Executors.newScheduledThreadPool(numThreads, new
NamedThreadFactory("DagProcessingEngineThread"));
+ this.dagTaskStream = dagTaskStream;
+ this.dagManagementStateStore = dagManagementStateStore;
+
+ for (int i=0; i < numThreads; i++) {
+ DagProcEngineThread dagProcEngineThread = new
DagProcEngineThread(dagTaskStream, dagProcFactory, dagManagementStateStore);
+ scheduledExecutorPool.submit(dagProcEngineThread);
+ }
+ }
+
+ public void addDagNodeToRetry(Dag.DagNode<JobExecutionPlan> dagNode) throws
OperationNotSupportedException {
+ // todo - how to add dag action for a dag node? should we create a dag
node action? right now dag action is synonymous to flow action
+ // this.dagTaskStream.addDagTask(new RetryDagTask(dagNode));
+ throw new OperationNotSupportedException();
+ }
+
+ @AllArgsConstructor
+ private static class DagProcEngineThread implements Runnable {
+
+ private DagTaskStream dagTaskStream;
+ private DagProcFactory dagProcFactory;
+ private DagManagementStateStore dagManagementStateStore;
+
+ @Override
+ public void run() {
+ while (true) {
+ DagTask dagTask = dagTaskStream.next(); // blocking call
+ if (dagTask == null) {
+ continue;
Review Comment:
given it's a blocking call, why not block until there's something real to
return? is there an actual circumstance you expect to be returning `null` or
you're just thinking defensively?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java:
##########
@@ -0,0 +1,430 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.Data;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+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.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.TopologySpec;
+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.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.JobStatus;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+import org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor;
+import org.apache.gobblin.service.monitoring.KillFlowEvent;
+import org.apache.gobblin.service.monitoring.ResumeFlowEvent;
+import org.apache.gobblin.service.monitoring.event.JobStatusEvent;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+
+import static org.apache.gobblin.service.ExecutionStatus.*;
+
+
+/**
+ * NewDagManager has these functionalities :
+ * a) manages {@link Dag}s through {@link DagManagementStateStore}.
+ * b) subscribes to {@link JobStatusEvent} sent by {@link
KafkaJobStatusMonitor}
+ * c) spawns a {@link DagManager.FailedDagRetentionThread} that cleans failed
dags.
+ * d) load {@link Dag}s on service-start / set-active.
+ */
+@Slf4j
+@Singleton
+@Data
+public class NewDagManager {
+ public static final String DAG_MANAGER_PREFIX =
"gobblin.service.dagManager.";
+ public static final int DEFAULT_NUM_THREADS = 3;
+ public static final String NUM_THREADS_KEY = DAG_MANAGER_PREFIX +
"numThreads";
+ private static final int TERMINATION_TIMEOUT = 30;
+ private static final String DAG_STATESTORE_CLASS_KEY = DAG_MANAGER_PREFIX +
"dagStateStoreClass";
+ private static final String FAILED_DAG_STATESTORE_PREFIX =
"failedDagStateStore";
+ private static final String FAILED_DAG_RETENTION_TIME_UNIT =
FAILED_DAG_STATESTORE_PREFIX + ".retention.timeUnit";
+ private static final String DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT = "DAYS";
+ private static final String FAILED_DAG_RETENTION_TIME =
FAILED_DAG_STATESTORE_PREFIX + ".retention.time";
+ private static final long DEFAULT_FAILED_DAG_RETENTION_TIME = 7L;
+ // Re-emit the final flow status if not detected within 5 minutes
+ public static final String FAILED_DAG_POLLING_INTERVAL =
FAILED_DAG_STATESTORE_PREFIX + ".retention.pollingIntervalMinutes";
+ public static final int DEFAULT_FAILED_DAG_POLLING_INTERVAL = 60;
+ private static final int INITIAL_HOUSEKEEPING_THREAD_DELAY = 2;
+ private final Config config;
+ private final int retentionPollingInterval;
+
+ private final ScheduledExecutorService scheduledExecutorPool;
+ @Inject UserQuotaManager userQuotaManager;
+ @Inject private FlowCatalog flowCatalog;
+ @Getter private DagStateStore failedDagStateStore;
+ private final boolean dagProcessingEngineEnabled;
+ @Getter private Set<String> failedDagIds;
+ private Map<URI, TopologySpec> topologySpecMap = new HashMap<>();
+ @Getter private DagStateStore dagStateStore;
+ @Getter private final JobStatusRetriever jobStatusRetriever;
+ @Getter private final Timer jobStatusPolledTimer;
+ @Getter private final EventSubmitter eventSubmitter;
+ private final long failedDagRetentionTime;
+ @Getter private static final DagManagerMetrics dagManagerMetrics = new
DagManagerMetrics();
+ private ScheduledExecutorService houseKeepingThreadPool;
+ private volatile boolean isActive = false;
+
+ @Inject(optional=true)
+ protected Optional<DagActionStore> dagActionStore;
+ @Inject(optional=true)
+ @Getter DagManagementStateStore dagManagementStateStore;
+ private static final int MAX_HOUSEKEEPING_THREAD_DELAY = 180;
+ protected final EventBus eventBus;
+ DagTaskStream dagTaskStream;
+
+ @Inject
+ public NewDagManager(Config config, JobStatusRetriever jobStatusRetriever,
Optional<DagActionStore> dagActionStore,
+ DagTaskStream dagTaskStream, DagManagementStateStore
dagManagementStateStore) {
+ this.config = config;
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.dagActionStore = dagActionStore;
+ this.dagTaskStream = dagTaskStream;
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.retentionPollingInterval = ConfigUtils.getInt(config,
FAILED_DAG_POLLING_INTERVAL, DEFAULT_FAILED_DAG_POLLING_INTERVAL);
+ this.eventBus = KafkaJobStatusMonitor.getEventBus();
+ this.eventBus.register(this);
+ this.dagProcessingEngineEnabled = ConfigUtils.getBoolean(config,
DagProcessingEngine.DAG_PROCESSING_ENGINE_ENABLED, false);
+ this.dagManagementStateStore = dagManagementStateStore;
+ MetricContext metricContext =
Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()),
getClass());
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.eventSubmitter = new EventSubmitter.Builder(metricContext,
"org.apache.gobblin.service").build();
+ this.jobStatusRetriever = jobStatusRetriever;
+ TimeUnit timeUnit = TimeUnit.valueOf(ConfigUtils.getString(config,
FAILED_DAG_RETENTION_TIME_UNIT, DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT));
+ this.failedDagRetentionTime =
timeUnit.toMillis(ConfigUtils.getLong(config, FAILED_DAG_RETENTION_TIME,
DEFAULT_FAILED_DAG_RETENTION_TIME));
+ }
+
+ public void setActive(boolean active) throws IOException {
+ if (this.isActive == active) {
+ log.info("DagManager already {}, skipping further actions.", (!active) ?
"inactive" : "active");
+ }
+ this.isActive = active;
+ try {
+ if (this.isActive) {
+ log.info("Activating NewDagManager.");
+ //Initializing state store for persisting Dags.
+ this.dagStateStore = createDagStateStore(config, topologySpecMap);
+ // todo - implement as a kill dag action
+ this.failedDagStateStore =
createDagStateStore(ConfigUtils.getConfigOrEmpty(config,
FAILED_DAG_STATESTORE_PREFIX).withFallback(config),
+ topologySpecMap);
+ this.failedDagIds =
Collections.synchronizedSet(failedDagStateStore.getDagIds());
+ dagManagerMetrics.activate();
+ userQuotaManager.init(dagStateStore.getDags());
+ DagManager.FailedDagRetentionThread failedDagRetentionThread =
+ new DagManager.FailedDagRetentionThread(failedDagStateStore,
failedDagIds, failedDagRetentionTime);
+
this.scheduledExecutorPool.scheduleAtFixedRate(failedDagRetentionThread, 0,
retentionPollingInterval, TimeUnit.MINUTES);
+ loadDagFromDagStateStore();
+ this.houseKeepingThreadPool =
Executors.newSingleThreadScheduledExecutor(new
NamedThreadFactory("LoadDagsThread"));
+ for (int delay = INITIAL_HOUSEKEEPING_THREAD_DELAY; delay <
MAX_HOUSEKEEPING_THREAD_DELAY; delay *= 2) {
+ this.houseKeepingThreadPool.schedule(() -> {
+ try {
+ loadDagFromDagStateStore();
+ } catch (Exception e) {
+ log.error("failed to sync dag state store due to ", e);
+ }
+ }, delay, TimeUnit.MINUTES);
+ }
+ } else { //Mark the DagManager inactive.
+ log.info("Inactivating the DagManager. Shutting down all DagManager
threads");
+ this.scheduledExecutorPool.shutdown();
+ dagManagerMetrics.cleanup();
+ this.houseKeepingThreadPool.shutdown();
+ try {
+ this.scheduledExecutorPool.awaitTermination(TERMINATION_TIMEOUT,
TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ log.error("Exception encountered when shutting down DagManager
threads.", e);
+ }
+ }
+ } catch (IOException e) {
+ log.error("Exception encountered when activating the new DagManager",
e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static DagStateStore createDagStateStore(Config config, Map<URI,
TopologySpec> topologySpecMap) {
+ try {
+ Class<?> dagStateStoreClass =
Class.forName(ConfigUtils.getString(config, DAG_STATESTORE_CLASS_KEY,
FSDagStateStore.class.getName()));
+ return (DagStateStore)
GobblinConstructorUtils.invokeLongestConstructor(dagStateStoreClass, config,
topologySpecMap);
+ } catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void loadDagFromDagStateStore() throws IOException {
+ List<Dag<JobExecutionPlan>> dags = this.dagStateStore.getDags();
+ log.info("Loading " + dags.size() + " dags from dag state store");
+ for (Dag<JobExecutionPlan> dag : dags) {
+ addDag(dag, false, false);
+ }
+ }
+
+ public synchronized void addDagAndRemoveAdhocFlowSpec(FlowSpec flowSpec,
Dag<JobExecutionPlan> dag, boolean persist, boolean setStatus)
+ throws IOException {
+ addDag(dag, persist, setStatus);
+ // Only the active newDagManager should delete the flowSpec
+ if (isActive) {
+ deleteSpecFromCatalogIfAdhoc(flowSpec);
+ }
+ }
+
+ private synchronized void addDag(Dag<JobExecutionPlan> dag, boolean persist,
boolean setStatus) throws IOException {
+ // TODO: Used to track missing dag issue, remove later as needed
+ log.info("Add dag (persist: {}, setStatus: {}): {}", persist, setStatus,
dag);
+ if (!isActive) {
+ log.warn("Skipping add dag because this instance of DagManager is not
active for dag: {}", dag);
+ return;
+ }
+
+ DagManager.DagId dagId = DagManagerUtils.generateDagId(dag);
+ String dagIdString = dagId.toString();
+ if (persist) {
+ // Persist the dag
+ this.dagStateStore.writeCheckpoint(dag);
+ // After persisting the dag, its status will be tracked by active
dagManagers so the action should be deleted
+ // to avoid duplicate executions upon leadership change
+ if (this.dagActionStore.isPresent()) {
+
this.dagActionStore.get().deleteDagAction(dagId.toDagAction(DagActionStore.FlowActionType.LAUNCH));
+ }
+ }
+
+ DagActionStore.DagAction dagAction = new DagActionStore.DagAction(
+ dagId.getFlowGroup(), dagId.getFlowName(), dagId.getFlowExecutionId(),
DagActionStore.FlowActionType.LAUNCH);
+
+ if (dagManagementStateStore.containsDag(dagIdString)) {
+ log.warn("Already tracking a dag with dagId {}, skipping.", dagIdString);
+ return;
+ }
+
+ dagManagementStateStore.addDag(dagIdString, dag);
+ this.dagTaskStream.addDagAction(dagAction);
+
+ if (setStatus) {
+ DagManagerUtils.submitPendingExecStatus(dag, this.eventSubmitter);
+ }
+ }
+
+ private void deleteSpecFromCatalogIfAdhoc(FlowSpec flowSpec) {
+ if (!flowSpec.isScheduled()) {
+ this.flowCatalog.remove(flowSpec.getUri(), new Properties(), false);
+ }
+ }
+
+ public void removeDagActionFromStore(DagActionStore.DagAction dagAction)
throws IOException {
+ if (this.dagActionStore.isPresent()) {
+ this.dagActionStore.get().deleteDagAction(dagAction);
+ }
+ }
+
+ //@Subscribe todo uncomment it when new dag manager is ready for kill
functionality
+ public void handleKillFlowEvent(KillFlowEvent killFlowEvent) {
+ String flowGroup = killFlowEvent.getFlowGroup();
+ String flowName = killFlowEvent.getFlowName();
+ if (killFlowEvent.getFlowExecutionId() != null) {
+ long flowExecutionId = killFlowEvent.getFlowExecutionId();
+ log.info("Received kill request for flow ({}, {}, {})", flowGroup,
flowName, flowExecutionId);
+ // todo - should this be added to the dac action store instead?
+ this.dagTaskStream.addDagAction(new DagActionStore.DagAction(flowGroup,
flowName, String.valueOf(flowExecutionId), DagActionStore.FlowActionType.KILL));
+
+ } else {
+ List<Long> flowExecutionIds =
getJobStatusRetriever().getLatestExecutionIdsForFlow(flowName, flowGroup, 10);
+ log.info("Found {} flows to cancel.", flowExecutionIds.size());
+ for (long flowExecutionId : flowExecutionIds) {
+ handleKillFlowEvent(new KillFlowEvent(flowGroup, flowName,
flowExecutionId));
+ }
+ }
+ }
+
+ @Subscribe
+ public void handleResumeFlowEvent(ResumeFlowEvent resumeFlowEvent) {
+ if (!this.dagProcessingEngineEnabled) {
+ log.info("Not handling resume flow event yet.");
+ return;
+ }
+ String flowGroup = resumeFlowEvent.getFlowGroup();
+ String flowName = resumeFlowEvent.getFlowName();
+ String flowExecutionId =
String.valueOf(resumeFlowEvent.getFlowExecutionId());
+ log.info("Received resume request for flow ({}, {}, {})", flowGroup,
flowName, flowExecutionId);
+
+ // todo - persist dag action instead of directly adding to the stream
+ this.dagTaskStream.addDagAction(new DagActionStore.DagAction(flowGroup,
flowName, String.valueOf(flowExecutionId),
DagActionStore.FlowActionType.RESUME));
+ }
+
+ @Subscribe
+ // todo - this does not sla kill like DagManager does, that functionality is
to move somewhere else
+ public void handleJobStatusEvent(JobStatusEvent jobStatusEvent) {
+ if (!this.dagProcessingEngineEnabled) {
+ return;
+ }
+ Map<String, Set<Dag.DagNode<JobExecutionPlan>>> nextSubmitted =
Maps.newHashMap();
+ List<Dag.DagNode<JobExecutionPlan>> nodesToCleanUp = Lists.newArrayList();
+
+ ExecutionStatus executionStatus = jobStatusEvent.getStatus();
+ JobStatus jobStatus = jobStatusEvent.getJobStatus();
+
+ String dagNodeId = DagManagerUtils.generateDagNodeId(jobStatusEvent);
+ Dag.DagNode<JobExecutionPlan> dagNode =
this.dagManagementStateStore.getDagNode(dagNodeId);
+
+ JobExecutionPlan jobExecutionPlan =
DagManagerUtils.getJobExecutionPlan(dagNode);
+
+ try {
+ switch (executionStatus) {
+ case COMPLETE:
+ jobExecutionPlan.setExecutionStatus(COMPLETE);
+ nextSubmitted.putAll(onJobFinish(dagNode));
+ nodesToCleanUp.add(dagNode);
+ break;
+ case FAILED:
+ jobExecutionPlan.setExecutionStatus(FAILED);
+ nextSubmitted.putAll(onJobFinish(dagNode));
+ nodesToCleanUp.add(dagNode);
+ break;
+ case CANCELLED:
+ jobExecutionPlan.setExecutionStatus(CANCELLED);
+ nextSubmitted.putAll(onJobFinish(dagNode));
+ nodesToCleanUp.add(dagNode);
+ break;
+ case PENDING:
+ jobExecutionPlan.setExecutionStatus(PENDING);
+ break;
+ case PENDING_RETRY:
+ jobExecutionPlan.setExecutionStatus(PENDING_RETRY);
+ break;
+ default:
+ jobExecutionPlan.setExecutionStatus(RUNNING);
+ break;
+ }
+
+ if (jobStatus.isShouldRetry()) {
+ log.info("Retrying job: {}, current attempts: {}, max attempts: {}",
DagManagerUtils.getFullyQualifiedJobName(dagNode),
+ jobStatus.getCurrentAttempts(), jobStatus.getMaxAttempts());
+
this.dagManagementStateStore.getDagForJob(dagNode).setFlowEvent(null);
+ // todo - add a retry dagnode dag action
+ }
+ } catch (Exception e) {
+ // Error occurred while processing dag, continue processing other dags
assigned to this thread
+ log.error(String.format("Exception caught in DagManager while
processing dag %s due to ",
+ DagManagerUtils.getFullyQualifiedDagName(dagNode)), e);
+ }
+
+ for (Map.Entry<String, Set<Dag.DagNode<JobExecutionPlan>>> entry:
nextSubmitted.entrySet()) {
+ String nextDagId = entry.getKey();
+ Set<Dag.DagNode<JobExecutionPlan>> dagNodes = entry.getValue();
+ for (Dag.DagNode<JobExecutionPlan> nextDagNode: dagNodes) {
+ this.dagManagementStateStore.addJobState(nextDagId, nextDagNode);
+ }
+ }
+
+ for (Dag.DagNode<JobExecutionPlan> dagNodeToClean: nodesToCleanUp) {
+ String dagId = DagManagerUtils.generateDagId(dagNodeToClean).toString();
+ this.dagManagementStateStore.deleteJobState(dagId, dagNodeToClean);
+ }
+ }
+
+ /**
+ * Method that defines the actions to be performed when a job finishes
either successfully or with failure.
+ * This method updates the state of the dag and performs clean up actions as
necessary.
+ */
+ public Map<String, Set<Dag.DagNode<JobExecutionPlan>>>
onJobFinish(Dag.DagNode<JobExecutionPlan> dagNode)
+ throws IOException {
+ Dag<JobExecutionPlan> dag =
this.dagManagementStateStore.getDagForJob(dagNode);
+ String dagId = DagManagerUtils.generateDagId(dag).toString();
+ String jobName = DagManagerUtils.getFullyQualifiedJobName(dagNode);
+ ExecutionStatus jobStatus = DagManagerUtils.getExecutionStatus(dagNode);
+ log.info("Job {} of Dag {} has finished with status {}", jobName, dagId,
jobStatus.name());
+ // Only decrement counters and quota for jobs that actually ran on the
executor, not from a GaaS side failure/skip event
+ if (this.userQuotaManager.releaseQuota(dagNode)) {
+ dagManagerMetrics.decrementRunningJobMetrics(dagNode);
+ }
+
+ switch (jobStatus) {
+ case FAILED:
+ dag.setMessage("Flow failed because job " + jobName + " failed");
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_FAILED);
+ dagManagerMetrics.incrementExecutorFailed(dagNode);
+ return Maps.newHashMap();
+ case CANCELLED:
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_CANCELLED);
+ return Maps.newHashMap();
+ case COMPLETE:
+ dagManagerMetrics.incrementExecutorSuccess(dagNode);
+ return submitNext(dagId);
+ default:
+ log.warn("It should not reach here. Job status is unexpected.");
+ return Maps.newHashMap();
+ }
+ }
+
+ /**
+ * Submit next set of Dag nodes in the Dag identified by the provided dagId
+ * @param dagId The dagId that should be processed.
+ * @return
+ * @throws IOException
+ */
+ private Map<String, Set<Dag.DagNode<JobExecutionPlan>>> submitNext(String
dagId) throws IOException {
+ Dag<JobExecutionPlan> dag = this.dagManagementStateStore.getDag(dagId);
+ Set<Dag.DagNode<JobExecutionPlan>> nextNodes =
DagManagerUtils.getNext(dag);
+ List<String> nextJobNames = new ArrayList<>();
+
+ //Submit jobs from the dag ready for execution.
+ for (Dag.DagNode<JobExecutionPlan> dagNode : nextNodes) {
+
this.dagTaskStream.addDagAction(DagManagerUtils.createDagActionFromDagNode(dagNode,
DagActionStore.FlowActionType.ADVANCE));
Review Comment:
let's first solidify the rest of this PR (esp. around `DagMgmtStateStore`)
and the let's revisit this Q: do we need this `NewDM` intermediary, or could
the `DagTaskStream` itself be the `EventBus` subscriber to the messages the
KJSM sends?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import com.google.common.base.Optional;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.orchestration.proc.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Responsible for polling {@link DagTask}s from {@link DagTaskStream} and
processing the {@link org.apache.gobblin.service.modules.flowgraph.Dag}
+ * based on the type of {@link DagTask} which is determined by the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * Each {@link DagTask} acquires a lease for the {@link
org.apache.gobblin.runtime.api.DagActionStore.DagAction}.
+ * The {@link DagProcFactory} then provides the appropriate {@link DagProc}
associated with the {@link DagTask}.
+ * The actual work or processing is done by the {@link
DagProc#process(DagManagementStateStore)}
+ */
+
+@Alpha
+@Slf4j
+@Singleton
+public class DagProcessingEngine {
+ public static final String GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX =
ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "dagProcessingEngine.";
+ public static final String DAG_PROCESSING_ENGINE_ENABLED =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "enabled";
+ public static final String NUM_THREADS_KEY =
GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "numThreads";
+ private static final Integer DEFAULT_NUM_THREADS = 3;
Review Comment:
I know what you mean, but in this situation, whether or not to enable the
DagProcEng functionality is pretty fundamental to be more or less "global".
I'd recommend in `ConfigurationKeys` for centralized discoverability
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java:
##########
@@ -0,0 +1,430 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.linkedin.r2.util.NamedThreadFactory;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import lombok.Data;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.ServiceMetricNames;
+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.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.TopologySpec;
+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.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.JobStatus;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+import org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor;
+import org.apache.gobblin.service.monitoring.KillFlowEvent;
+import org.apache.gobblin.service.monitoring.ResumeFlowEvent;
+import org.apache.gobblin.service.monitoring.event.JobStatusEvent;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+
+import static org.apache.gobblin.service.ExecutionStatus.*;
+
+
+/**
+ * NewDagManager has these functionalities :
+ * a) manages {@link Dag}s through {@link DagManagementStateStore}.
+ * b) subscribes to {@link JobStatusEvent} sent by {@link
KafkaJobStatusMonitor}
+ * c) spawns a {@link DagManager.FailedDagRetentionThread} that cleans failed
dags.
+ * d) load {@link Dag}s on service-start / set-active.
+ */
+@Slf4j
+@Singleton
+@Data
+public class NewDagManager {
+ public static final String DAG_MANAGER_PREFIX =
"gobblin.service.dagManager.";
+ public static final int DEFAULT_NUM_THREADS = 3;
+ public static final String NUM_THREADS_KEY = DAG_MANAGER_PREFIX +
"numThreads";
+ private static final int TERMINATION_TIMEOUT = 30;
+ private static final String DAG_STATESTORE_CLASS_KEY = DAG_MANAGER_PREFIX +
"dagStateStoreClass";
+ private static final String FAILED_DAG_STATESTORE_PREFIX =
"failedDagStateStore";
+ private static final String FAILED_DAG_RETENTION_TIME_UNIT =
FAILED_DAG_STATESTORE_PREFIX + ".retention.timeUnit";
+ private static final String DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT = "DAYS";
+ private static final String FAILED_DAG_RETENTION_TIME =
FAILED_DAG_STATESTORE_PREFIX + ".retention.time";
+ private static final long DEFAULT_FAILED_DAG_RETENTION_TIME = 7L;
+ // Re-emit the final flow status if not detected within 5 minutes
+ public static final String FAILED_DAG_POLLING_INTERVAL =
FAILED_DAG_STATESTORE_PREFIX + ".retention.pollingIntervalMinutes";
+ public static final int DEFAULT_FAILED_DAG_POLLING_INTERVAL = 60;
+ private static final int INITIAL_HOUSEKEEPING_THREAD_DELAY = 2;
+ private final Config config;
+ private final int retentionPollingInterval;
+
+ private final ScheduledExecutorService scheduledExecutorPool;
+ @Inject UserQuotaManager userQuotaManager;
+ @Inject private FlowCatalog flowCatalog;
+ @Getter private DagStateStore failedDagStateStore;
+ private final boolean dagProcessingEngineEnabled;
+ @Getter private Set<String> failedDagIds;
+ private Map<URI, TopologySpec> topologySpecMap = new HashMap<>();
+ @Getter private DagStateStore dagStateStore;
+ @Getter private final JobStatusRetriever jobStatusRetriever;
+ @Getter private final Timer jobStatusPolledTimer;
+ @Getter private final EventSubmitter eventSubmitter;
+ private final long failedDagRetentionTime;
+ @Getter private static final DagManagerMetrics dagManagerMetrics = new
DagManagerMetrics();
+ private ScheduledExecutorService houseKeepingThreadPool;
+ private volatile boolean isActive = false;
+
+ @Inject(optional=true)
+ protected Optional<DagActionStore> dagActionStore;
+ @Inject(optional=true)
+ @Getter DagManagementStateStore dagManagementStateStore;
+ private static final int MAX_HOUSEKEEPING_THREAD_DELAY = 180;
+ protected final EventBus eventBus;
+ DagTaskStream dagTaskStream;
+
+ @Inject
+ public NewDagManager(Config config, JobStatusRetriever jobStatusRetriever,
Optional<DagActionStore> dagActionStore,
+ DagTaskStream dagTaskStream, DagManagementStateStore
dagManagementStateStore) {
+ this.config = config;
+ Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY,
DEFAULT_NUM_THREADS);
+ this.dagActionStore = dagActionStore;
+ this.dagTaskStream = dagTaskStream;
+ this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads);
+ this.retentionPollingInterval = ConfigUtils.getInt(config,
FAILED_DAG_POLLING_INTERVAL, DEFAULT_FAILED_DAG_POLLING_INTERVAL);
+ this.eventBus = KafkaJobStatusMonitor.getEventBus();
+ this.eventBus.register(this);
+ this.dagProcessingEngineEnabled = ConfigUtils.getBoolean(config,
DagProcessingEngine.DAG_PROCESSING_ENGINE_ENABLED, false);
+ this.dagManagementStateStore = dagManagementStateStore;
+ MetricContext metricContext =
Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()),
getClass());
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.eventSubmitter = new EventSubmitter.Builder(metricContext,
"org.apache.gobblin.service").build();
+ this.jobStatusRetriever = jobStatusRetriever;
+ TimeUnit timeUnit = TimeUnit.valueOf(ConfigUtils.getString(config,
FAILED_DAG_RETENTION_TIME_UNIT, DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT));
+ this.failedDagRetentionTime =
timeUnit.toMillis(ConfigUtils.getLong(config, FAILED_DAG_RETENTION_TIME,
DEFAULT_FAILED_DAG_RETENTION_TIME));
+ }
+
+ public void setActive(boolean active) throws IOException {
+ if (this.isActive == active) {
+ log.info("DagManager already {}, skipping further actions.", (!active) ?
"inactive" : "active");
+ }
+ this.isActive = active;
+ try {
+ if (this.isActive) {
+ log.info("Activating NewDagManager.");
+ //Initializing state store for persisting Dags.
+ this.dagStateStore = createDagStateStore(config, topologySpecMap);
+ // todo - implement as a kill dag action
+ this.failedDagStateStore =
createDagStateStore(ConfigUtils.getConfigOrEmpty(config,
FAILED_DAG_STATESTORE_PREFIX).withFallback(config),
+ topologySpecMap);
+ this.failedDagIds =
Collections.synchronizedSet(failedDagStateStore.getDagIds());
+ dagManagerMetrics.activate();
+ userQuotaManager.init(dagStateStore.getDags());
+ DagManager.FailedDagRetentionThread failedDagRetentionThread =
+ new DagManager.FailedDagRetentionThread(failedDagStateStore,
failedDagIds, failedDagRetentionTime);
+
this.scheduledExecutorPool.scheduleAtFixedRate(failedDagRetentionThread, 0,
retentionPollingInterval, TimeUnit.MINUTES);
+ loadDagFromDagStateStore();
+ this.houseKeepingThreadPool =
Executors.newSingleThreadScheduledExecutor(new
NamedThreadFactory("LoadDagsThread"));
+ for (int delay = INITIAL_HOUSEKEEPING_THREAD_DELAY; delay <
MAX_HOUSEKEEPING_THREAD_DELAY; delay *= 2) {
+ this.houseKeepingThreadPool.schedule(() -> {
+ try {
+ loadDagFromDagStateStore();
+ } catch (Exception e) {
+ log.error("failed to sync dag state store due to ", e);
+ }
+ }, delay, TimeUnit.MINUTES);
+ }
+ } else { //Mark the DagManager inactive.
+ log.info("Inactivating the DagManager. Shutting down all DagManager
threads");
+ this.scheduledExecutorPool.shutdown();
+ dagManagerMetrics.cleanup();
+ this.houseKeepingThreadPool.shutdown();
+ try {
+ this.scheduledExecutorPool.awaitTermination(TERMINATION_TIMEOUT,
TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ log.error("Exception encountered when shutting down DagManager
threads.", e);
+ }
+ }
+ } catch (IOException e) {
+ log.error("Exception encountered when activating the new DagManager",
e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static DagStateStore createDagStateStore(Config config, Map<URI,
TopologySpec> topologySpecMap) {
+ try {
+ Class<?> dagStateStoreClass =
Class.forName(ConfigUtils.getString(config, DAG_STATESTORE_CLASS_KEY,
FSDagStateStore.class.getName()));
+ return (DagStateStore)
GobblinConstructorUtils.invokeLongestConstructor(dagStateStoreClass, config,
topologySpecMap);
+ } catch (ReflectiveOperationException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void loadDagFromDagStateStore() throws IOException {
+ List<Dag<JobExecutionPlan>> dags = this.dagStateStore.getDags();
+ log.info("Loading " + dags.size() + " dags from dag state store");
+ for (Dag<JobExecutionPlan> dag : dags) {
+ addDag(dag, false, false);
+ }
+ }
+
+ public synchronized void addDagAndRemoveAdhocFlowSpec(FlowSpec flowSpec,
Dag<JobExecutionPlan> dag, boolean persist, boolean setStatus)
+ throws IOException {
+ addDag(dag, persist, setStatus);
+ // Only the active newDagManager should delete the flowSpec
+ if (isActive) {
+ deleteSpecFromCatalogIfAdhoc(flowSpec);
+ }
+ }
+
+ private synchronized void addDag(Dag<JobExecutionPlan> dag, boolean persist,
boolean setStatus) throws IOException {
+ // TODO: Used to track missing dag issue, remove later as needed
+ log.info("Add dag (persist: {}, setStatus: {}): {}", persist, setStatus,
dag);
+ if (!isActive) {
+ log.warn("Skipping add dag because this instance of DagManager is not
active for dag: {}", dag);
+ return;
+ }
+
+ DagManager.DagId dagId = DagManagerUtils.generateDagId(dag);
+ String dagIdString = dagId.toString();
+ if (persist) {
+ // Persist the dag
+ this.dagStateStore.writeCheckpoint(dag);
+ // After persisting the dag, its status will be tracked by active
dagManagers so the action should be deleted
+ // to avoid duplicate executions upon leadership change
+ if (this.dagActionStore.isPresent()) {
+
this.dagActionStore.get().deleteDagAction(dagId.toDagAction(DagActionStore.FlowActionType.LAUNCH));
+ }
+ }
+
+ DagActionStore.DagAction dagAction = new DagActionStore.DagAction(
+ dagId.getFlowGroup(), dagId.getFlowName(), dagId.getFlowExecutionId(),
DagActionStore.FlowActionType.LAUNCH);
+
+ if (dagManagementStateStore.containsDag(dagIdString)) {
+ log.warn("Already tracking a dag with dagId {}, skipping.", dagIdString);
+ return;
+ }
+
+ dagManagementStateStore.addDag(dagIdString, dag);
+ this.dagTaskStream.addDagAction(dagAction);
+
+ if (setStatus) {
+ DagManagerUtils.submitPendingExecStatus(dag, this.eventSubmitter);
+ }
+ }
+
+ private void deleteSpecFromCatalogIfAdhoc(FlowSpec flowSpec) {
+ if (!flowSpec.isScheduled()) {
+ this.flowCatalog.remove(flowSpec.getUri(), new Properties(), false);
+ }
+ }
+
+ public void removeDagActionFromStore(DagActionStore.DagAction dagAction)
throws IOException {
+ if (this.dagActionStore.isPresent()) {
+ this.dagActionStore.get().deleteDagAction(dagAction);
+ }
+ }
+
+ //@Subscribe todo uncomment it when new dag manager is ready for kill
functionality
+ public void handleKillFlowEvent(KillFlowEvent killFlowEvent) {
+ String flowGroup = killFlowEvent.getFlowGroup();
+ String flowName = killFlowEvent.getFlowName();
+ if (killFlowEvent.getFlowExecutionId() != null) {
+ long flowExecutionId = killFlowEvent.getFlowExecutionId();
+ log.info("Received kill request for flow ({}, {}, {})", flowGroup,
flowName, flowExecutionId);
+ // todo - should this be added to the dac action store instead?
+ this.dagTaskStream.addDagAction(new DagActionStore.DagAction(flowGroup,
flowName, String.valueOf(flowExecutionId), DagActionStore.FlowActionType.KILL));
+
+ } else {
+ List<Long> flowExecutionIds =
getJobStatusRetriever().getLatestExecutionIdsForFlow(flowName, flowGroup, 10);
+ log.info("Found {} flows to cancel.", flowExecutionIds.size());
+ for (long flowExecutionId : flowExecutionIds) {
+ handleKillFlowEvent(new KillFlowEvent(flowGroup, flowName,
flowExecutionId));
+ }
+ }
+ }
+
+ @Subscribe
+ public void handleResumeFlowEvent(ResumeFlowEvent resumeFlowEvent) {
+ if (!this.dagProcessingEngineEnabled) {
+ log.info("Not handling resume flow event yet.");
+ return;
+ }
+ String flowGroup = resumeFlowEvent.getFlowGroup();
+ String flowName = resumeFlowEvent.getFlowName();
+ String flowExecutionId =
String.valueOf(resumeFlowEvent.getFlowExecutionId());
+ log.info("Received resume request for flow ({}, {}, {})", flowGroup,
flowName, flowExecutionId);
+
+ // todo - persist dag action instead of directly adding to the stream
+ this.dagTaskStream.addDagAction(new DagActionStore.DagAction(flowGroup,
flowName, String.valueOf(flowExecutionId),
DagActionStore.FlowActionType.RESUME));
+ }
+
+ @Subscribe
+ // todo - this does not sla kill like DagManager does, that functionality is
to move somewhere else
+ public void handleJobStatusEvent(JobStatusEvent jobStatusEvent) {
+ if (!this.dagProcessingEngineEnabled) {
+ return;
+ }
+ Map<String, Set<Dag.DagNode<JobExecutionPlan>>> nextSubmitted =
Maps.newHashMap();
+ List<Dag.DagNode<JobExecutionPlan>> nodesToCleanUp = Lists.newArrayList();
+
+ ExecutionStatus executionStatus = jobStatusEvent.getStatus();
+ JobStatus jobStatus = jobStatusEvent.getJobStatus();
+
+ String dagNodeId = DagManagerUtils.generateDagNodeId(jobStatusEvent);
+ Dag.DagNode<JobExecutionPlan> dagNode =
this.dagManagementStateStore.getDagNode(dagNodeId);
+
+ JobExecutionPlan jobExecutionPlan =
DagManagerUtils.getJobExecutionPlan(dagNode);
+
+ try {
+ switch (executionStatus) {
+ case COMPLETE:
+ jobExecutionPlan.setExecutionStatus(COMPLETE);
+ nextSubmitted.putAll(onJobFinish(dagNode));
+ nodesToCleanUp.add(dagNode);
+ break;
+ case FAILED:
+ jobExecutionPlan.setExecutionStatus(FAILED);
+ nextSubmitted.putAll(onJobFinish(dagNode));
+ nodesToCleanUp.add(dagNode);
+ break;
Review Comment:
if using `switch`, let's use `case` fall-through to our advantage...
otherwise use `if/else` to capture that there are truly only 3 alternatives.
e.g.
```
switch (execStatus) {
case COMPLETE:
case FAILED:
case CANCELLED:
jobExecPlan.setExecStatus(execStatus);
nextSub.putAll(onFin(...));
nodesToClean.add(...);
break
case PENDING:
case PENDING_RETRY:
jobExecPlan.setExecStatus(execStatus);
break;
default:
...
}
```
Issue Time Tracking
-------------------
Worklog Id: (was: 904497)
Time Spent: 19h (was: 18h 50m)
> 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: 19h
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)