[
https://issues.apache.org/jira/browse/GOBBLIN-1910?focusedWorklogId=908501&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-908501
]
ASF GitHub Bot logged work on GOBBLIN-1910:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 06/Mar/24 07:32
Start Date: 06/Mar/24 07:32
Worklog Time Spent: 10m
Work Description: phet commented on code in PR #3858:
URL: https://github.com/apache/gobblin/pull/3858#discussion_r1513733706
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementTaskStreamImpl.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+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.event.EventSubmitter;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.service.modules.orchestration.proc.DagProc;
+import org.apache.gobblin.service.modules.orchestration.task.DagTask;
+import org.apache.gobblin.service.modules.orchestration.task.LaunchDagTask;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * NewDagManager has these functionalities :
+ * a) interact with {@link DagManagementStateStore} to update/retrieve dags,
checkpoint
+ * b) add {@link org.apache.gobblin.runtime.api.DagActionStore.DagAction} to
the {@link DagTaskStream}
+ */
+@Slf4j
+@Singleton
+@Data
+public class DagManagementTaskStreamImpl implements DagManagement,
DagTaskStream {
+ private final Config config;
+ @Getter private final EventSubmitter eventSubmitter;
+ @Getter private static final DagManagerMetrics dagManagerMetrics = new
DagManagerMetrics();
+ private volatile boolean isActive = false;
+
+ @Inject(optional=true)
+ protected Optional<DagActionStore> dagActionStore;
+ @Inject
+ @Getter DagManagementStateStore dagManagementStateStore;
+ private static final int MAX_HOUSEKEEPING_THREAD_DELAY = 180;
+ private final BlockingQueue<DagActionStore.DagAction> dagActionQueue = new
LinkedBlockingQueue<>();
+
+ @Inject
+ public DagManagementTaskStreamImpl(Config config, Optional<DagActionStore>
dagActionStore, DagManagementStateStore dagManagementStateStore) {
+ this.config = config;
+ this.dagActionStore = dagActionStore;
+ this.dagManagementStateStore = dagManagementStateStore;
+ MetricContext metricContext =
Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()),
getClass());
+ this.eventSubmitter = new EventSubmitter.Builder(metricContext,
"org.apache.gobblin.service").build();
+ }
+
+ public void setActive(boolean active) {
+ if (this.isActive == active) {
+ log.info("DagManagementTaskStreamImpl already {}, skipping further
actions.", (!active) ? "inactive" : "active");
+ }
+ this.isActive = active;
+ try {
+ if (this.isActive) {
+ log.info("Activating DagManagementTaskStreamImpl.");
+ //Initializing state store for persisting Dags.
+ this.dagManagementStateStore.start();
+ dagManagerMetrics.activate();
+ } else { //Mark the DagManager inactive.
+ log.info("Inactivating the DagManagementTaskStreamImpl. Shutting down
all DagManager threads");
+ dagManagerMetrics.cleanup();
+ }
+ } catch (IOException e) {
+ log.error("Exception encountered when activating the
DagManagementTaskStreamImpl", e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public synchronized void addDagAction(DagActionStore.DagAction dagAction)
throws IOException {
+ // TODO: Used to track missing dag issue, remove later as needed
+ log.info("Add dagAction{}", dagAction);
+ if (!isActive) {
+ log.warn("Skipping add dagAction because this instance of
DagManagementTaskStreamImpl is not active for dag: {}",
+ dagAction);
+ return;
+ }
+
+ DagManager.DagId dagId =
DagManagerUtils.generateDagId(dagAction.getFlowGroup(),
dagAction.getFlowName(), dagAction.getFlowExecutionId());
+ String dagIdString = dagId.toString();
+ if (this.dagManagementStateStore.containsDag(dagId)) {
+ log.warn("Already tracking a dag with dagId {}, skipping.", dagIdString);
+ return;
+ }
+
+ // 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(dagAction);
+ }
Review Comment:
seems naively that if multiple active participants that they'd all try to
delete the same dag action from the shared store.
or are you relying on some trick, such as that `dagActionStore.isPresent()`
is only true for active participants?
perhaps it would be sensible to limit deletion to the participant currently
holding a lease for that same dag action
Issue Time Tracking
-------------------
Worklog Id: (was: 908501)
Time Spent: 27h 50m (was: 27h 40m)
> Refactor code to move current in-memory references to new design for REST
> calls: Launch, Resume and Kill
> --------------------------------------------------------------------------------------------------------
>
> Key: GOBBLIN-1910
> URL: https://issues.apache.org/jira/browse/GOBBLIN-1910
> Project: Apache Gobblin
> Issue Type: New Feature
> Reporter: Meeth Gala
> Priority: Major
> Time Spent: 27h 50m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)