You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by "umustafi (via GitHub)" <gi...@apache.org> on 2023/05/26 21:06:43 UTC

[GitHub] [gobblin] umustafi opened a new pull request, #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

umustafi opened a new pull request, #3700:
URL: https://github.com/apache/gobblin/pull/3700

   Dear Gobblin maintainers,
   
   Please accept this PR. I understand that it will not be reviewed until I have checked off all the steps below!
   
   
   ### JIRA
   - [X] My PR addresses the following [Gobblin JIRA](https://issues.apache.org/jira/browse/GOBBLIN/) issues and references them in the PR title. For example, "[GOBBLIN-XXX] My Gobblin PR"
       - https://issues.apache.org/jira/browse/GOBBLIN-1837
   
   
   ### Description
   - [X] Here are some details about my PR, including screenshots (if applicable):
   This task will include the implementation of non-blocking, multi-active scheduler for each host. It will NOT include metric emission or unit tests for validation. That will be done in a separate follow-up ticket. The work in this ticket includes
   
   - define a table to do scheduler lease determination for each flow's trigger event and related methods to execute actions on this table 
   - update DagActionStore schema and DagActionStoreMonitor to act upon new "LAUNCH" type events in addition to KILL/RESUME
   - update scheduler/orchestrator logic to apply the non-blocking algorithm when "multi-active scheduler mode" is enabled, otherwise submit events directly to the DagManager after receiving a scheduler trigger
   - implement the non-blocking algorithm, particularly handling reminder events if another host is in the process of securing the lease for a particular flow trigger
   
   NOTE: because I'm updating the DagActionStore schema this change will require manually altering the primary key of the table before deploying these changes. MySQL only creates/updates the table if the same table name does not exist.
   
   ### Tests
   - [X] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   Limiting the scope of this PR to the implementation and will focus on metrics, logging for validation, and unit tests in a separate PR. 
   
   ### Commits
   - [X] My commits all reference JIRA issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
       1. Subject is separated from body by a blank line
       2. Subject is limited to 50 characters
       3. Subject does not end with a period
       4. Subject uses the imperative mood ("add", not "adding")
       5. Body wraps at 72 characters
       6. Body explains "what" and "why", not "how"
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded the leases expiration.  each host must defer within the "linger" period, and should only attempt to acquire after that expires.  if the host should fail because another won the race to acquire it, that newly acquired lease should be nowhere near expiration.
   c. with NO_LONGER_LEASING, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even set a reminder.  in fact, under normal operating conditions, such a self-timer should regularly conclude by the host determining "there's nothing to lease", because the trigger event has been fully handled.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211093441


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -27,7 +27,9 @@
 public interface DagActionStore {
   enum DagActionValue {
     KILL,
-    RESUME
+    RESUME,
+    // TODO: potentially combine this enum with {@link SchedulerLeaseDeterminationStore.FlowActionType}
+    LAUNCH
   }
 
   @Getter

Review Comment:
   I imagined simply:
   ```
   @Data
   class DagAction {
       final String flowGroup;
       final String flowName;
       final String flowExecutionId;
       final DagActionValue dagActionValue;
   }
   ```
   
   also, not a bad idea to reuse enums, whenever perfect overlap



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1213713080


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";

Review Comment:
   Removing it as primary key so we do event consolidation for the same flow action and adding explanation to the JavaDoc



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] codecov-commenter commented on pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#issuecomment-1581229792

   ## [Codecov](https://app.codecov.io/gh/apache/gobblin/pull/3700?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#3700](https://app.codecov.io/gh/apache/gobblin/pull/3700?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (82708a1) into [master](https://app.codecov.io/gh/apache/gobblin/commit/7bbf6761c63055eb9ecc9f2756d4b3d68b7a1b08?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (7bbf676) will **decrease** coverage by `2.12%`.
   > The diff coverage is `n/a`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3700      +/-   ##
   ============================================
   - Coverage     46.98%   44.87%   -2.12%     
   + Complexity    10792     2095    -8697     
   ============================================
     Files          2138      411    -1727     
     Lines         84065    17733   -66332     
     Branches       9342     2162    -7180     
   ============================================
   - Hits          39501     7957   -31544     
   + Misses        40973     8917   -32056     
   + Partials       3591      859    -2732     
   ```
   
   
   [see 1730 files with indirect coverage changes](https://app.codecov.io/gh/apache/gobblin/pull/3700/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227294488


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+    try {
+      this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
+      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
+        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+        this.numLeasesCompleted.mark();
+        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
+            status.getMyLeaseAcquisitionTimestamp());
+      }
+    } catch (IOException | SQLException e) {
+      throw new RuntimeException(e);
+    }
+    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
+    return false;
+  }
+
+  /**
+   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
+   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
+   * expire.
+   * @param jobProps
+   * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
+   * @param originalEventTimeMillis the event timestamp we were originally handling
+   * @param flowAction
+   */
+  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+      DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
+    // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
+    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
+    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);

Review Comment:
   I added some comments to clarify but there is an existing function in `JobScheduler` named `getTrigger` (will rename this to `createTrigger`) we are using to create a new Trigger for the job that will fire after the lease should expire and passing it to the `SchedulerService` to schedule it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227677700


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {

Review Comment:
   that name is fine.  "perform action" is too.
   
   since the class was named `SchedulerLeaseAlgoHandler`, I was presuming that the expected/only action would be launching the flow... but perhaps this is more general than I perceived... 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227819575


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -100,8 +114,8 @@ public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
   // Insert or update row to acquire lease if values have not changed since the previous read
   // Need to define three separate statements to handle cases where row does not exist or has null values to check
   protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
-      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
-      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?); "
+      + SELECT_AFTER_INSERT_STATEMENT;

Review Comment:
   just thinking... it may be clearer not to append `SELECT_AFTER_INSERT_STATEMENT` to many constants, but rather to catenate them at point of use (so the maintainer easily spots it)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211263838


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -54,27 +53,26 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending action " + action + " for this flow. Please wait to resubmit and wait for"
+      // If an existing resume request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME,
+            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
                 + " action to be completed."));
         return;
       }
       this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME);
-    } catch (IOException | SQLException | SpecNotFoundException e) {
+    } catch (IOException | SQLException e) {
       log.warn(
           String.format("Failed to add execution resume action for flow %s %s %s to dag action store due to", flowGroup,
               flowName, flowExecutionId), e);
-      this.handleException(flowGroup, flowName, flowExecutionId.toString(), e);
+      this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME, e);
     }
 
   }
 
-  private void handleException (String flowGroup, String flowName, String flowExecutionId, Exception e) {
+  private void handleException (String flowGroup, String flowName, String flowExecutionId, DagActionStore.DagActionValue dagActionValue, Exception e) {

Review Comment:
   I'm not clear whether I've missed something, but is `Exception e` used merely to `e.getMessage()`?  if so, why are we passing in an exception, which generates a stacktrace, etc. (at the place where it's constructed), rather than merely passing in a `String errMessage`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211266080


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -91,21 +89,20 @@ public UpdateResponse delete(ComplexResourceKey<org.apache.gobblin.service.FlowS
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending " + action + " action for this flow. Please wait to resubmit and wait for"
+      // If an existing kill request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.KILL)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.KILL,
+            new RuntimeException("There is already a pending KILL action for this flow. Please wait to resubmit and wait for"

Review Comment:
   feels like déjà vu... I believe I just read this code above, but now w/ `s/RESUME/KILL/g`.  if so, can't we DRY it up?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210912630


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -310,6 +316,17 @@ public void orchestrate(Spec spec) throws Exception {
         flowCompilationTimer.get().stop(flowMetadata);
       }
 
+      // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
+      if (this.isMultiActiveSchedulerEnabled) {
+        String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+        boolean leaseAttemptSucceeded = schedulerLeaseAlgoHandler.handleNewTriggerEvent(jobProps, flowGroup, flowName,
+            flowExecutionId, SchedulerLeaseDeterminationStore.FlowActionType.LAUNCH, triggerTimestampMillis);
+        _log.info("scheduler attempted lease on flowGroup: %s, flowName: %s, flowExecutionId: %s, LAUNCH event for "

Review Comment:
   This is a great suggestion. It'll also be useful to keep the prefix and formatting constant across log events for the scheduler lease obtaining so we can easily grep to understand what happened for each host. 
   `Multi-active scheduler lease attempt: [flowGroup: 'foo-grp', flowName: 'bar-fn', execId: 1234, dagAction: LAUNCH] - triggerEvent: 98765 - SUCCESS/FAILURE `
   
   that way `Multi-active scheduler lease attempt: [flowGroup: 'foo-grp', flowName: 'bar-fn', execId: 1234, dagAction: LAUNCH] - ` can always be the prefix and followed by a more descriptive log message if we want to add one



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded its expiration.  why?  because each host must defer for the "linger" period, and should only attempt to acquire AFTER that expires.  given that, if acquisition should fail because another won the race, that newly granted lease should be nowhere near expiration.
   c. with NO_LONGER_LEASING, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even set a reminder.  in fact, under normal operating conditions, such a self-timer should regularly conclude by the host determining "there's nothing to lease", because the trigger event has been fully handled.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222063860


##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,20 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "scheduler.lease.determination.store.db.table";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "gobblin_scheduler_lease_determination_store";
+  public static final String SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY = "reminderEventTimestampMillis";
+  public static final String SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY = "newEventTimestampMillis";
+  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS = 100;
+  // Note: linger should be on the order of seconds even though we measure in millis
+  public static final String SCHEDULER_EVENT_LINGER_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS = 30000;
+  public static final String SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY = "";

Review Comment:
   `UPPER_BOUND` => `MAX` ?
   
   `STAGGERING` => `DELAY` or `BACKOFF` or `DEFERENCE`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1220128830


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.SchedulerLeaseDeterminationStore;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final long linger;
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected SchedulerLeaseDeterminationStore leaseDeterminationStore;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, SchedulerLeaseDeterminationStore leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService)
+      throws IOException {
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.leaseDeterminationStore = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+  }
+  private SchedulerLeaseDeterminationStore schedulerLeaseDeterminationStore;
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow's trigger event
+   * by attempting a lease for the flow event.
+   * @param jobProps
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param flowActionType
+   * @param triggerTimeMillis
+   * @return true if this host obtained the lease for this flow's trigger event, false otherwise.
+   * @throws IOException
+   */
+  public boolean handleNewTriggerEvent(Properties jobProps, String flowGroup, String flowName, String flowExecutionId,
+      SchedulerLeaseDeterminationStore.FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    SchedulerLeaseDeterminationStore.LeaseAttemptStatus leaseAttemptStatus =
+        schedulerLeaseDeterminationStore.attemptInsertAndGetPursuantTimestamp(flowGroup, flowName, flowExecutionId,
+            flowActionType, triggerTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus) {
+      case LEASE_OBTAINED:
+        return true;
+      case PREVIOUS_LEASE_EXPIRED:
+        // recursively try obtaining lease again immediately, stops when reaches one of the other cases
+        return handleNewTriggerEvent(jobProps, flowGroup, flowName, flowExecutionId, flowActionType, triggerTimeMillis);
+      case PREVIOUS_LEASE_VALID:
+        scheduleReminderForTriggerEvent(jobProps, flowGroup, flowName, flowExecutionId, flowActionType, triggerTimeMillis);
+    }
+    return false;
+  }
+
+  /**
+   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewTriggerEvent} to schedule a reminder for itself to
+   * check on the other participant's progress during pursuing orchestration after the time the lease should expire.
+   * If the previous participant was successful, then no further action is taken otherwise we re-attempt pursuing
+   * orchestration ourselves.
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param flowActionType
+   * @param triggerTimeMillis
+   */
+  protected void scheduleReminderForTriggerEvent(Properties jobProps, String flowGroup, String flowName, String flowExecutionId,
+      SchedulerLeaseDeterminationStore.FlowActionType flowActionType, long triggerTimeMillis) {
+    // Check-in `linger` time after the current timestamp which is "close-enough" to the time the pursuant attempted
+    // the flow action. We also add a small randomization to avoid 'thundering herd' issue
+    String cronExpression = createCronFromDelayPeriod(linger + random.nextInt(staggerUpperBoundSec));
+    jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
+    // This timestamp is what will be used to identify the particular flow trigger event it's associated with
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_ORIGINAL_TRIGGER_TIMESTAMP_MILLIS_KEY, String.valueOf(triggerTimeMillis));
+    JobKey key = new JobKey(flowName, flowGroup);
+    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);
+    try {
+      LOG.info("Attempting to add job reminder to Scheduler Service where job is %s trigger event %s and reminder is at "
+          + "%s.", key, triggerTimeMillis, trigger.getNextFireTime());
+      this.schedulerService.getScheduler().scheduleJob(trigger);
+    } catch (SchedulerException e) {
+      LOG.warn("Failed to add job reminder due to SchedulerException for job %s trigger event %s ", key, triggerTimeMillis, e);
+    }
+    LOG.info(String.format("Scheduled reminder for job %s trigger event %s. Next run: %s.", key, triggerTimeMillis, trigger.getNextFireTime()));
+  }
+
+  /**
+   * These methods should only be called from the Orchestrator or JobScheduler classes as it directly adds jobs to the
+   * Quartz scheduler
+   * @param delayPeriodSeconds
+   * @return
+   */
+  protected static String createCronFromDelayPeriod(long delayPeriodSeconds) {
+    LocalDateTime now = LocalDateTime.now();
+    LocalDateTime delaySecondsLater = now.plus(delayPeriodSeconds, ChronoUnit.SECONDS);
+    // TODO: investigate potentially better way of generating cron expression that does not make it US dependent
+    DateTimeFormatter formatter = DateTimeFormatter.ofPattern("ss mm HH dd MM ? yyyy", Locale.US);

Review Comment:
   I changed `LocaleDateTime.now(UTC)` to ensure the time zone is right but I think the formatter is only putting the timestamp into the cron schedule format, not sure if the locale there matters. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227261902


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;

Review Comment:
   I tried to use boolean value for it but if the column is NULL then the boolean returned is `false` so it becomes hard to distinguish between leaseValid and noLease. I end up having to specially define the no lease case. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227819575


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -100,8 +114,8 @@ public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
   // Insert or update row to acquire lease if values have not changed since the previous read
   // Need to define three separate statements to handle cases where row does not exist or has null values to check
   protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
-      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
-      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?); "
+      + SELECT_AFTER_INSERT_STATEMENT;

Review Comment:
   just thinking... it may be clearer not to append `SELECT_AFTER_INSERT_STATEMENT` to many constants, but rather to catenate them at point of use (so the maintainer sees it)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1223556391


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeasedToAnotherStatus.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The event in question has been leased to another. This object contains `reminderEventTimestamp` which is the event
+timestamp the lease is associated with as well as `minimumReminderWaitMillis` the minimum amount of time to wait
+before returning to check if the lease has completed or expired.
+ */
+public class LeasedToAnotherStatus extends LeaseAttemptStatus {
+  @Getter
+  private final long reminderEventTimeMillis;
+
+  @Getter
+  private final long minimumReminderWaitMillis;

Review Comment:
   It's a duration in millis, let me update the name to be more clear that its a duration and can remove the "reminder" like you're suggesting. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   a. the most consequential discernment is whether the current host acquired the lease (hence must be separated out so THE ONLY possibility for case 1)
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded the leases expiration.  each host must defer within the "linger" period, and should only attempt to acquire after that expires.  if the host should fail because another won the race to acquire it, that newly acquired lease should be nowhere near expiration.
   c. with NO_LONGER_LEASING, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even set a reminder.  in fact, under normal operating conditions, such a self-timer should regularly conclude by the host determining "there's nothing to lease", because the trigger event has been fully handled.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded its expiration.  why?  because each host must defer for the "linger" period, and *should only attempt to acquire after* that expires.  hence, if acquisition should fail because another won the race, that newly granted lease should be nowhere near expiration.
   
   c. with `NO_LONGER_LEASING`, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even to set a reminder.  under normal operating conditions, this should be the regular conclusion of most every self-timer, that is set as the follow up after failing to acquire a lease.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded its expiration.  each host must defer within the "linger" period, and should only attempt to acquire after that expires.  if the host should fail because another won the race to acquire it, that newly acquired lease should be nowhere near expiration.
   c. with NO_LONGER_LEASING, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even set a reminder.  in fact, under normal operating conditions, such a self-timer should regularly conclude by the host determining "there's nothing to lease", because the trigger event has been fully handled.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] Will-Lo commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "Will-Lo (via GitHub)" <gi...@apache.org>.
Will-Lo commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211941503


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.SchedulerLeaseDeterminationStore;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final long linger;
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected SchedulerLeaseDeterminationStore leaseDeterminationStore;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, SchedulerLeaseDeterminationStore leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService)
+      throws IOException {
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.leaseDeterminationStore = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+  }
+  private SchedulerLeaseDeterminationStore schedulerLeaseDeterminationStore;
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow's trigger event
+   * by attempting a lease for the flow event.
+   * @param jobProps
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param flowActionType
+   * @param triggerTimeMillis
+   * @return true if this host obtained the lease for this flow's trigger event, false otherwise.
+   * @throws IOException
+   */
+  public boolean handleNewTriggerEvent(Properties jobProps, String flowGroup, String flowName, String flowExecutionId,
+      SchedulerLeaseDeterminationStore.FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    SchedulerLeaseDeterminationStore.LeaseAttemptStatus leaseAttemptStatus =
+        schedulerLeaseDeterminationStore.attemptInsertAndGetPursuantTimestamp(flowGroup, flowName, flowExecutionId,
+            flowActionType, triggerTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus) {
+      case LEASE_OBTAINED:
+        return true;
+      case PREVIOUS_LEASE_EXPIRED:
+        // recursively try obtaining lease again immediately, stops when reaches one of the other cases
+        return handleNewTriggerEvent(jobProps, flowGroup, flowName, flowExecutionId, flowActionType, triggerTimeMillis);
+      case PREVIOUS_LEASE_VALID:
+        scheduleReminderForTriggerEvent(jobProps, flowGroup, flowName, flowExecutionId, flowActionType, triggerTimeMillis);
+    }
+    return false;
+  }
+
+  /**
+   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewTriggerEvent} to schedule a reminder for itself to
+   * check on the other participant's progress during pursuing orchestration after the time the lease should expire.
+   * If the previous participant was successful, then no further action is taken otherwise we re-attempt pursuing
+   * orchestration ourselves.
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param flowActionType
+   * @param triggerTimeMillis
+   */
+  protected void scheduleReminderForTriggerEvent(Properties jobProps, String flowGroup, String flowName, String flowExecutionId,
+      SchedulerLeaseDeterminationStore.FlowActionType flowActionType, long triggerTimeMillis) {
+    // Check-in `linger` time after the current timestamp which is "close-enough" to the time the pursuant attempted
+    // the flow action. We also add a small randomization to avoid 'thundering herd' issue
+    String cronExpression = createCronFromDelayPeriod(linger + random.nextInt(staggerUpperBoundSec));
+    jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
+    // This timestamp is what will be used to identify the particular flow trigger event it's associated with
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_ORIGINAL_TRIGGER_TIMESTAMP_MILLIS_KEY, String.valueOf(triggerTimeMillis));
+    JobKey key = new JobKey(flowName, flowGroup);
+    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);
+    try {
+      LOG.info("Attempting to add job reminder to Scheduler Service where job is %s trigger event %s and reminder is at "
+          + "%s.", key, triggerTimeMillis, trigger.getNextFireTime());
+      this.schedulerService.getScheduler().scheduleJob(trigger);
+    } catch (SchedulerException e) {
+      LOG.warn("Failed to add job reminder due to SchedulerException for job %s trigger event %s ", key, triggerTimeMillis, e);
+    }
+    LOG.info(String.format("Scheduled reminder for job %s trigger event %s. Next run: %s.", key, triggerTimeMillis, trigger.getNextFireTime()));
+  }
+
+  /**
+   * These methods should only be called from the Orchestrator or JobScheduler classes as it directly adds jobs to the
+   * Quartz scheduler
+   * @param delayPeriodSeconds
+   * @return
+   */
+  protected static String createCronFromDelayPeriod(long delayPeriodSeconds) {
+    LocalDateTime now = LocalDateTime.now();
+    LocalDateTime delaySecondsLater = now.plus(delayPeriodSeconds, ChronoUnit.SECONDS);
+    // TODO: investigate potentially better way of generating cron expression that does not make it US dependent
+    DateTimeFormatter formatter = DateTimeFormatter.ofPattern("ss mm HH dd MM ? yyyy", Locale.US);

Review Comment:
   Does LocalDateTime.now() default to US timezone? Otherwise this could cause issues. I would suggest you use `LocalDateTime.now(<timezone>)` to ensure consistency in this system across timezones.
   Also, I think GaaS scheduler defaults to UTC.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1212436241


##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,24 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  // TODO: multiActiveScheduler change here update values for the following keys and rename to more meaningful
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_JDBC_DRIVER_KEY = "state.store.db.jdbc.driver";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_JDBC_DRIVER = "com.mysql.cj.jdbc.Driver";

Review Comment:
   Yes, we can actually reuse the state store configs above. I will remove these keys



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222050277


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/InjectionNames.java:
##########
@@ -26,4 +26,5 @@ public final class InjectionNames {
   public static final String FORCE_LEADER = "forceLeader";
   public static final String FLOW_CATALOG_LOCAL_COMMIT = "flowCatalogLocalCommit";
   public static final String WARM_STANDBY_ENABLED = "statelessRestAPIEnabled";

Review Comment:
   shall we add a TODO to rename "WARM_STANDBY", being a misnomer?  since, happily it's not the actual value of underlying config name, it should be easier to replace.



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitorFactory.java:
##########
@@ -56,7 +64,7 @@ private DagActionStoreChangeMonitor createDagActionStoreMonitor()
     String topic = ""; // Pass empty string because we expect underlying client to dynamically determine the Kafka topic
     int numThreads = ConfigUtils.getInt(dagActionStoreChangeConfig, DAG_ACTION_STORE_CHANGE_MONITOR_NUM_THREADS_KEY, 5);
 
-    return new DagActionStoreChangeMonitor(topic, dagActionStoreChangeConfig, this.dagActionStore, this.dagManager, numThreads);
+    return new DagActionStoreChangeMonitor(topic, dagActionStoreChangeConfig, this.dagActionStore, this.dagManager, numThreads, isMultiActiveSchedulerEnabled, flowCatalog);

Review Comment:
   minor, but the ordering of the two new params is switched between the factory and the ctor of what it creates.  I suggest aligning.



##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,20 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "scheduler.lease.determination.store.db.table";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "gobblin_scheduler_lease_determination_store";
+  public static final String SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY = "reminderEventTimestampMillis";
+  public static final String SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY = "newEventTimestampMillis";
+  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS = 100;
+  // Note: linger should be on the order of seconds even though we measure in millis
+  public static final String SCHEDULER_EVENT_LINGER_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS = 30000;
+  public static final String SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY = "";
+  public static final int DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC = 5;

Review Comment:
   those above are in millis... might it be confusing to switch to secs for this one?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {

Review Comment:
   spelled differently than `MysqlDagActionStore`.  the latter capitalization seems prevalent in our code base



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeasedToAnotherStatus.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The event in question has been leased to another. This object contains `reminderEventTimestamp` which is the event
+timestamp the lease is associated with as well as `minimumReminderWaitMillis` the minimum amount of time to wait
+before returning to check if the lease has completed or expired.
+ */
+public class LeasedToAnotherStatus extends LeaseAttemptStatus {
+  @Getter
+  private final long reminderEventTimeMillis;

Review Comment:
   merely `eventTimestamp` is preferable here, since that aligns w/ the naming in `LeaseObtainedStatus`



##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,20 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "scheduler.lease.determination.store.db.table";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "gobblin_scheduler_lease_determination_store";
+  public static final String SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY = "reminderEventTimestampMillis";
+  public static final String SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY = "newEventTimestampMillis";
+  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS = 100;
+  // Note: linger should be on the order of seconds even though we measure in millis
+  public static final String SCHEDULER_EVENT_LINGER_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS = 30000;
+  public static final String SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY = "";

Review Comment:
   `UPPER_BOUND` => `MAX` ?
   
   `STAGGERING` => `DELAY` ?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java:
##########
@@ -580,7 +581,9 @@ public void run() {
 
     private void clearUpDagAction(DagId dagId) throws IOException {
       if (this.dagActionStore.isPresent()) {
-        this.dagActionStore.get().deleteDagAction(dagId.flowGroup, dagId.flowName, dagId.flowExecutionId);
+        this.dagActionStore.get().deleteDagAction(
+            new DagActionStore.DagAction(dagId.flowGroup, dagId.flowName, dagId.flowExecutionId,
+                DagActionStore.FlowActionType.KILL));

Review Comment:
   how do we know for sure that this would be a `FlowActionType.KILL`?  (may be worth a code comment)



##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,20 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";

Review Comment:
   is this supposed to be the default name for the db table?  if so, should it have '.' in it?  (I'm going by what I see for `DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE`)



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/metrics/RuntimeMetrics.java:
##########
@@ -72,6 +73,8 @@ public class RuntimeMetrics {
   public static final String GOBBLIN_JOB_SCHEDULER_TOTAL_GET_SPEC_TIME_NANOS = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".jobScheduler.totalGetSpecTimeNanos";
   public static final String GOBBLIN_JOB_SCHEDULER_TOTAL_ADD_SPEC_TIME_NANOS = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".jobScheduler.totalAddSpecTimeNanos";
   public static final String GOBBLIN_JOB_SCHEDULER_NUM_JOBS_SCHEDULED_DURING_STARTUP = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".jobScheduler.numJobsScheduledDuringStartup";
+  // Metrics Used to Track SchedulerLeaseAlgoHandlerProgress
+  public static final String GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".schedulerLeaseAlgoHandler.numLeasesCompleted";

Review Comment:
   nit: what does it mean to "complete" a lease?  for this host to be the one to get it... or for any host to get it?  for the lease to expire?  to actually accomplish something before it expired?
   
   let's seek a different word to unambiguously articulate what's measured.



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -440,12 +446,24 @@ public synchronized void scheduleJob(Properties jobProps, JobListener jobListene
   public void runJob(Properties jobProps, JobListener jobListener) throws JobException {
     try {
       Spec flowSpec = this.scheduledFlowSpecs.get(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
-      this.orchestrator.orchestrate(flowSpec);
+      String triggerTimestampMillis = extractTriggerTimestampMillis(jobProps);

Review Comment:
   just wondering... what are the ramifications of neither of the keys being defined so the trigger timestamp winds up as `0L`?



##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,20 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "scheduler.lease.determination.store.db.table";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "gobblin_scheduler_lease_determination_store";
+  public static final String SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY = "reminderEventTimestampMillis";
+  public static final String SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY = "newEventTimestampMillis";
+  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "";

Review Comment:
   empty string here (and for linger and staggering_upper_bound)?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -54,27 +53,26 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending action " + action + " for this flow. Please wait to resubmit and wait for"
+      // If an existing resume request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME,
+            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"

Review Comment:
   the signature of `handleException` is encouraging this anti-pattern of constructing an `Exception` just to tunnel through a message.  let's change the signature to take a `String message` as the final param, and push the invocation of `.getMessage()` up to the caller.



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -71,17 +88,38 @@ public String load(String key) throws Exception {
   protected DagActionStore dagActionStore;
 
   protected DagManager dagManager;
+  protected SpecCompiler specCompiler;
+  protected boolean isMultiActiveSchedulerEnabled;
+  protected FlowCatalog flowCatalog;
+  protected EventSubmitter eventSubmitter;
 
   // Note that the topic is an empty string (rather than null to avoid NPE) because this monitor relies on the consumer
   // client itself to determine all Kafka related information dynamically rather than through the config.
   public DagActionStoreChangeMonitor(String topic, Config config, DagActionStore dagActionStore, DagManager dagManager,
-      int numThreads) {
+      int numThreads, boolean isMultiActiveSchedulerEnabled, FlowCatalog flowCatalog) {
     // Differentiate group id for each host
     super(topic, config.withValue(GROUP_ID_KEY,
         ConfigValueFactory.fromAnyRef(DAG_ACTION_CHANGE_MONITOR_PREFIX + UUID.randomUUID().toString())),
         numThreads);
     this.dagActionStore = dagActionStore;
     this.dagManager = dagManager;
+    ClassAliasResolver aliasResolver = new ClassAliasResolver(SpecCompiler.class);
+    try {
+      String specCompilerClassName = ServiceConfigKeys.DEFAULT_GOBBLIN_SERVICE_FLOWCOMPILER_CLASS;
+      if (config.hasPath(ServiceConfigKeys.GOBBLIN_SERVICE_FLOWCOMPILER_CLASS_KEY)) {
+        specCompilerClassName = config.getString(ServiceConfigKeys.GOBBLIN_SERVICE_FLOWCOMPILER_CLASS_KEY);
+      }
+      log.info("Using specCompiler class name/alias " + specCompilerClassName);
+
+      this.specCompiler = (SpecCompiler) ConstructorUtils.invokeConstructor(Class.forName(aliasResolver.resolve(
+          specCompilerClassName)), config);
+    } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException
+             | ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }

Review Comment:
   if avoidable, I'd prefer to neither initialize nor maintain an additional `SpecCompiler` here, apart from the one already in the `Orchestrator`.  what would it look like if we initialized this class w/ the `Orchestrator` and then added a method to the latter handling essentially the `submitFlowToDagManager` defined just below?
   
   although you might still require a `FlowCatalog` on hand, the `EventSubmitter` could go away to also be encapsulated within the `Orchestrator`



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -177,15 +207,46 @@ protected void processMessage(DecodeableKafkaRecord message) {
     dagActionsSeenCache.put(changeIdentifier, changeIdentifier);
   }
 
+  protected void submitFlowToDagManager(String flowGroup, String flowName) {
+    // Retrieve job execution plan by recompiling the flow spec to send to the DagManager
+    FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName);
+    FlowSpec spec = null;
+    try {
+      URI flowUri = FlowSpec.Utils.createFlowSpecUri(flowId);
+      spec = (FlowSpec) flowCatalog.getSpecs(flowUri);
+      Dag<JobExecutionPlan> jobExecutionPlanDag = specCompiler.compileFlow(spec);
+      //Send the dag to the DagManager.
+      dagManager.addDag(jobExecutionPlanDag, true, true);
+    } catch (URISyntaxException e) {
+      log.warn("Could not create URI object for flowId {} due to error {}", flowId, e.getMessage());
+      this.unexpectedErrors.mark();
+      return;
+    } catch (SpecNotFoundException e) {
+      log.warn("Spec not found for flow group: {} name: {} Exception: {}", flowGroup, flowName, e);
+      this.unexpectedErrors.mark();
+      return;
+    } catch (IOException e) {
+      Map<String, String> flowMetadata = TimingEventUtils.getFlowMetadata(spec);
+      String failureMessage = "Failed to add Job Execution Plan due to: " + e.getMessage();
+      flowMetadata.put(TimingEvent.METADATA_MESSAGE, failureMessage);
+      new TimingEvent(this.eventSubmitter, TimingEvent.FlowTimings.FLOW_FAILED).stop(flowMetadata);
+      log.warn("Failed to add Job Execution Plan for flow group: {} name: {} due to error {}", flowGroup, flowName, e);

Review Comment:
   again, `flowId`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -20,28 +20,31 @@
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.Collection;
-import lombok.EqualsAndHashCode;
-import lombok.Getter;
+
+import lombok.Data;
 
 
 public interface DagActionStore {
-  enum DagActionValue {
-    KILL,
-    RESUME
+  enum FlowActionType {
+    KILL, // Kill invoked through API call
+    RESUME, // Resume flow invoked through API call
+    LAUNCH, // Launch new flow execution invoked adhoc or through scheduled trigger
+    RETRY, // Invoked through DagManager for flows configured to allow retries
+    CANCEL, // Invoked through DagManager if flow has been stuck in Orchestrated state for a while
+    ADVANCE // Launch next step in multi-hop dag
   }
 
-  @Getter
-  @EqualsAndHashCode
+  @Data
   class DagAction {
     String flowGroup;
     String flowName;
     String flowExecutionId;
-    DagActionValue dagActionValue;
-    public DagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue) {
+    FlowActionType flowActionType;
+    public DagAction(String flowGroup, String flowName, String flowExecutionId, FlowActionType flowActionType) {
       this.flowGroup = flowGroup;
       this.flowName = flowName;
       this.flowExecutionId = flowExecutionId;
-      this.dagActionValue = dagActionValue;
+      this.flowActionType = flowActionType;
     }

Review Comment:
   I actually thought `@Data` would synthesize such a ctor, but if not, then just add `@AllArgsConstructor`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease
+ *        b) another has acquired the lease -> then another will attempt to complete the lease
+ *        c) flow event no longer needs to be acted upon -> terminal state
+ *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it
+ *    needs to attempt the lease again [status (b) above].
+ *  4. Once the instance which acquired the lease completes its work on the flow event, it calls completeLeaseUse to
+ *    indicate to all other instances that the flow event no longer needs to be acted upon [status (c) above]
+ */
+public interface MultiActiveLeaseArbiter {
+  static final Logger LOG = LoggerFactory.getLogger(MultiActiveLeaseArbiter.class);

Review Comment:
   instead use `@Slf4j`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease
+ *        b) another has acquired the lease -> then another will attempt to complete the lease
+ *        c) flow event no longer needs to be acted upon -> terminal state
+ *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it
+ *    needs to attempt the lease again [status (b) above].
+ *  4. Once the instance which acquired the lease completes its work on the flow event, it calls completeLeaseUse to
+ *    indicate to all other instances that the flow event no longer needs to be acted upon [status (c) above]
+ */
+public interface MultiActiveLeaseArbiter {
+  static final Logger LOG = LoggerFactory.getLogger(MultiActiveLeaseArbiter.class);
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow action event if one does not already
+   * exist in the store for the flow action or has expired. Regardless of the outcome it also reads the lease
+   * acquisition timestamp of the entry for that flow action event (it could have pre-existed in the table or been newly
+   *  added by the previous write). Based on the transaction results, it will return @LeaseAttemptStatus to determine

Review Comment:
   `{@link LeaseAttemptStatus}`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -20,28 +20,31 @@
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.Collection;
-import lombok.EqualsAndHashCode;
-import lombok.Getter;
+
+import lombok.Data;
 
 
 public interface DagActionStore {
-  enum DagActionValue {
-    KILL,
-    RESUME
+  enum FlowActionType {

Review Comment:
   better name--nice!



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does

Review Comment:
   suggestions:
   "instances" => "participants" (since to avoid OO connotations)
   
   "over ownership of" => "to take responsibility for"



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease

Review Comment:
   "to complete the lease" => "to carry out the required action before the lease expires"
   
   for clarity's sake, I'd suggest stating the class that each of the three corresponds to, as in:
   "acquired the lease (`LeaseObtainedStatus` returned) -> ..."



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -54,27 +53,26 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending action " + action + " for this flow. Please wait to resubmit and wait for"
+      // If an existing resume request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME,
+            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
                 + " action to be completed."));
         return;
       }
-      this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME);
-    } catch (IOException | SQLException | SpecNotFoundException e) {
+      this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME);
+    } catch (IOException | SQLException e) {
       log.warn(
           String.format("Failed to add execution resume action for flow %s %s %s to dag action store due to", flowGroup,
               flowName, flowExecutionId), e);
-      this.handleException(flowGroup, flowName, flowExecutionId.toString(), e);
+      this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME, e);
     }
 
   }
 
-  private void handleException (String flowGroup, String flowName, String flowExecutionId, Exception e) {
+  private void handleException (String flowGroup, String flowName, String flowExecutionId, DagActionStore.FlowActionType flowActionType, Exception e) {

Review Comment:
   nit: extra space after method name



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon

Review Comment:
   "independently learn of"



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeaseAttemptStatus.java:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.runtime.api;
+
+public abstract class LeaseAttemptStatus {
+  protected LeaseAttemptStatus() {}

Review Comment:
   I know the default ctor will be `public`, but just go w/ that.  besides this class is `abstract`, so none may create one anyway



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeaseAttemptStatus.java:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.runtime.api;
+
+public abstract class LeaseAttemptStatus {

Review Comment:
   I agree w/ defining this as the common base of a three-way alt. (roughly modeling an algebraic data type).  for clarity's sake, however, let's keep the entire hierarchy together rather than splintering aross four files.
   
   do so by making these static inner classes of `MultiActiveLeaseArbiter`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease
+ *        b) another has acquired the lease -> then another will attempt to complete the lease
+ *        c) flow event no longer needs to be acted upon -> terminal state
+ *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it
+ *    needs to attempt the lease again [status (b) above].
+ *  4. Once the instance which acquired the lease completes its work on the flow event, it calls completeLeaseUse to
+ *    indicate to all other instances that the flow event no longer needs to be acted upon [status (c) above]
+ */
+public interface MultiActiveLeaseArbiter {
+  static final Logger LOG = LoggerFactory.getLogger(MultiActiveLeaseArbiter.class);
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow action event if one does not already
+   * exist in the store for the flow action or has expired. Regardless of the outcome it also reads the lease
+   * acquisition timestamp of the entry for that flow action event (it could have pre-existed in the table or been newly
+   *  added by the previous write). Based on the transaction results, it will return @LeaseAttemptStatus to determine
+   *  the next action.
+   * @param flowAction uniquely identifies the flow
+   * @param eventTimeMillis is the time this flow action should occur
+   * @return LeaseAttemptStatus
+   * @throws IOException
+   */
+  LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis) throws IOException;
+
+  /**
+   * This method is used to indicate the owner of the lease has successfully completed required actions while holding
+   * the lease of the flow action event. It marks the lease as "no longer leasing", if the eventTimeMillis and
+   * leaseAcquisitionTimeMillis values have not changed since this owner acquired the lease (indicating the lease did
+   * not expire).
+   * @return true if successfully updated, indicating no further actions need to be taken regarding this event.

Review Comment:
   it may be non-obvious, so let's describe the meaning of `false` as well (that the caller should continue seeking to acquire the lease, as if any actions it did successfully accomplish, do not actually count)
   
   relatedly, minor suggestion on naming: `recordLeaseSuccess` or `markLeaseSuccess`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease
+ *        b) another has acquired the lease -> then another will attempt to complete the lease
+ *        c) flow event no longer needs to be acted upon -> terminal state
+ *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it
+ *    needs to attempt the lease again [status (b) above].
+ *  4. Once the instance which acquired the lease completes its work on the flow event, it calls completeLeaseUse to
+ *    indicate to all other instances that the flow event no longer needs to be acted upon [status (c) above]
+ */
+public interface MultiActiveLeaseArbiter {
+  static final Logger LOG = LoggerFactory.getLogger(MultiActiveLeaseArbiter.class);
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow action event if one does not already
+   * exist in the store for the flow action or has expired. Regardless of the outcome it also reads the lease
+   * acquisition timestamp of the entry for that flow action event (it could have pre-existed in the table or been newly
+   *  added by the previous write). Based on the transaction results, it will return @LeaseAttemptStatus to determine
+   *  the next action.
+   * @param flowAction uniquely identifies the flow
+   * @param eventTimeMillis is the time this flow action should occur

Review Comment:
   "...the flow and the present action upon it"
   
   "...the time `flowAction` was triggered"



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease
+ *        b) another has acquired the lease -> then another will attempt to complete the lease
+ *        c) flow event no longer needs to be acted upon -> terminal state
+ *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it
+ *    needs to attempt the lease again [status (b) above].
+ *  4. Once the instance which acquired the lease completes its work on the flow event, it calls completeLeaseUse to
+ *    indicate to all other instances that the flow event no longer needs to be acted upon [status (c) above]
+ */
+public interface MultiActiveLeaseArbiter {
+  static final Logger LOG = LoggerFactory.getLogger(MultiActiveLeaseArbiter.class);
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow action event if one does not already
+   * exist in the store for the flow action or has expired. Regardless of the outcome it also reads the lease
+   * acquisition timestamp of the entry for that flow action event (it could have pre-existed in the table or been newly
+   *  added by the previous write). Based on the transaction results, it will return @LeaseAttemptStatus to determine
+   *  the next action.
+   * @param flowAction uniquely identifies the flow
+   * @param eventTimeMillis is the time this flow action should occur
+   * @return LeaseAttemptStatus
+   * @throws IOException
+   */
+  LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis) throws IOException;
+
+  /**
+   * This method is used to indicate the owner of the lease has successfully completed required actions while holding
+   * the lease of the flow action event. It marks the lease as "no longer leasing", if the eventTimeMillis and
+   * leaseAcquisitionTimeMillis values have not changed since this owner acquired the lease (indicating the lease did
+   * not expire).
+   * @return true if successfully updated, indicating no further actions need to be taken regarding this event.
+   */
+  boolean completeLeaseUse(DagActionStore.DagAction flowAction, long eventTimeMillis, long leaseAcquisitionTimeMillis)

Review Comment:
   to hammer home that this should ONLY be called by someone actually holding the lease, what about having its only param be `LeaseObtainedStatus`?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeaseObtainedStatus.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The instance calling this method acquired the lease for the event in question. The class contains the `eventTimestamp`
+associated with the lease as well as the time the lease was obtained by me or `myLeaseAcquisitionTimestamp`.
+ */
+public class LeaseObtainedStatus extends LeaseAttemptStatus {
+  @Getter
+  private final long eventTimestamp;
+
+  @Getter
+  private final long myLeaseAcquisitionTimestamp;

Review Comment:
   nit: no need to clarify it's "mine"/"yours"... just `leaseAcq...Tstamp` is enough



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeaseObtainedStatus.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The instance calling this method acquired the lease for the event in question. The class contains the `eventTimestamp`
+associated with the lease as well as the time the lease was obtained by me or `myLeaseAcquisitionTimestamp`.
+ */
+public class LeaseObtainedStatus extends LeaseAttemptStatus {
+  @Getter
+  private final long eventTimestamp;
+
+  @Getter
+  private final long myLeaseAcquisitionTimestamp;
+
+  protected LeaseObtainedStatus(long eventTimestamp, long myLeaseAcquisitionTimestamp) {
+    super();
+    this.eventTimestamp = eventTimestamp;
+    this.myLeaseAcquisitionTimestamp = myLeaseAcquisitionTimestamp;
+  }

Review Comment:
   replace w/ `@Data` (and `@RequiredArgsConstructor`, if also necessary)



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);

Review Comment:
   `@Slf4j`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeasedToAnotherStatus.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The event in question has been leased to another. This object contains `reminderEventTimestamp` which is the event
+timestamp the lease is associated with as well as `minimumReminderWaitMillis` the minimum amount of time to wait
+before returning to check if the lease has completed or expired.
+ */
+public class LeasedToAnotherStatus extends LeaseAttemptStatus {
+  @Getter
+  private final long reminderEventTimeMillis;
+
+  @Getter
+  private final long minimumReminderWaitMillis;

Review Comment:
   nit: I suggest against naming 'reminder', as that unnecessarily suggests how we expect the field to be used, and better to keep this class blissfully unaware of what goes on downstream.  instead, maybe adopt the "linger" naming, used elsewhere.  or else "lease expiration".
   
   I'm unclear whether this is meant to be an absolute timestamp or a relative duration (e.g. 5000 more millis).  the former may be preferable, since it's more resilient to unpredictable delays, like long GC pause.



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**

Review Comment:
   some suggestions below... but overall, this being complicated documentation that you wrote up very well!



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/NoLongerLeasingStatus.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.runtime.api;
+
+/*
+This status is returned when a flow event was successfully leased and an instance completed the requirements for the
+event, so no further leasing is required.
+ */
+public class NoLongerLeasingStatus extends LeaseAttemptStatus {
+
+  protected NoLongerLeasingStatus() {
+    super();
+  }

Review Comment:
   I don't see a need to be `protected` (vs. just accepting synthesized default ctor's `public`). 
   
   also, NBD, but know that an implicit `super()` would be inserted for you if you don't explicitly call one of the super class's ctors



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -200,9 +206,9 @@ public GobblinServiceJobScheduler(@Named(InjectionNames.SERVICE_NAME) String ser
   public GobblinServiceJobScheduler(String serviceName, Config config, FlowStatusGenerator flowStatusGenerator,
       Optional<HelixManager> helixManager,
       Optional<FlowCatalog> flowCatalog, Optional<TopologyCatalog> topologyCatalog, Optional<DagManager> dagManager, Optional<UserQuotaManager> quotaManager,
-      SchedulerService schedulerService,  Optional<Logger> log, boolean warmStandbyEnabled) throws Exception {
+      SchedulerService schedulerService,  Optional<Logger> log, boolean warmStandbyEnabled, boolean multiActiveSchedulerEnabled, SchedulerLeaseAlgoHandler schedulerLeaseAlgoHandler) throws Exception {
     this(serviceName, config, helixManager, flowCatalog, topologyCatalog,
-        new Orchestrator(config, flowStatusGenerator, topologyCatalog, dagManager, log), schedulerService, quotaManager, log, warmStandbyEnabled);
+        new Orchestrator(config, flowStatusGenerator, topologyCatalog, dagManager, log, multiActiveSchedulerEnabled, schedulerLeaseAlgoHandler), schedulerService, quotaManager, log, warmStandbyEnabled, multiActiveSchedulerEnabled, schedulerLeaseAlgoHandler);

Review Comment:
   seeing this repeated over and over `(boolean, ScheduledLeaseAlgoHandler)` make me wonder... would `Optional<ScheduledLeaseAlgoHandler>` be sufficient?  e.g. `Optional.absent()` stands in for when currently the `boolean` is `false`?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeasedToAnotherStatus.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The event in question has been leased to another. This object contains `reminderEventTimestamp` which is the event

Review Comment:
   strictly speaking the event wasn't leases, but rather a lease was acquired to exclusively handle the event



##########
gobblin-runtime/src/main/java/org/apache/gobblin/scheduler/JobScheduler.java:
##########
@@ -600,11 +600,17 @@ public static class GobblinJob extends BaseGobblinJob implements InterruptableJo
     @Override
     public void executeImpl(JobExecutionContext context)
         throws JobExecutionException {
-      LOG.info("Starting job " + context.getJobDetail().getKey());
-      JobDataMap dataMap = context.getJobDetail().getJobDataMap();
+      JobDetail jobDetail = context.getJobDetail();
+      LOG.info("Starting job " + jobDetail.getKey());
+      JobDataMap dataMap = jobDetail.getJobDataMap();
       JobScheduler jobScheduler = (JobScheduler) dataMap.get(JOB_SCHEDULER_KEY);
       Properties jobProps = (Properties) dataMap.get(PROPERTIES_KEY);
       JobListener jobListener = (JobListener) dataMap.get(JOB_LISTENER_KEY);
+      // Obtain trigger timestamp from trigger to pass to jobProps
+      Trigger trigger = context.getTrigger();
+      long triggerTimestampMillis = trigger.getPreviousFireTime().getTime();
+      jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY,
+          String.valueOf(triggerTimestampMillis));

Review Comment:
   maybe it's seeing "previous" (fire time) used to populate "new" (event timestamp), but I'm unclear, when reading job props--what is indicated by "newEventTimestampMillis"?  please explain and/or propose a more self-explanatory name... perhaps "triggerTimestampMillis"?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -177,15 +207,46 @@ protected void processMessage(DecodeableKafkaRecord message) {
     dagActionsSeenCache.put(changeIdentifier, changeIdentifier);
   }
 
+  protected void submitFlowToDagManager(String flowGroup, String flowName) {
+    // Retrieve job execution plan by recompiling the flow spec to send to the DagManager
+    FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName);
+    FlowSpec spec = null;
+    try {
+      URI flowUri = FlowSpec.Utils.createFlowSpecUri(flowId);
+      spec = (FlowSpec) flowCatalog.getSpecs(flowUri);
+      Dag<JobExecutionPlan> jobExecutionPlanDag = specCompiler.compileFlow(spec);
+      //Send the dag to the DagManager.
+      dagManager.addDag(jobExecutionPlanDag, true, true);
+    } catch (URISyntaxException e) {
+      log.warn("Could not create URI object for flowId {} due to error {}", flowId, e.getMessage());
+      this.unexpectedErrors.mark();
+      return;
+    } catch (SpecNotFoundException e) {
+      log.warn("Spec not found for flow group: {} name: {} Exception: {}", flowGroup, flowName, e);

Review Comment:
   just above we use `flowId`... can we do same here?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -440,12 +446,24 @@ public synchronized void scheduleJob(Properties jobProps, JobListener jobListene
   public void runJob(Properties jobProps, JobListener jobListener) throws JobException {
     try {
       Spec flowSpec = this.scheduledFlowSpecs.get(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
-      this.orchestrator.orchestrate(flowSpec);
+      String triggerTimestampMillis = extractTriggerTimestampMillis(jobProps);
+      this.orchestrator.orchestrate(flowSpec, jobProps, Long.parseLong(triggerTimestampMillis));
     } catch (Exception e) {
       throw new JobException("Failed to run Spec: " + jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), e);
     }
   }
 
+  /*
+  Helper method used to extract the trigger timestamp from Properties object. If key for `original` trigger exists, then
+  we use that because this is a reminder event and the actual event trigger is the time we wanted to be reminded of the
+  original trigger.
+   */
+  public static String extractTriggerTimestampMillis(Properties jobProps) {

Review Comment:
   does this need to be `public`?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -54,27 +53,26 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending action " + action + " for this flow. Please wait to resubmit and wait for"
+      // If an existing resume request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME,
+            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
                 + " action to be completed."));
         return;
       }
-      this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME);
-    } catch (IOException | SQLException | SpecNotFoundException e) {
+      this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME);
+    } catch (IOException | SQLException e) {
       log.warn(
           String.format("Failed to add execution resume action for flow %s %s %s to dag action store due to", flowGroup,
               flowName, flowExecutionId), e);
-      this.handleException(flowGroup, flowName, flowExecutionId.toString(), e);
+      this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME, e);
     }
 
   }
 
-  private void handleException (String flowGroup, String flowName, String flowExecutionId, Exception e) {
+  private void handleException (String flowGroup, String flowName, String flowExecutionId, DagActionStore.FlowActionType flowActionType, Exception e) {
     try {
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId)) {
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, flowActionType)) {

Review Comment:
   having a separate, subsequent, and repeated check of `exists` looks like a race condition.  instead whoever calls `handleException` should indicate whether or not it previously existed upon the first (and ideally only) call to `exists`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease
+ *        b) another has acquired the lease -> then another will attempt to complete the lease
+ *        c) flow event no longer needs to be acted upon -> terminal state
+ *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it

Review Comment:
   "if another participant acquired the lease before this one could, then the present participant must..."



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210912630


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -310,6 +316,17 @@ public void orchestrate(Spec spec) throws Exception {
         flowCompilationTimer.get().stop(flowMetadata);
       }
 
+      // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
+      if (this.isMultiActiveSchedulerEnabled) {
+        String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+        boolean leaseAttemptSucceeded = schedulerLeaseAlgoHandler.handleNewTriggerEvent(jobProps, flowGroup, flowName,
+            flowExecutionId, SchedulerLeaseDeterminationStore.FlowActionType.LAUNCH, triggerTimestampMillis);
+        _log.info("scheduler attempted lease on flowGroup: %s, flowName: %s, flowExecutionId: %s, LAUNCH event for "

Review Comment:
   This is a great suggestion. It'll also be useful to keep the prefix and formatting constant across log events for the scheduler lease obtaining so we can easily grep to understand what happened for each host. 
   `Multi-active scheduler lease attempt: [flowGroup: 'foo-grp', flowName: 'bar-fn', execId: 1234, dagAction: LAUNCH, triggerEvent: 98765] - SUCCESS/FAILURE `
   
   that way `Multi-active scheduler lease attempt: [flowGroup: 'foo-grp', flowName: 'bar-fn', execId: 1234, dagAction: LAUNCH, triggerEventTimeStamp: 98765] - ` can always be the prefix and followed by a more descriptive log message if we want to add one



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] Will-Lo commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "Will-Lo (via GitHub)" <gi...@apache.org>.
Will-Lo commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211927683


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {

Review Comment:
   Can you add a javadoc describing what this class does?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -151,7 +170,16 @@ protected void processMessage(DecodeableKafkaRecord message) {
           log.info("Received insert dag action and about to send kill flow request");
           dagManager.handleKillFlowRequest(flowGroup, flowName, Long.parseLong(flowExecutionId));
           this.killsInvoked.mark();
-        } else {
+        } else if (dagAction.equals(DagActionStore.DagActionValue.LAUNCH)) {
+          // If multi-active scheduler is NOT turned on we should not receive these type of events
+          if (!this.isMultiActiveSchedulerEnabled) {
+            log.warn("Received LAUNCH dagAction while not in multi-active scheduler mode for flow group: {}, flow name:"
+                + "{}, execution id: {}, dagAction: {}", flowGroup, flowName, flowExecutionId, dagAction);
+            this.unexpectedErrors.mark();
+          }
+          log.info("Received insert dag action and about to forward launch request to DagManager");
+          submitFlowToDagManager(flowGroup, flowName);
+        }else {

Review Comment:
   nit: missing space



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -27,7 +27,9 @@
 public interface DagActionStore {
   enum DagActionValue {
     KILL,
-    RESUME
+    RESUME,
+    // TODO: potentially combine this enum with {@link SchedulerLeaseDeterminationStore.FlowActionType}
+    LAUNCH

Review Comment:
   Naive question for the sake of my understanding here:
   So we are making the assumption that the host scheduler that wins the lease to run the job does not imply that the host actually runs the job here? So any host can run a job that is stored/scheduled by other hosts.



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";
+
+  @Inject
+  public MysqlSchedulerLeaseDeterminationStore(Config config, DagActionStore dagActionStore) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MysqlSchedulerLeaseDeterminationStore");
+    }
+
+    this.tableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.epsilon = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + tableName, e);
+    }
+    this.dagActionStore = dagActionStore;
+  }
+
+  @Override
+  public LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    Timestamp triggerTimestamp = new Timestamp(triggerTimeMillis);
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement insertStatement = connection.prepareStatement(
+            String.format(ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT, tableName, tableName, epsilon, tableName,
+                epsilon))) {
+      int i = 0;
+      // Values to set in new row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to check if existing row matches
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to make select statement to read row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      ResultSet resultSet = insertStatement.executeQuery();
+      connection.commit();
+
+      if (!resultSet.next()) {
+        resultSet.close();
+        throw new IOException(String.format("Unexpected error where no result returned while trying to obtain lease. "
+                + "This error indicates that no entry existed for trigger flow event for table %s flow group: %s, flow "
+                + "name: %s flow execution id: %s and trigger timestamp: %s when one should have been inserted",
+            tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      // If a row was inserted, then we have obtained the lease
+      int rowsUpdated = resultSet.getInt(1);
+      if (rowsUpdated == 1) {
+        // If the pursuing flow launch has been persisted to the {@link DagActionStore} we have completed lease obtainment
+        this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH);
+        if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH)) {
+          if (updatePursuantTimestamp(flowGroup, flowName, flowExecutionId, flowActionType, triggerTimestamp)) {
+            // TODO: potentially add metric here to count number of flows scheduled by each scheduler

Review Comment:
   I think this would be an excellent metric also for the purpose of loadbalancing, maybe worth implementing directly



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.SchedulerLeaseDeterminationStore;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final long linger;
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected SchedulerLeaseDeterminationStore leaseDeterminationStore;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, SchedulerLeaseDeterminationStore leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService)
+      throws IOException {
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.leaseDeterminationStore = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+  }
+  private SchedulerLeaseDeterminationStore schedulerLeaseDeterminationStore;
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow's trigger event
+   * by attempting a lease for the flow event.
+   * @param jobProps
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param flowActionType
+   * @param triggerTimeMillis
+   * @return true if this host obtained the lease for this flow's trigger event, false otherwise.
+   * @throws IOException
+   */
+  public boolean handleNewTriggerEvent(Properties jobProps, String flowGroup, String flowName, String flowExecutionId,
+      SchedulerLeaseDeterminationStore.FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    SchedulerLeaseDeterminationStore.LeaseAttemptStatus leaseAttemptStatus =
+        schedulerLeaseDeterminationStore.attemptInsertAndGetPursuantTimestamp(flowGroup, flowName, flowExecutionId,
+            flowActionType, triggerTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus) {
+      case LEASE_OBTAINED:
+        return true;
+      case PREVIOUS_LEASE_EXPIRED:
+        // recursively try obtaining lease again immediately, stops when reaches one of the other cases
+        return handleNewTriggerEvent(jobProps, flowGroup, flowName, flowExecutionId, flowActionType, triggerTimeMillis);
+      case PREVIOUS_LEASE_VALID:
+        scheduleReminderForTriggerEvent(jobProps, flowGroup, flowName, flowExecutionId, flowActionType, triggerTimeMillis);
+    }
+    return false;
+  }
+
+  /**
+   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewTriggerEvent} to schedule a reminder for itself to
+   * check on the other participant's progress during pursuing orchestration after the time the lease should expire.
+   * If the previous participant was successful, then no further action is taken otherwise we re-attempt pursuing
+   * orchestration ourselves.
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param flowActionType
+   * @param triggerTimeMillis
+   */
+  protected void scheduleReminderForTriggerEvent(Properties jobProps, String flowGroup, String flowName, String flowExecutionId,
+      SchedulerLeaseDeterminationStore.FlowActionType flowActionType, long triggerTimeMillis) {
+    // Check-in `linger` time after the current timestamp which is "close-enough" to the time the pursuant attempted
+    // the flow action. We also add a small randomization to avoid 'thundering herd' issue
+    String cronExpression = createCronFromDelayPeriod(linger + random.nextInt(staggerUpperBoundSec));
+    jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
+    // This timestamp is what will be used to identify the particular flow trigger event it's associated with
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_ORIGINAL_TRIGGER_TIMESTAMP_MILLIS_KEY, String.valueOf(triggerTimeMillis));
+    JobKey key = new JobKey(flowName, flowGroup);
+    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);
+    try {
+      LOG.info("Attempting to add job reminder to Scheduler Service where job is %s trigger event %s and reminder is at "
+          + "%s.", key, triggerTimeMillis, trigger.getNextFireTime());
+      this.schedulerService.getScheduler().scheduleJob(trigger);
+    } catch (SchedulerException e) {
+      LOG.warn("Failed to add job reminder due to SchedulerException for job %s trigger event %s ", key, triggerTimeMillis, e);
+    }
+    LOG.info(String.format("Scheduled reminder for job %s trigger event %s. Next run: %s.", key, triggerTimeMillis, trigger.getNextFireTime()));
+  }
+
+  /**
+   * These methods should only be called from the Orchestrator or JobScheduler classes as it directly adds jobs to the
+   * Quartz scheduler
+   * @param delayPeriodSeconds
+   * @return
+   */
+  protected static String createCronFromDelayPeriod(long delayPeriodSeconds) {
+    LocalDateTime now = LocalDateTime.now();
+    LocalDateTime delaySecondsLater = now.plus(delayPeriodSeconds, ChronoUnit.SECONDS);
+    // TODO: investigate potentially better way of generating cron expression that does not make it US dependent
+    DateTimeFormatter formatter = DateTimeFormatter.ofPattern("ss mm HH dd MM ? yyyy", Locale.US);

Review Comment:
   Does LocalDateTime.now() default to US timezone? Otherwise this could cause issues. I would suggest you use LocalDateTime.now(<timezone>) to ensure consistency in this system across timezones.
   Also, I think GaaS scheduler defaults to UTC.



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -151,7 +170,16 @@ protected void processMessage(DecodeableKafkaRecord message) {
           log.info("Received insert dag action and about to send kill flow request");
           dagManager.handleKillFlowRequest(flowGroup, flowName, Long.parseLong(flowExecutionId));
           this.killsInvoked.mark();
-        } else {
+        } else if (dagAction.equals(DagActionStore.DagActionValue.LAUNCH)) {
+          // If multi-active scheduler is NOT turned on we should not receive these type of events
+          if (!this.isMultiActiveSchedulerEnabled) {
+            log.warn("Received LAUNCH dagAction while not in multi-active scheduler mode for flow group: {}, flow name:"
+                + "{}, execution id: {}, dagAction: {}", flowGroup, flowName, flowExecutionId, dagAction);
+            this.unexpectedErrors.mark();
+          }
+          log.info("Received insert dag action and about to forward launch request to DagManager");
+          submitFlowToDagManager(flowGroup, flowName);

Review Comment:
   I agree with this sentiment, rather would let it fail loudly if it's not expected.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211098334


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -70,21 +73,23 @@ public DagAction(String flowGroup, String flowName, String flowExecutionId, DagA
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException
    * @return true if we successfully delete one record, return false if the record does not exist
    */
-  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException;
+  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue) throws IOException;
 
   /***
    * Retrieve action value by the flow group, flow name and flow execution id from the {@link DagActionStore}.
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException Exception in retrieving the {@link DagAction}.
    * @throws SpecNotFoundException If {@link DagAction} being retrieved is not present in store.
    */
-  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException, SpecNotFoundException,
-                                                                                           SQLException;
+  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue)

Review Comment:
   for the delete case especially, I wondered whether we'd already have a `DagAction` on hand.
   
   overall, and IMO unfortunately, we use quite little abstraction throughout gobblin service.  most emblematic is the regular use of `(String, String, String)` or `(String, String, long)` to specify a flow execution ID.  an alternative impl, by contrast might combine a `FlowExecutionId` w/ a `DagActionValue` to form a `DagAction`.  that would be not only more succinct and self-documenting, but also more typesafe.  it's from this general perspective that I prefer:
   ```
   deleteDagAction(DagAction)
   ```
   to
   ```
   deleteDagAction(String, String, String, DagActionValue)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1212509675


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -45,6 +44,11 @@ public DagAction(String flowGroup, String flowName, String flowExecutionId, DagA
       this.flowExecutionId = flowExecutionId;
       this.dagActionValue = dagActionValue;
     }

Review Comment:
   won't `@Data` synthesize a ctor for us?  also, how does the synthesized `toString()` compare to the hand-coded one below?



##########
gobblin-metrics-libs/gobblin-metrics-base/src/main/avro/DagActionStoreChangeEvent.avsc:
##########
@@ -25,7 +25,13 @@
     "compliance" : "NONE"
   }, {
     "name" : "dagAction",
-    "type" : "string",
+    "type": "enum",
+      "name": "dagActionValue",
+      "symbols": [
+        "KILL",
+        "RESUME",
+        "LAUNCH"
+      ],

Review Comment:
   usually each enum gets documentation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222653152


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+    try {
+      this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
+      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {

Review Comment:
   just wondering: why can't `addDagAction` provide a guarantee that it will have succeeded--or else it throws an exception?  i.e. why the need to immediately check `exists` to confirm?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1223557068


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeasedToAnotherStatus.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The event in question has been leased to another. This object contains `reminderEventTimestamp` which is the event
+timestamp the lease is associated with as well as `minimumReminderWaitMillis` the minimum amount of time to wait
+before returning to check if the lease has completed or expired.
+ */
+public class LeasedToAnotherStatus extends LeaseAttemptStatus {
+  @Getter
+  private final long reminderEventTimeMillis;

Review Comment:
   I wanted to distinguish that the event with a lease may not be the same event that the host tried to acquire a lease for ie: a previous trigger for the same flow action so it's the eventTimestamp associated with the lease and `minimumLingerDurationMillis`. Edit: Clarifying the aforementioned with in the comment/javadoc for the class and update the naming. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222573977


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;

Review Comment:
   what about:
   ```
   (lease_acquisition_timestamp + linger) < CURRENT_TIMESTAMP as isLeaseExpired
   ```
   then you either have:
   1 (TRUE) - expired
   0 (FALSE) - not expired
   NULL - no lease



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of

Review Comment:
   nice clear and informative javadoc!



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"

Review Comment:
   `WHERE_CLAUSE_TO_MATCH_KEY + " AND event_timestamp=? AND lease_acquisition_timestamp=?"`?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "

Review Comment:
   `CONDITIONALLY_INITIALIZE_LEASE_STATEMENT` or `CONDITIONALLY_CREATE_LEASE_STATEMENT`?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL; " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?; " + SELECT_AFTER_INSERT_STATEMENT;
+
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("

Review Comment:
   nit: place this definition first, so the schema is clear before the other statements come along



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;

Review Comment:
   is this `WHERE NOT EXISTS` necessary?  isn't it equivalent to the PK uniqueness constraint already to be enforced?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;

Review Comment:
   param name doesn't match



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":

Review Comment:
   `instanceof` would be more canonical



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:

Review Comment:
   let's code defensively by throwing an exception even in cases we believe "can't happen"



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)

Review Comment:
   AFAICT, this is the only "real" method in the class's external interface.  revisiting the class and method naming in light of that might suggest `FlowTriggerHandler`, since it's not really "handling" a "scheduler lease".  WDYT?
   
   if so this method might be `handleTriggerEvent` or `handle`, or even just `apply`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL; " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?; " + SELECT_AFTER_INSERT_STATEMENT;
+
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "lease_acquisition_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action))";
+
+  private static final String CREATE_CONSTANTS_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %s "
+      + "(epsilon INT, linger INT), PRIMARY KEY (epsilon, linger); INSERT INTO %s (epsilon, linger) VALUES (?,?)";
+
+  @Inject
+  public MySQLMultiActiveLeaseArbiter(Config config) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MySQLMultiActiveLeaseArbiter");
+    }
+
+    this.leaseArbiterTableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.constantsTableName = ConfigUtils.getString(config, ConfigurationKeys.MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE);
+    this.epsilon = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_LINGER_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS);
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(
+            CREATE_LEASE_ARBITER_TABLE_STATEMENT, leaseArbiterTableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + leaseArbiterTableName, e);
+    }
+    withPreparedStatement(String.format(CREATE_CONSTANTS_TABLE_STATEMENT, this.constantsTableName, this.constantsTableName),
+        createStatement -> {
+      int i = 0;
+      createStatement.setInt(++i, epsilon);
+      createStatement.setInt(++i, linger);
+      return createStatement.executeUpdate();}, true);
+  }
+
+  @Override
+  public LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    String flowGroup = flowAction.getFlowGroup();
+    String flowName = flowAction.getFlowName();
+    String flowExecutionId = flowAction.getFlowExecutionId();
+    Timestamp eventTimestamp = new Timestamp(eventTimeMillis);

Review Comment:
   up to you, but not sure these add anything beyond, say:
   ```
   getInfoStatement.setTimestamp(++i, flowAction.getFlowGroup());
   ```



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;

Review Comment:
   do we also need to get the `event_timestamp` in case another value, different from ours was successfully inserted?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL; " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?; " + SELECT_AFTER_INSERT_STATEMENT;
+
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "lease_acquisition_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action))";
+
+  private static final String CREATE_CONSTANTS_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %s "
+      + "(epsilon INT, linger INT), PRIMARY KEY (epsilon, linger); INSERT INTO %s (epsilon, linger) VALUES (?,?)";
+
+  @Inject
+  public MySQLMultiActiveLeaseArbiter(Config config) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MySQLMultiActiveLeaseArbiter");
+    }
+
+    this.leaseArbiterTableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.constantsTableName = ConfigUtils.getString(config, ConfigurationKeys.MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE);
+    this.epsilon = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_LINGER_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS);
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(
+            CREATE_LEASE_ARBITER_TABLE_STATEMENT, leaseArbiterTableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + leaseArbiterTableName, e);
+    }
+    withPreparedStatement(String.format(CREATE_CONSTANTS_TABLE_STATEMENT, this.constantsTableName, this.constantsTableName),
+        createStatement -> {
+      int i = 0;
+      createStatement.setInt(++i, epsilon);
+      createStatement.setInt(++i, linger);
+      return createStatement.executeUpdate();}, true);
+  }
+
+  @Override
+  public LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    String flowGroup = flowAction.getFlowGroup();
+    String flowName = flowAction.getFlowName();
+    String flowExecutionId = flowAction.getFlowExecutionId();
+    Timestamp eventTimestamp = new Timestamp(eventTimeMillis);
+
+    // Check table for an existing entry for this flow action and event time
+    ResultSet resultSet = withPreparedStatement(
+        String.format(GET_EVENT_INFO_STATEMENT, this.leaseArbiterTableName, this.constantsTableName),
+        getInfoStatement -> {
+          int i = 0;
+          getInfoStatement.setTimestamp(i, eventTimestamp);
+          getInfoStatement.setString(i, flowGroup);
+          getInfoStatement.setString(i, flowName);
+          getInfoStatement.setString(i, flowExecutionId);
+          getInfoStatement.setString(i, flowAction.getFlowActionType().toString());

Review Comment:
   `++i` (!)



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL; " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?; " + SELECT_AFTER_INSERT_STATEMENT;
+
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "lease_acquisition_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action))";
+
+  private static final String CREATE_CONSTANTS_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %s "
+      + "(epsilon INT, linger INT), PRIMARY KEY (epsilon, linger); INSERT INTO %s (epsilon, linger) VALUES (?,?)";
+
+  @Inject
+  public MySQLMultiActiveLeaseArbiter(Config config) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MySQLMultiActiveLeaseArbiter");

Review Comment:
   minor: but I suggest printing `CONFIG_PREFIX` w/ the error, so there's no confusion there



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL; " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?; " + SELECT_AFTER_INSERT_STATEMENT;
+
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "lease_acquisition_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action))";
+
+  private static final String CREATE_CONSTANTS_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %s "
+      + "(epsilon INT, linger INT), PRIMARY KEY (epsilon, linger); INSERT INTO %s (epsilon, linger) VALUES (?,?)";
+
+  @Inject
+  public MySQLMultiActiveLeaseArbiter(Config config) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MySQLMultiActiveLeaseArbiter");
+    }
+
+    this.leaseArbiterTableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.constantsTableName = ConfigUtils.getString(config, ConfigurationKeys.MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE);
+    this.epsilon = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_LINGER_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS);
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(
+            CREATE_LEASE_ARBITER_TABLE_STATEMENT, leaseArbiterTableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + leaseArbiterTableName, e);
+    }
+    withPreparedStatement(String.format(CREATE_CONSTANTS_TABLE_STATEMENT, this.constantsTableName, this.constantsTableName),
+        createStatement -> {
+      int i = 0;
+      createStatement.setInt(++i, epsilon);
+      createStatement.setInt(++i, linger);
+      return createStatement.executeUpdate();}, true);
+  }
+
+  @Override
+  public LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    String flowGroup = flowAction.getFlowGroup();
+    String flowName = flowAction.getFlowName();
+    String flowExecutionId = flowAction.getFlowExecutionId();
+    Timestamp eventTimestamp = new Timestamp(eventTimeMillis);
+
+    // Check table for an existing entry for this flow action and event time
+    ResultSet resultSet = withPreparedStatement(
+        String.format(GET_EVENT_INFO_STATEMENT, this.leaseArbiterTableName, this.constantsTableName),
+        getInfoStatement -> {
+          int i = 0;
+          getInfoStatement.setTimestamp(i, eventTimestamp);
+          getInfoStatement.setString(i, flowGroup);
+          getInfoStatement.setString(i, flowName);
+          getInfoStatement.setString(i, flowExecutionId);
+          getInfoStatement.setString(i, flowAction.getFlowActionType().toString());
+          return getInfoStatement.executeQuery();
+        }, true);
+
+    try {
+      // CASE 1: If no existing row for this flow action, then go ahead and insert
+      if (!resultSet.next()) {
+        ResultSet rs = withPreparedStatement(
+            String.format(CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT, this.leaseArbiterTableName,
+                this.leaseArbiterTableName, this.leaseArbiterTableName, this.constantsTableName),
+            insertStatement -> {
+              completeInsertPreparedStatement(insertStatement, flowAction, eventTimeMillis);
+              return insertStatement.executeQuery();
+            }, true);
+       return handleResultFromAttemptedLeaseObtainment(rs, eventTimeMillis);
+      }
+
+      // Extract values from result set
+      Timestamp dbEventTimestamp = resultSet.getTimestamp(1);
+      Timestamp dbLeaseAcquisitionTimestamp = resultSet.getTimestamp(2);
+      boolean isWithinEpsilon = resultSet.getBoolean(3);
+      int leaseValidityStatus = resultSet.getInt(4);
+      int dbLinger = resultSet.getInt(5);
+
+      // Lease is valid
+      if (leaseValidityStatus == 1) {
+        // CASE 2: Same event, lease is valid
+        if (isWithinEpsilon) {
+          // Utilize db timestamp for reminder
+          return new LeasedToAnotherStatus(dbEventTimestamp.getTime(),
+              dbLeaseAcquisitionTimestamp.getTime() + dbLinger);
+        }
+        // CASE 3: Distinct event, lease is valid
+        // Utilize db timestamp for wait time, but be reminded of own event timestamp
+        return new LeasedToAnotherStatus(eventTimeMillis,
+            dbLeaseAcquisitionTimestamp.getTime() + dbLinger);
+      }
+      // CASE 4: Lease is out of date (regardless of whether same or distinct event)
+      else if (leaseValidityStatus == 2) {
+        if (isWithinEpsilon) {
+          LOG.warn("Lease should not be out of date for the same trigger event since epsilon << linger for flowAction"
+                  + " {}, db eventTimestamp {}, db leaseAcquisitionTimestamp {}, linger {}", flowAction,
+              dbEventTimestamp, dbLeaseAcquisitionTimestamp, dbLinger);
+        }
+        // Use our event to acquire lease, check for previous db eventTimestamp and leaseAcquisitionTimestamp
+        ResultSet rs = withPreparedStatement(
+            String.format(CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT, this.leaseArbiterTableName,
+                this.leaseArbiterTableName, this.constantsTableName),
+            updateStatement -> {
+              completeUpdatePreparedStatement(updateStatement, flowAction, eventTimeMillis, true,
+                  true, dbEventTimestamp, dbLeaseAcquisitionTimestamp);
+              return updateStatement.executeQuery();
+            }, true);
+        return handleResultFromAttemptedLeaseObtainment(rs, eventTimeMillis);
+      } // No longer leasing this event
+        // CASE 5: Same event, no longer leasing event in db: terminate
+        if (isWithinEpsilon) {
+          return new NoLongerLeasingStatus();
+        }
+        // CASE 6: Distinct event, no longer leasing event in db
+        // Use our event to acquire lease, check for previous db eventTimestamp and NULL leaseAcquisitionTimestamp
+        ResultSet rs = withPreparedStatement(
+            String.format(CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT, this.leaseArbiterTableName,
+                this.leaseArbiterTableName, this.constantsTableName),
+            updateStatement -> {
+              completeUpdatePreparedStatement(updateStatement, flowAction, eventTimeMillis, true,
+                  false, dbEventTimestamp, null);
+              return updateStatement.executeQuery();
+            }, true);
+        return handleResultFromAttemptedLeaseObtainment(rs, eventTimeMillis);
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Attempt lease by insert or update following a read based on the condition the state of the table has not changed
+   * since the read. Parse the result to return the corresponding status based on successful insert/update or not.
+   * @param resultSet
+   * @param eventTimeMillis
+   * @return LeaseAttemptStatus
+   * @throws SQLException
+   * @throws IOException
+   */
+  protected LeaseAttemptStatus handleResultFromAttemptedLeaseObtainment(ResultSet resultSet, long eventTimeMillis)
+      throws SQLException, IOException {
+    if (!resultSet.next()) {
+      throw new IOException("Expected num rows and lease_acquisition_timestamp returned from query but received nothing");
+    }
+    int numRowsUpdated = resultSet.getInt(1);
+    long leaseAcquisitionTimeMillis = resultSet.getTimestamp(2).getTime();

Review Comment:
   couldn't this be `null`, in which case `NoLongerLeasingStatus`?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+    try {
+      this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
+      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {

Review Comment:
   just wondering: why can't `addDagAction` provide a guarantee that it will have succeeded (or else throws an exception).  i.e. why the need to immediately check `exists` to confirm?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {

Review Comment:
   name this `launchFlow`?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+    try {
+      this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
+      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
+        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+        this.numLeasesCompleted.mark();
+        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
+            status.getMyLeaseAcquisitionTimestamp());
+      }
+    } catch (IOException | SQLException e) {
+      throw new RuntimeException(e);
+    }
+    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
+    return false;
+  }
+
+  /**
+   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
+   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
+   * expire.
+   * @param jobProps
+   * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
+   * @param originalEventTimeMillis the event timestamp we were originally handling
+   * @param flowAction
+   */
+  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+      DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
+    // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
+    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
+    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);

Review Comment:
   sorry, I guess I'm unfamiliar: what's this `Trigger` we get from one scheduler and give to another?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1226986012


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -440,12 +446,24 @@ public synchronized void scheduleJob(Properties jobProps, JobListener jobListene
   public void runJob(Properties jobProps, JobListener jobListener) throws JobException {
     try {
       Spec flowSpec = this.scheduledFlowSpecs.get(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
-      this.orchestrator.orchestrate(flowSpec);
+      String triggerTimestampMillis = extractTriggerTimestampMillis(jobProps);

Review Comment:
   Updated the code here to check for 0L trigger timestamp and skip flow execution



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227716310


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -62,7 +62,38 @@ public interface MultiActiveLeaseArbiter {
    * leaseAcquisitionTimeMillis values have not changed since this owner acquired the lease (indicating the lease did
    * not expire).
    * @return true if successfully updated, indicating no further actions need to be taken regarding this event.
+   *         false if failed to update the lease properly, the caller should continue seeking to acquire the lease as
+   *         if any actions it did successfully accomplish, do not count
    */
-  boolean completeLeaseUse(DagActionStore.DagAction flowAction, long eventTimeMillis, long leaseAcquisitionTimeMillis)
-      throws IOException;
+  boolean recordLeaseSuccess(DagActionStore.DagAction flowAction, LeaseObtainedStatus status) throws IOException;
+
+  /*
+   Object used to encapsulate status of lease acquisition attempt and derived should contain information specific to

Review Comment:
   nits:
   a. this (and all those below) should really be class javadoc to show up in that tool
   b. it's a class (not an object)
   c. "derived [classes/types]" OR "derivations"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1207471784


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java:
##########
@@ -137,12 +139,12 @@ public boolean deleteDagAction(String flowGroup, String flowName, String flowExe
       connection.commit();
       return result != 0;
     } catch (SQLException e) {
-      throw new IOException(String.format("Failure to delete action for table %s of flow with flow group:%s, flow name:%s and flow execution id:%s",
-          tableName, flowGroup, flowName, flowExecutionId), e);
+      throw new IOException(String.format("Failure to delete action for table %s of flow with flow group:%s, flow name:%s, flow execution id:%s and dagAction: %s",
+          tableName, flowGroup, flowName, flowExecutionId, dagActionValue), e);

Review Comment:
   tip: if you used `DagAction` as a POJO, you could use its `.toString()` to the same effect--w/o duplication across each similar log message.  e.g.:
   ```
   IOE(String.format("...of flow with %s", tableName, dagAction), e);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded its expiration.  why?  because each host must defer for the "linger" period, and *should only attempt to acquire after* that expires.  hence, if acquisition should fail because another won the race, that newly granted lease should be nowhere near expiration.
   
   c. with `NO_LONGER_LEASING`, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even to set a reminder.  under normal operating conditions, this should be the regular conclusion of most every self-timer, set upon failing to acquire a lease.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210912630


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -310,6 +316,17 @@ public void orchestrate(Spec spec) throws Exception {
         flowCompilationTimer.get().stop(flowMetadata);
       }
 
+      // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
+      if (this.isMultiActiveSchedulerEnabled) {
+        String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+        boolean leaseAttemptSucceeded = schedulerLeaseAlgoHandler.handleNewTriggerEvent(jobProps, flowGroup, flowName,
+            flowExecutionId, SchedulerLeaseDeterminationStore.FlowActionType.LAUNCH, triggerTimestampMillis);
+        _log.info("scheduler attempted lease on flowGroup: %s, flowName: %s, flowExecutionId: %s, LAUNCH event for "

Review Comment:
   This is a great suggestion. It'll also be useful to keep the prefix and formatting constant across log events for the scheduler lease obtaining so we can easily grep to understand what happened for each host. 
   `Multi-active scheduler lease attempt: [flowGroup: 'foo-grp', flowName: 'bar-fn', execId: 1234, dagAction: LAUNCH] - triggerEvent: 98765 - SUCCESS/FAILURE `



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222130343


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -200,9 +206,9 @@ public GobblinServiceJobScheduler(@Named(InjectionNames.SERVICE_NAME) String ser
   public GobblinServiceJobScheduler(String serviceName, Config config, FlowStatusGenerator flowStatusGenerator,
       Optional<HelixManager> helixManager,
       Optional<FlowCatalog> flowCatalog, Optional<TopologyCatalog> topologyCatalog, Optional<DagManager> dagManager, Optional<UserQuotaManager> quotaManager,
-      SchedulerService schedulerService,  Optional<Logger> log, boolean warmStandbyEnabled) throws Exception {
+      SchedulerService schedulerService,  Optional<Logger> log, boolean warmStandbyEnabled, boolean multiActiveSchedulerEnabled, SchedulerLeaseAlgoHandler schedulerLeaseAlgoHandler) throws Exception {
     this(serviceName, config, helixManager, flowCatalog, topologyCatalog,
-        new Orchestrator(config, flowStatusGenerator, topologyCatalog, dagManager, log), schedulerService, quotaManager, log, warmStandbyEnabled);
+        new Orchestrator(config, flowStatusGenerator, topologyCatalog, dagManager, log, multiActiveSchedulerEnabled, schedulerLeaseAlgoHandler), schedulerService, quotaManager, log, warmStandbyEnabled, multiActiveSchedulerEnabled, schedulerLeaseAlgoHandler);

Review Comment:
   seeing this repeated over and over `(boolean, ScheduledLeaseAlgoHandler)` make me wonder... would `Optional<ScheduledLeaseAlgoHandler>` be sufficient?
   
   e.g. `Optional.absent()` stands in for when currently the `boolean` is `false`?  if `schedulerLeaseAlgoHandler` would be `null` in that case, `Optional` would definitely be preferable for its type safety.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1220091314


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -70,21 +73,23 @@ public DagAction(String flowGroup, String flowName, String flowExecutionId, DagA
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException
    * @return true if we successfully delete one record, return false if the record does not exist
    */
-  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException;
+  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue) throws IOException;
 
   /***
    * Retrieve action value by the flow group, flow name and flow execution id from the {@link DagActionStore}.
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException Exception in retrieving the {@link DagAction}.
    * @throws SpecNotFoundException If {@link DagAction} being retrieved is not present in store.
    */
-  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException, SpecNotFoundException,
-                                                                                           SQLException;
+  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue)

Review Comment:
   Changing to utilize `DagAction` in signature but leaving the other methods as is for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222573977


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;

Review Comment:
   what about:
   ```
   (lease_acquisition_timestamp + linger) < CURRENT_TIMESTAMP as isLeaseExpired
   ```
   then you either have:
   1 (TRUE) - expired
   0 (FALSE) - not expired
   NULL - no lease
   
   (IMO easier to follow than SQL `CASE`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222665925


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)

Review Comment:
   AFAICT, this is the only "real" method in the class's external interface.  revisiting the class and method naming in light of that might suggest `FlowTriggerHandler`.  (it's not really "handling" a "scheduler lease".)  WDYT?
   
   if so this method might be `handleTriggerEvent` or `handle`, or even just `apply`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222214881


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**

Review Comment:
   some suggestions below... but overall, you've done a great job writing up complicated documentation!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1223667545


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/metrics/RuntimeMetrics.java:
##########
@@ -72,6 +73,8 @@ public class RuntimeMetrics {
   public static final String GOBBLIN_JOB_SCHEDULER_TOTAL_GET_SPEC_TIME_NANOS = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".jobScheduler.totalGetSpecTimeNanos";
   public static final String GOBBLIN_JOB_SCHEDULER_TOTAL_ADD_SPEC_TIME_NANOS = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".jobScheduler.totalAddSpecTimeNanos";
   public static final String GOBBLIN_JOB_SCHEDULER_NUM_JOBS_SCHEDULED_DURING_STARTUP = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".jobScheduler.numJobsScheduledDuringStartup";
+  // Metrics Used to Track SchedulerLeaseAlgoHandlerProgress
+  public static final String GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX + ".schedulerLeaseAlgoHandler.numLeasesCompleted";

Review Comment:
   resolved offline to use name `NUM_FLOWS_SUBMITTED`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222657174


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+    try {
+      this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
+      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
+        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+        this.numLeasesCompleted.mark();
+        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
+            status.getMyLeaseAcquisitionTimestamp());
+      }
+    } catch (IOException | SQLException e) {
+      throw new RuntimeException(e);
+    }
+    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
+    return false;
+  }
+
+  /**
+   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
+   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
+   * expire.
+   * @param jobProps
+   * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
+   * @param originalEventTimeMillis the event timestamp we were originally handling
+   * @param flowAction
+   */
+  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+      DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
+    // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
+    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
+    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);

Review Comment:
   sorry, I guess I'm unfamiliar: what's the meaning/nature of this `Trigger` we get from one scheduler and give to another?
   
   (I'd love to avoid bringing in the dependency on `JobScheduler` if we can avoid it, and instead have this class depend only on the `SchedulerService`.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227275800


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;

Review Comment:
   we obtain those values in `GET_EVENT_INFO_STATEMENT`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210832676


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -27,7 +27,9 @@
 public interface DagActionStore {
   enum DagActionValue {
     KILL,
-    RESUME
+    RESUME,
+    // TODO: potentially combine this enum with {@link SchedulerLeaseDeterminationStore.FlowActionType}
+    LAUNCH
   }
 
   @Getter

Review Comment:
   How does the `@Data` annotation help here? That is for generating constructor/setter/getter right. I am noticing that the DagActionValues will include all the actions that FlowActionType will too so I am wondering if they should reuse the same enum.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211093441


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -27,7 +27,9 @@
 public interface DagActionStore {
   enum DagActionValue {
     KILL,
-    RESUME
+    RESUME,
+    // TODO: potentially combine this enum with {@link SchedulerLeaseDeterminationStore.FlowActionType}
+    LAUNCH
   }
 
   @Getter

Review Comment:
   I imagined simply:
   ```
   @Data
   class DagAction {
       final String flowGroup;
       final String flowName;
       final String flowExecutionId;
       final DagActionValue dagActionValue;
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210688091


##########
gobblin-metrics-libs/gobblin-metrics-base/src/main/avro/DagActionStoreChangeEvent.avsc:
##########
@@ -23,6 +23,11 @@
     "type" : "string",
     "doc" : "flow execution id for the dag action",
     "compliance" : "NONE"
+  }, {
+    "name" : "dagAction",
+    "type" : "string",

Review Comment:
   actually in the code this is an enum, but I did not think to encode the enum in the avro schema. Overall, I see benefits of including this as an enum here. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210700371


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {

Review Comment:
   Yes this can be generalized. We can rename the implementations of each depending on which module of our system is using this "multi-active lease" logic.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227286463


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {

Review Comment:
   this could be a variety of actions including launch, retry, trigger, etc... which is hard to encapsulate with launchFlow. I changed to `persistFlowAction`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227765802


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());

Review Comment:
   since this represents a gaping hole in our impl, it's actually more appropriate to scream/panic/freak out via `RuntimeException`, than it is to presume to continue (as if completely shirking all responsibility for the `flowAction`)



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());

Review Comment:
   since this represents a gaping hole in our impl, it's actually more appropriate to scream/panic/freak out via `RuntimeException`, than to presume to continue (as if completely shirking all responsibility for the `flowAction`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227790521


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -55,31 +55,27 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     try {
       // If an existing resume request is still pending then do not accept this request
       if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME)) {
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME,
-            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
-                + " action to be completed."));
+        this.handleException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait "

Review Comment:
   nit: `handleError` / `prepareError` / `formulateError` (?)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1207475031


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java:
##########
@@ -151,20 +153,21 @@ private DagAction getDagActionWithRetry(String flowGroup, String flowName, Strin
       getStatement.setString(++i, flowGroup);
       getStatement.setString(++i, flowName);
       getStatement.setString(++i, flowExecutionId);
+      getStatement.setString(++i, dagActionValue.toString());

Review Comment:
   tip: if you want to use `DagAction` as a value type/POJO, you could even add a method (on `DagAction`) to encapsulate this repetitive setup:
   ```
   void prepareStatement(PreparedStatement ps, int nextVarIndex)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210699415


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   those are the similar ideas, although I'm not sure what the second and third states exactly entail in the states you described above. For the second state it tells us the lease is given to another but not sure if another host is in the process of pursuing or completed lease. What is the third case? Is that if another host completed pursuing the lease?
   
   I was thinking having the states represent the next course of action. 
   ```
   Case 1: lease is acquired completely by itself or another host (do nothing after)
   Case 2: lease attempted by another expired (can try to get lease now)
   Case 3: previous lease attempt still ongoing (set a reminder to check back on lease)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210688091


##########
gobblin-metrics-libs/gobblin-metrics-base/src/main/avro/DagActionStoreChangeEvent.avsc:
##########
@@ -23,6 +23,11 @@
     "type" : "string",
     "doc" : "flow execution id for the dag action",
     "compliance" : "NONE"
+  }, {
+    "name" : "dagAction",
+    "type" : "string",

Review Comment:
   actually in the code this is an enum, but I did not think to encode the enum in the avro schema. Overall, I see benefits of including this as an enum here. Replacing with an enum. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211113437


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {

Review Comment:
   that's a possibility, but I'm not convinced we would need to have a class for each place this is used.  seems more like that's the function of the `DagActionValue` enum value (where the specific action marks the context within the system impl, such as trigger events or flow resume, etc.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211201572


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "

Review Comment:
   nit: there's no reason to limit this impl to being about solely trigger events.  (e.g. it could also be in addition for resume events).  hence I suggest: naming simply `event_timestamp`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1213668012


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID
+  }
+
+  // Action to take on a particular flow
+  enum FlowActionType {
+    LAUNCH,
+    RETRY,
+    CANCEL,
+    NEXT_HOP
+  }
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow's trigger event if one does not already
+   * exist in the store for the same trigger event. Regardless of the outcome it also reads the pursuant timestamp of
+   * the entry for that trigger event (it could have pre-existed in the table or been newly added by the previous
+   * write). Based on the transaction results, it will return @LeaseAttemptStatus to determine the next action.
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param triggerTimeMillis is the time this flow is supposed to be launched
+   * @return LeaseAttemptStatus
+   * @throws IOException
+   */
+  LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis) throws IOException;
+
+  /**
+   * This method is used by `attemptInsertAndGetPursuantTimestamp` above to indicate the host has successfully completed

Review Comment:
   Previously it was used only internally, but in the new abstraction this method will be used publicly by the instance who has obtained the lease then calls this method to terminate the lease so I will keep it public but rename it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210948304


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -310,6 +316,17 @@ public void orchestrate(Spec spec) throws Exception {
         flowCompilationTimer.get().stop(flowMetadata);
       }
 
+      // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
+      if (this.isMultiActiveSchedulerEnabled) {
+        String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+        boolean leaseAttemptSucceeded = schedulerLeaseAlgoHandler.handleNewTriggerEvent(jobProps, flowGroup, flowName,
+            flowExecutionId, SchedulerLeaseDeterminationStore.FlowActionType.LAUNCH, triggerTimestampMillis);
+        _log.info("scheduler attempted lease on flowGroup: %s, flowName: %s, flowExecutionId: %s, LAUNCH event for "
+            + "triggerTimestamp: %s that was " + (leaseAttemptSucceeded ? "" : "NOT") + "successful", flowGroup,
+            flowName, flowExecutionId, triggerTimestampMillis);
+        return;

Review Comment:
   That's a good point I'm changing to be 
   
   `if multi-active scheduler ...
   else if dagManager present ...
   else ...`
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211098334


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -70,21 +73,23 @@ public DagAction(String flowGroup, String flowName, String flowExecutionId, DagA
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException
    * @return true if we successfully delete one record, return false if the record does not exist
    */
-  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException;
+  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue) throws IOException;
 
   /***
    * Retrieve action value by the flow group, flow name and flow execution id from the {@link DagActionStore}.
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException Exception in retrieving the {@link DagAction}.
    * @throws SpecNotFoundException If {@link DagAction} being retrieved is not present in store.
    */
-  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException, SpecNotFoundException,
-                                                                                           SQLException;
+  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue)

Review Comment:
   for the delete case especially, I wondered whether we'd already have a `DagAction` on hand.
   
   overall, and IMO unfortunately, we use quite little abstraction throughout gobblin service.  most emblematic is the regular use of `(String, String, String)` or `(String, String, long)` to specify a flow execution ID.  an alternative impl, by contrast might combine a `FlowExecutionId` w/ a `DagActionValue` to form a `DagAction`.  that would be not only more succinct and self-documenting, but also more typesafe.  it's from this general perspective that I prefer the signature:
   ```
   deleteDagAction(DagAction)
   ```
   to
   ```
   deleteDagAction(String, String, String, DagActionValue)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded its expiration.  why?  because each host must defer for the "linger" period, and should only attempt to acquire AFTER that expires.  given that, if acquisition should fail because another won the race, that newly granted lease should be nowhere near expiration.
   
   c. with NO_LONGER_LEASING, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even set a reminder.  in fact, under normal operating conditions, such a self-timer should regularly conclude by the host determining "there's nothing to lease", because the trigger event has been fully handled.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211180809


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;

Review Comment:
   suggest naming these w/ a unit suffix, like `Millis`, etc.
   
   ... but wait!  shouldn't these be values stored in the DB and referenced from queries, rather than held in java-space, where there's no guarantee they would always be exactly the same for all hosts?
   
   it is all but impossible to debug a distributed algo, if the participants are permitted to "drift", and thereby reach inconsistent conclusions about the same inputs at a given point in time.



##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,24 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  // TODO: multiActiveScheduler change here update values for the following keys and rename to more meaningful
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_JDBC_DRIVER_KEY = "state.store.db.jdbc.driver";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_JDBC_DRIVER = "com.mysql.cj.jdbc.Driver";

Review Comment:
   this constant is already in a few other places.  could we reference one here rather than repeating?  e.g. `DEFAULT_STATE_STORE_DB_JDBC_DRIVER`



##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,24 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  // TODO: multiActiveScheduler change here update values for the following keys and rename to more meaningful
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_JDBC_DRIVER_KEY = "state.store.db.jdbc.driver";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_JDBC_DRIVER = "com.mysql.cj.jdbc.Driver";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_URL_KEY = "state.store.db.url";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_USER_KEY = "state.store.db.user";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_PASSWORD_KEY = "state.store.db.password";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "state.store.db.table";

Review Comment:
   is it intentional that these would "borrow" the same key as others, like:
   ```
   public static final String STATE_STORE_DB_URL_KEY = "state.store.db.url";
   public static final String STATE_STORE_DB_USER_KEY = "state.store.db.user";
     ```
   (just 20 or so lines above)?
   
   doing so seems very confusing



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";

Review Comment:
   why should `trigger_event_timestamp` be part of the primary key?  rather it seems an attribute of the record identified by the other four fields.



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "

Review Comment:
   nit: there's no reason to limit this impl to being about solely trigger events.  (it could be instead for resume events).  hence I suggest: naming simply `event_timestamp`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;

Review Comment:
   is there a missing close parens here?  I'm having trouble mentally parsing...



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"

Review Comment:
   as mentioned, 'pursuing' is likely to be clearer and more familiar than 'pursuant'.  `lease_timestamp` or `lease_acquisition_timestamp` are other possibilities.



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -91,21 +89,20 @@ public UpdateResponse delete(ComplexResourceKey<org.apache.gobblin.service.FlowS
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending " + action + " action for this flow. Please wait to resubmit and wait for"
+      // If an existing kill request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.KILL)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.KILL,
+            new RuntimeException("There is already a pending KILL action for this flow. Please wait to resubmit and wait for"

Review Comment:
   feels like déjà vu... I believe I just read this code above, but w/ `s/RESUME/KILL/g`.  if so, can't we DRY it up?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";

Review Comment:
   this doesn't necessarily pinpoint a row, so much as constrain what could be multiple rows.  I suggest leaving it as only the first four comparisons, which are all exact equality, and leaving off the `ABS`/`<=` relative cmp here.  the latter can be added in the few instances where the specific value of `trigger_event_timestamp` is unknown.



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";
+
+  @Inject
+  public MysqlSchedulerLeaseDeterminationStore(Config config, DagActionStore dagActionStore) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MysqlSchedulerLeaseDeterminationStore");
+    }
+
+    this.tableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.epsilon = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + tableName, e);
+    }
+    this.dagActionStore = dagActionStore;
+  }
+
+  @Override
+  public LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    Timestamp triggerTimestamp = new Timestamp(triggerTimeMillis);
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement insertStatement = connection.prepareStatement(
+            String.format(ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT, tableName, tableName, epsilon, tableName,
+                epsilon))) {
+      int i = 0;
+      // Values to set in new row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to check if existing row matches
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to make select statement to read row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);

Review Comment:
   I'm in favor of raising the level of abstraction here.  no need necessarily to follow the pattern of `completeInsertPreparedStatement`, but that is one way to do so - https://github.com/apache/gobblin/blob/51a852d506b749b9ac33568aff47105e14972a57/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_store/MysqlBaseSpecStore.java#L112



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";
+
+  @Inject
+  public MysqlSchedulerLeaseDeterminationStore(Config config, DagActionStore dagActionStore) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MysqlSchedulerLeaseDeterminationStore");
+    }
+
+    this.tableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.epsilon = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + tableName, e);
+    }
+    this.dagActionStore = dagActionStore;
+  }
+
+  @Override
+  public LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    Timestamp triggerTimestamp = new Timestamp(triggerTimeMillis);
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement insertStatement = connection.prepareStatement(
+            String.format(ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT, tableName, tableName, epsilon, tableName,
+                epsilon))) {
+      int i = 0;
+      // Values to set in new row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to check if existing row matches
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to make select statement to read row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      ResultSet resultSet = insertStatement.executeQuery();
+      connection.commit();
+
+      if (!resultSet.next()) {
+        resultSet.close();
+        throw new IOException(String.format("Unexpected error where no result returned while trying to obtain lease. "
+                + "This error indicates that no entry existed for trigger flow event for table %s flow group: %s, flow "
+                + "name: %s flow execution id: %s and trigger timestamp: %s when one should have been inserted",
+            tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      // If a row was inserted, then we have obtained the lease
+      int rowsUpdated = resultSet.getInt(1);
+      if (rowsUpdated == 1) {
+        // If the pursuing flow launch has been persisted to the {@link DagActionStore} we have completed lease obtainment
+        this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH);
+        if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH)) {
+          if (updatePursuantTimestamp(flowGroup, flowName, flowExecutionId, flowActionType, triggerTimestamp)) {
+            // TODO: potentially add metric here to count number of flows scheduled by each scheduler
+            LOG.info("Host completed obtaining lease for flow group: %s, flow name: %s flow execution id: %s and "
+                + "trigger timestamp: %s", flowGroup, flowName, flowExecutionId, triggerTimestamp);
+            resultSet.close();
+            return LeaseAttemptStatus.LEASE_OBTAINED;
+          } else {
+            LOG.warn("Unable to update pursuant timestamp after persisting flow launch to DagActionStore for flow "
+                + "group: %s, flow name: %s flow execution id: %s and trigger timestamp: %s.", flowGroup, flowName,
+                flowExecutionId, triggerTimestamp);
+          }
+        } else {
+          LOG.warn("Did not find flow launch action in DagActionStore after adding it for flow group: %s, flow name: "
+                  + "%s flow execution id: %s and trigger timestamp: %s.", flowGroup, flowName, flowExecutionId,
+              triggerTimestamp);
+        }
+      } else if (rowsUpdated > 1) {
+        resultSet.close();
+        throw new IOException(String.format("Expect at most 1 row in table for a given trigger event. %s rows "
+            + "exist for the trigger flow event for table %s flow group: %s, flow name: %s flow execution id: %s "
+            + "and trigger timestamp: %s.", i, tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      Timestamp pursuantTimestamp = resultSet.getTimestamp(2);
+      resultSet.close();
+      long currentTimeMillis = System.currentTimeMillis();
+      // Another host has obtained lease and no further steps required
+      if (pursuantTimestamp == null) {
+        LOG.info("Another host has already successfully obtained lease for flow group: %s, flow name: %s flow execution "
+            + "id: %s and trigger timestamp: %s", flowGroup, flowName, flowExecutionId, triggerTimeMillis);
+        return LeaseAttemptStatus.LEASE_OBTAINED;
+      } else if (pursuantTimestamp.getTime() + linger <= currentTimeMillis) {
+        return LeaseAttemptStatus.PREVIOUS_LEASE_EXPIRED;
+      }
+      // Previous lease owner still has valid lease (pursuant + linger > current timestamp)
+        return LeaseAttemptStatus.PREVIOUS_LEASE_VALID;
+    } catch (SQLException e) {
+      throw new IOException(String.format("Error encountered while trying to obtain lease on trigger flow event for "
+              + "table %s flow group: %s, flow name: %s flow execution id: %s and trigger timestamp: %s", tableName,
+          flowGroup, flowName, flowExecutionId, triggerTimestamp), e);
+    }
+  }
+
+  @Override
+  public boolean updatePursuantTimestamp(String flowGroup, String flowName, String flowExecutionId,
+      FlowActionType flowActionType, Timestamp triggerTimestamp)
+      throws IOException {
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement updateStatement = connection.prepareStatement(
+            String.format(UPDATE_PURSUANT_TIMESTAMP_STATEMENT, tableName, epsilon))) {
+        int i = 0;
+        updateStatement.setString(++i, flowGroup);
+        updateStatement.setString(++i, flowName);
+        updateStatement.setString(++i, flowExecutionId);
+        updateStatement.setString(++i, flowActionType.toString());
+        updateStatement.setTimestamp(++i, triggerTimestamp);
+        i = updateStatement.executeUpdate();
+        connection.commit();
+
+        if (i != 1) {
+          LOG.warn("Expected to update 1 row's pursuant timestamp for a flow trigger event but instead updated {}", i);
+        }
+        return i >= 1;
+    } catch (SQLException e) {
+      throw new IOException(String.format("Encountered exception while trying to update pursuant timestamp to null for "
+              + "flowGroup: %s flowName: %s flowExecutionId: %s flowAction: %s triggerTimestamp: %s. Exception is %s",
+          flowGroup, flowName, flowExecutionId, flowActionType, triggerTimestamp, e));

Review Comment:
   to preserve the stacktrace of the orig exception, we'd want the two-arg version of `IOException`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";
+
+  @Inject
+  public MysqlSchedulerLeaseDeterminationStore(Config config, DagActionStore dagActionStore) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MysqlSchedulerLeaseDeterminationStore");
+    }
+
+    this.tableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.epsilon = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + tableName, e);
+    }
+    this.dagActionStore = dagActionStore;
+  }
+
+  @Override
+  public LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    Timestamp triggerTimestamp = new Timestamp(triggerTimeMillis);
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement insertStatement = connection.prepareStatement(
+            String.format(ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT, tableName, tableName, epsilon, tableName,
+                epsilon))) {
+      int i = 0;
+      // Values to set in new row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to check if existing row matches
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to make select statement to read row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      ResultSet resultSet = insertStatement.executeQuery();
+      connection.commit();
+
+      if (!resultSet.next()) {
+        resultSet.close();
+        throw new IOException(String.format("Unexpected error where no result returned while trying to obtain lease. "
+                + "This error indicates that no entry existed for trigger flow event for table %s flow group: %s, flow "
+                + "name: %s flow execution id: %s and trigger timestamp: %s when one should have been inserted",
+            tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      // If a row was inserted, then we have obtained the lease
+      int rowsUpdated = resultSet.getInt(1);
+      if (rowsUpdated == 1) {
+        // If the pursuing flow launch has been persisted to the {@link DagActionStore} we have completed lease obtainment
+        this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH);
+        if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH)) {

Review Comment:
   not certain I haven't gotten confused, but...
   
   is the code here jumping in to carry out its task immediately upon obtaining the lease?  if so, I recommend instead to separate concerns and have the present class solely determine lease status.  leave it instead to the caller to separately act on the situation where the lease was successfully acquired.  not only would code be more reusable, but also easier to test in isolation (i.e. w/o mocking `DagActionStore`).



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";
+
+  @Inject
+  public MysqlSchedulerLeaseDeterminationStore(Config config, DagActionStore dagActionStore) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MysqlSchedulerLeaseDeterminationStore");
+    }
+
+    this.tableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.epsilon = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + tableName, e);
+    }
+    this.dagActionStore = dagActionStore;
+  }
+
+  @Override
+  public LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    Timestamp triggerTimestamp = new Timestamp(triggerTimeMillis);
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement insertStatement = connection.prepareStatement(
+            String.format(ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT, tableName, tableName, epsilon, tableName,
+                epsilon))) {
+      int i = 0;
+      // Values to set in new row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to check if existing row matches
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to make select statement to read row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      ResultSet resultSet = insertStatement.executeQuery();
+      connection.commit();
+
+      if (!resultSet.next()) {
+        resultSet.close();
+        throw new IOException(String.format("Unexpected error where no result returned while trying to obtain lease. "
+                + "This error indicates that no entry existed for trigger flow event for table %s flow group: %s, flow "
+                + "name: %s flow execution id: %s and trigger timestamp: %s when one should have been inserted",
+            tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      // If a row was inserted, then we have obtained the lease
+      int rowsUpdated = resultSet.getInt(1);
+      if (rowsUpdated == 1) {
+        // If the pursuing flow launch has been persisted to the {@link DagActionStore} we have completed lease obtainment
+        this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH);
+        if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH)) {
+          if (updatePursuantTimestamp(flowGroup, flowName, flowExecutionId, flowActionType, triggerTimestamp)) {
+            // TODO: potentially add metric here to count number of flows scheduled by each scheduler
+            LOG.info("Host completed obtaining lease for flow group: %s, flow name: %s flow execution id: %s and "
+                + "trigger timestamp: %s", flowGroup, flowName, flowExecutionId, triggerTimestamp);
+            resultSet.close();
+            return LeaseAttemptStatus.LEASE_OBTAINED;
+          } else {
+            LOG.warn("Unable to update pursuant timestamp after persisting flow launch to DagActionStore for flow "
+                + "group: %s, flow name: %s flow execution id: %s and trigger timestamp: %s.", flowGroup, flowName,
+                flowExecutionId, triggerTimestamp);
+          }
+        } else {
+          LOG.warn("Did not find flow launch action in DagActionStore after adding it for flow group: %s, flow name: "
+                  + "%s flow execution id: %s and trigger timestamp: %s.", flowGroup, flowName, flowExecutionId,
+              triggerTimestamp);
+        }
+      } else if (rowsUpdated > 1) {
+        resultSet.close();
+        throw new IOException(String.format("Expect at most 1 row in table for a given trigger event. %s rows "
+            + "exist for the trigger flow event for table %s flow group: %s, flow name: %s flow execution id: %s "
+            + "and trigger timestamp: %s.", i, tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      Timestamp pursuantTimestamp = resultSet.getTimestamp(2);
+      resultSet.close();
+      long currentTimeMillis = System.currentTimeMillis();
+      // Another host has obtained lease and no further steps required
+      if (pursuantTimestamp == null) {
+        LOG.info("Another host has already successfully obtained lease for flow group: %s, flow name: %s flow execution "
+            + "id: %s and trigger timestamp: %s", flowGroup, flowName, flowExecutionId, triggerTimeMillis);
+        return LeaseAttemptStatus.LEASE_OBTAINED;
+      } else if (pursuantTimestamp.getTime() + linger <= currentTimeMillis) {
+        return LeaseAttemptStatus.PREVIOUS_LEASE_EXPIRED;
+      }
+      // Previous lease owner still has valid lease (pursuant + linger > current timestamp)
+        return LeaseAttemptStatus.PREVIOUS_LEASE_VALID;
+    } catch (SQLException e) {
+      throw new IOException(String.format("Error encountered while trying to obtain lease on trigger flow event for "
+              + "table %s flow group: %s, flow name: %s flow execution id: %s and trigger timestamp: %s", tableName,
+          flowGroup, flowName, flowExecutionId, triggerTimestamp), e);
+    }
+  }
+
+  @Override
+  public boolean updatePursuantTimestamp(String flowGroup, String flowName, String flowExecutionId,
+      FlowActionType flowActionType, Timestamp triggerTimestamp)
+      throws IOException {
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement updateStatement = connection.prepareStatement(
+            String.format(UPDATE_PURSUANT_TIMESTAMP_STATEMENT, tableName, epsilon))) {
+        int i = 0;
+        updateStatement.setString(++i, flowGroup);
+        updateStatement.setString(++i, flowName);
+        updateStatement.setString(++i, flowExecutionId);
+        updateStatement.setString(++i, flowActionType.toString());
+        updateStatement.setTimestamp(++i, triggerTimestamp);
+        i = updateStatement.executeUpdate();

Review Comment:
   quite confusing to repurpose the var used for the index during statement preparation to later become the count of rows modified.  why not just create a separate `final` var dedicated to the latter purpose?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -54,27 +53,26 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending action " + action + " for this flow. Please wait to resubmit and wait for"
+      // If an existing resume request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME,
+            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
                 + " action to be completed."));
         return;
       }
       this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME);
-    } catch (IOException | SQLException | SpecNotFoundException e) {
+    } catch (IOException | SQLException e) {
       log.warn(
           String.format("Failed to add execution resume action for flow %s %s %s to dag action store due to", flowGroup,
               flowName, flowExecutionId), e);
-      this.handleException(flowGroup, flowName, flowExecutionId.toString(), e);
+      this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME, e);
     }
 
   }
 
-  private void handleException (String flowGroup, String flowName, String flowExecutionId, Exception e) {
+  private void handleException (String flowGroup, String flowName, String flowExecutionId, DagActionStore.DagActionValue dagActionValue, Exception e) {

Review Comment:
   I'm not clear whether I've missed something, but is `Exception e` used merely to `e.getMessage()`?  if so, why are we passing in an exception, which generates a stacktrace, etc., rather than merely passing in a `String errMessage`?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -119,28 +158,8 @@ protected void processMessage(DecodeableKafkaRecord message) {
       return;
     }
 
-    // Retrieve the Dag Action taken from MySQL table unless operation is DELETE
-    DagActionStore.DagActionValue dagAction = null;
-    if (!operation.equals("DELETE")) {
-      try {
-        dagAction = dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId).getDagActionValue();
-      } catch (IOException e) {
-        log.error("Encountered IOException trying to retrieve dagAction for flow group: {} name: {} executionId: {}. " + "Exception: {}", flowGroup, flowName, flowExecutionId, e);
-        this.unexpectedErrors.mark();
-        return;
-      } catch (SpecNotFoundException e) {
-        log.error("DagAction not found for flow group: {} name: {} executionId: {} Exception: {}", flowGroup, flowName,
-            flowExecutionId, e);
-        this.unexpectedErrors.mark();
-        return;
-      } catch (SQLException throwables) {
-        log.error("Encountered SQLException trying to retrieve dagAction for flow group: {} name: {} executionId: {}. " + "Exception: {}", flowGroup, flowName, flowExecutionId, throwables);
-        return;
-      }
-    }
-
-    // We only expert INSERT and DELETE operations done to this table. INSERTs correspond to resume or delete flow
-    // requests that have to be processed. DELETEs require no action.
+    // We only expect INSERT and DELETE operations done to this table. INSERTs correspond to any type of
+    // {@link DagActionStore.DagACtionValue} flow requests that have to be processed. DELETEs require no action.

Review Comment:
   DagA(c)tionValue



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -151,7 +170,16 @@ protected void processMessage(DecodeableKafkaRecord message) {
           log.info("Received insert dag action and about to send kill flow request");
           dagManager.handleKillFlowRequest(flowGroup, flowName, Long.parseLong(flowExecutionId));
           this.killsInvoked.mark();
-        } else {
+        } else if (dagAction.equals(DagActionStore.DagActionValue.LAUNCH)) {
+          // If multi-active scheduler is NOT turned on we should not receive these type of events
+          if (!this.isMultiActiveSchedulerEnabled) {
+            log.warn("Received LAUNCH dagAction while not in multi-active scheduler mode for flow group: {}, flow name:"
+                + "{}, execution id: {}, dagAction: {}", flowGroup, flowName, flowExecutionId, dagAction);
+            this.unexpectedErrors.mark();
+          }
+          log.info("Received insert dag action and about to forward launch request to DagManager");
+          submitFlowToDagManager(flowGroup, flowName);
+        }else {
           log.warn("Received unsupported dagAction {}. Expected to be a KILL or RESUME", dagAction);

Review Comment:
   ...or `LAUNCH`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";
+
+  @Inject
+  public MysqlSchedulerLeaseDeterminationStore(Config config, DagActionStore dagActionStore) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MysqlSchedulerLeaseDeterminationStore");
+    }
+
+    this.tableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.epsilon = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + tableName, e);
+    }
+    this.dagActionStore = dagActionStore;
+  }
+
+  @Override
+  public LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    Timestamp triggerTimestamp = new Timestamp(triggerTimeMillis);
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement insertStatement = connection.prepareStatement(
+            String.format(ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT, tableName, tableName, epsilon, tableName,
+                epsilon))) {
+      int i = 0;
+      // Values to set in new row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to check if existing row matches
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to make select statement to read row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      ResultSet resultSet = insertStatement.executeQuery();
+      connection.commit();
+
+      if (!resultSet.next()) {
+        resultSet.close();
+        throw new IOException(String.format("Unexpected error where no result returned while trying to obtain lease. "
+                + "This error indicates that no entry existed for trigger flow event for table %s flow group: %s, flow "
+                + "name: %s flow execution id: %s and trigger timestamp: %s when one should have been inserted",
+            tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      // If a row was inserted, then we have obtained the lease
+      int rowsUpdated = resultSet.getInt(1);
+      if (rowsUpdated == 1) {
+        // If the pursuing flow launch has been persisted to the {@link DagActionStore} we have completed lease obtainment
+        this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH);
+        if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH)) {
+          if (updatePursuantTimestamp(flowGroup, flowName, flowExecutionId, flowActionType, triggerTimestamp)) {
+            // TODO: potentially add metric here to count number of flows scheduled by each scheduler
+            LOG.info("Host completed obtaining lease for flow group: %s, flow name: %s flow execution id: %s and "
+                + "trigger timestamp: %s", flowGroup, flowName, flowExecutionId, triggerTimestamp);
+            resultSet.close();
+            return LeaseAttemptStatus.LEASE_OBTAINED;
+          } else {
+            LOG.warn("Unable to update pursuant timestamp after persisting flow launch to DagActionStore for flow "
+                + "group: %s, flow name: %s flow execution id: %s and trigger timestamp: %s.", flowGroup, flowName,
+                flowExecutionId, triggerTimestamp);
+          }
+        } else {
+          LOG.warn("Did not find flow launch action in DagActionStore after adding it for flow group: %s, flow name: "
+                  + "%s flow execution id: %s and trigger timestamp: %s.", flowGroup, flowName, flowExecutionId,
+              triggerTimestamp);
+        }
+      } else if (rowsUpdated > 1) {
+        resultSet.close();
+        throw new IOException(String.format("Expect at most 1 row in table for a given trigger event. %s rows "
+            + "exist for the trigger flow event for table %s flow group: %s, flow name: %s flow execution id: %s "
+            + "and trigger timestamp: %s.", i, tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      Timestamp pursuantTimestamp = resultSet.getTimestamp(2);
+      resultSet.close();
+      long currentTimeMillis = System.currentTimeMillis();
+      // Another host has obtained lease and no further steps required
+      if (pursuantTimestamp == null) {
+        LOG.info("Another host has already successfully obtained lease for flow group: %s, flow name: %s flow execution "
+            + "id: %s and trigger timestamp: %s", flowGroup, flowName, flowExecutionId, triggerTimeMillis);
+        return LeaseAttemptStatus.LEASE_OBTAINED;
+      } else if (pursuantTimestamp.getTime() + linger <= currentTimeMillis) {
+        return LeaseAttemptStatus.PREVIOUS_LEASE_EXPIRED;
+      }
+      // Previous lease owner still has valid lease (pursuant + linger > current timestamp)
+        return LeaseAttemptStatus.PREVIOUS_LEASE_VALID;
+    } catch (SQLException e) {
+      throw new IOException(String.format("Error encountered while trying to obtain lease on trigger flow event for "
+              + "table %s flow group: %s, flow name: %s flow execution id: %s and trigger timestamp: %s", tableName,
+          flowGroup, flowName, flowExecutionId, triggerTimestamp), e);
+    }
+  }
+
+  @Override
+  public boolean updatePursuantTimestamp(String flowGroup, String flowName, String flowExecutionId,
+      FlowActionType flowActionType, Timestamp triggerTimestamp)
+      throws IOException {
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement updateStatement = connection.prepareStatement(

Review Comment:
   perhaps consider reining in boiler plate w/ something like `getPreparedStatement` - https://github.com/apache/gobblin/blob/51a852d506b749b9ac33568aff47105e14972a57/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_store/MysqlBaseSpecStore.java#L356
   
   



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.SchedulerLeaseDeterminationStore;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final long linger;
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected SchedulerLeaseDeterminationStore leaseDeterminationStore;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, SchedulerLeaseDeterminationStore leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService)
+      throws IOException {
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.leaseDeterminationStore = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+  }
+  private SchedulerLeaseDeterminationStore schedulerLeaseDeterminationStore;
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow's trigger event
+   * by attempting a lease for the flow event.
+   * @param jobProps
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param flowActionType
+   * @param triggerTimeMillis
+   * @return true if this host obtained the lease for this flow's trigger event, false otherwise.
+   * @throws IOException
+   */
+  public boolean handleNewTriggerEvent(Properties jobProps, String flowGroup, String flowName, String flowExecutionId,
+      SchedulerLeaseDeterminationStore.FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    SchedulerLeaseDeterminationStore.LeaseAttemptStatus leaseAttemptStatus =
+        schedulerLeaseDeterminationStore.attemptInsertAndGetPursuantTimestamp(flowGroup, flowName, flowExecutionId,
+            flowActionType, triggerTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus) {
+      case LEASE_OBTAINED:
+        return true;
+      case PREVIOUS_LEASE_EXPIRED:
+        // recursively try obtaining lease again immediately, stops when reaches one of the other cases
+        return handleNewTriggerEvent(jobProps, flowGroup, flowName, flowExecutionId, flowActionType, triggerTimeMillis);

Review Comment:
   see comment elsewhere... this should not happen



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.IOException;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.SchedulerLeaseDeterminationStore;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class SchedulerLeaseAlgoHandler {

Review Comment:
   all classes deserve javadoc



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -151,7 +170,16 @@ protected void processMessage(DecodeableKafkaRecord message) {
           log.info("Received insert dag action and about to send kill flow request");
           dagManager.handleKillFlowRequest(flowGroup, flowName, Long.parseLong(flowExecutionId));
           this.killsInvoked.mark();
-        } else {
+        } else if (dagAction.equals(DagActionStore.DagActionValue.LAUNCH)) {
+          // If multi-active scheduler is NOT turned on we should not receive these type of events
+          if (!this.isMultiActiveSchedulerEnabled) {
+            log.warn("Received LAUNCH dagAction while not in multi-active scheduler mode for flow group: {}, flow name:"
+                + "{}, execution id: {}, dagAction: {}", flowGroup, flowName, flowExecutionId, dagAction);
+            this.unexpectedErrors.mark();
+          }
+          log.info("Received insert dag action and about to forward launch request to DagManager");
+          submitFlowToDagManager(flowGroup, flowName);

Review Comment:
   so even if not enabled for such behavior, we still proceed to submit this flow?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1228770953


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -317,26 +312,27 @@ public void orchestrate(Spec spec, Properties jobProps, long triggerTimestampMil
       }
 
       // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
-      if (this.isMultiActiveSchedulerEnabled) {
+      if (schedulerLeaseAlgoHandler.isPresent()) {
+        // If triggerTimestampMillis is 0, then it was not set by the job trigger handler, and we cannot handle this event
+        if (triggerTimestampMillis == 0L) {
+          _log.warn("Skipping execution of spec: {} because missing trigger timestamp in job properties",
+              jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
+          flowMetadata.put(TimingEvent.METADATA_MESSAGE, "Flow orchestration skipped because no trigger timestamp "
+              + "associated with flow action.");
+          if (this.eventSubmitter.isPresent()) {
+            new TimingEvent(this.eventSubmitter.get(), TimingEvent.FlowTimings.FLOW_FAILED).stop(flowMetadata);

Review Comment:
   This format is used in many other places in the `Orchestrator` so I will leave as is



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1213679270


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID
+  }
+
+  // Action to take on a particular flow
+  enum FlowActionType {
+    LAUNCH,
+    RETRY,
+    CANCEL,
+    NEXT_HOP

Review Comment:
   In terms of action RETRY and RESUME work similarly, but we use them to describe different starting points. RETRY is invoked by [DagManager](https://jarvis.corp.linkedin.com/codesearch/result/?name=DagManager.java&path=gobblin-elr%2Fgobblin-service%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fgobblin%2Fservice%2Fmodules%2Forchestration&reponame=linkedin%2Fgobblin-elr#808) automatically if a flow fails and is configured to allow retries. RESUME is manually invoked by the user. It may be worth to have the differentiation noted for logging purposes but treat these cases the same when it comes to acting on them. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1223557068


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeasedToAnotherStatus.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The event in question has been leased to another. This object contains `reminderEventTimestamp` which is the event
+timestamp the lease is associated with as well as `minimumReminderWaitMillis` the minimum amount of time to wait
+before returning to check if the lease has completed or expired.
+ */
+public class LeasedToAnotherStatus extends LeaseAttemptStatus {
+  @Getter
+  private final long reminderEventTimeMillis;

Review Comment:
   I want to distinguish that the event with a lease may not be the same event that the host tried to acquire a lease for ie: a previous trigger for the same flow action so it's the eventTimestamp associated with the lease and `minimumLingerDurationMillis`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222081871


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/LeasedToAnotherStatus.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.runtime.api;
+
+import lombok.Getter;
+
+/*
+The event in question has been leased to another. This object contains `reminderEventTimestamp` which is the event

Review Comment:
   strictly speaking the event wasn't leased, but rather a lease was acquired to exclusively handle the event



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227228937


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
+ * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
+ * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
+ * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ *
+ * At a high level the lease arbiter works as follows:
+ *  1. Multiple instances receive knowledge of a flow action event to act upon
+ *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
+ *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
+ *        a) acquired the lease -> then this instance will attempt to complete the lease
+ *        b) another has acquired the lease -> then another will attempt to complete the lease
+ *        c) flow event no longer needs to be acted upon -> terminal state
+ *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it
+ *    needs to attempt the lease again [status (b) above].
+ *  4. Once the instance which acquired the lease completes its work on the flow event, it calls completeLeaseUse to
+ *    indicate to all other instances that the flow event no longer needs to be acted upon [status (c) above]
+ */
+public interface MultiActiveLeaseArbiter {
+  static final Logger LOG = LoggerFactory.getLogger(MultiActiveLeaseArbiter.class);
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow action event if one does not already
+   * exist in the store for the flow action or has expired. Regardless of the outcome it also reads the lease
+   * acquisition timestamp of the entry for that flow action event (it could have pre-existed in the table or been newly
+   *  added by the previous write). Based on the transaction results, it will return @LeaseAttemptStatus to determine
+   *  the next action.
+   * @param flowAction uniquely identifies the flow
+   * @param eventTimeMillis is the time this flow action should occur
+   * @return LeaseAttemptStatus
+   * @throws IOException
+   */
+  LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis) throws IOException;
+
+  /**
+   * This method is used to indicate the owner of the lease has successfully completed required actions while holding
+   * the lease of the flow action event. It marks the lease as "no longer leasing", if the eventTimeMillis and
+   * leaseAcquisitionTimeMillis values have not changed since this owner acquired the lease (indicating the lease did
+   * not expire).
+   * @return true if successfully updated, indicating no further actions need to be taken regarding this event.
+   */
+  boolean completeLeaseUse(DagActionStore.DagAction flowAction, long eventTimeMillis, long leaseAcquisitionTimeMillis)

Review Comment:
   I replaced the last two arguments with `LeaseObtainedStatus` but still kept the `DagActionStore.DagAction` so we have the ability to identify the flow for updating the record



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227277996


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL; " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?; " + SELECT_AFTER_INSERT_STATEMENT;
+
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "lease_acquisition_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action))";
+
+  private static final String CREATE_CONSTANTS_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %s "
+      + "(epsilon INT, linger INT), PRIMARY KEY (epsilon, linger); INSERT INTO %s (epsilon, linger) VALUES (?,?)";
+
+  @Inject
+  public MySQLMultiActiveLeaseArbiter(Config config) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MySQLMultiActiveLeaseArbiter");
+    }
+
+    this.leaseArbiterTableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.constantsTableName = ConfigUtils.getString(config, ConfigurationKeys.MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE);
+    this.epsilon = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_LINGER_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS);
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(
+            CREATE_LEASE_ARBITER_TABLE_STATEMENT, leaseArbiterTableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + leaseArbiterTableName, e);
+    }
+    withPreparedStatement(String.format(CREATE_CONSTANTS_TABLE_STATEMENT, this.constantsTableName, this.constantsTableName),
+        createStatement -> {
+      int i = 0;
+      createStatement.setInt(++i, epsilon);
+      createStatement.setInt(++i, linger);
+      return createStatement.executeUpdate();}, true);
+  }
+
+  @Override
+  public LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    String flowGroup = flowAction.getFlowGroup();
+    String flowName = flowAction.getFlowName();
+    String flowExecutionId = flowAction.getFlowExecutionId();
+    Timestamp eventTimestamp = new Timestamp(eventTimeMillis);
+
+    // Check table for an existing entry for this flow action and event time
+    ResultSet resultSet = withPreparedStatement(
+        String.format(GET_EVENT_INFO_STATEMENT, this.leaseArbiterTableName, this.constantsTableName),
+        getInfoStatement -> {
+          int i = 0;
+          getInfoStatement.setTimestamp(i, eventTimestamp);
+          getInfoStatement.setString(i, flowGroup);
+          getInfoStatement.setString(i, flowName);
+          getInfoStatement.setString(i, flowExecutionId);
+          getInfoStatement.setString(i, flowAction.getFlowActionType().toString());
+          return getInfoStatement.executeQuery();
+        }, true);
+
+    try {
+      // CASE 1: If no existing row for this flow action, then go ahead and insert
+      if (!resultSet.next()) {
+        ResultSet rs = withPreparedStatement(
+            String.format(CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT, this.leaseArbiterTableName,
+                this.leaseArbiterTableName, this.leaseArbiterTableName, this.constantsTableName),
+            insertStatement -> {
+              completeInsertPreparedStatement(insertStatement, flowAction, eventTimeMillis);
+              return insertStatement.executeQuery();
+            }, true);
+       return handleResultFromAttemptedLeaseObtainment(rs, eventTimeMillis);
+      }
+
+      // Extract values from result set
+      Timestamp dbEventTimestamp = resultSet.getTimestamp(1);
+      Timestamp dbLeaseAcquisitionTimestamp = resultSet.getTimestamp(2);
+      boolean isWithinEpsilon = resultSet.getBoolean(3);
+      int leaseValidityStatus = resultSet.getInt(4);
+      int dbLinger = resultSet.getInt(5);
+
+      // Lease is valid
+      if (leaseValidityStatus == 1) {
+        // CASE 2: Same event, lease is valid
+        if (isWithinEpsilon) {
+          // Utilize db timestamp for reminder
+          return new LeasedToAnotherStatus(dbEventTimestamp.getTime(),
+              dbLeaseAcquisitionTimestamp.getTime() + dbLinger);
+        }
+        // CASE 3: Distinct event, lease is valid
+        // Utilize db timestamp for wait time, but be reminded of own event timestamp
+        return new LeasedToAnotherStatus(eventTimeMillis,
+            dbLeaseAcquisitionTimestamp.getTime() + dbLinger);
+      }
+      // CASE 4: Lease is out of date (regardless of whether same or distinct event)
+      else if (leaseValidityStatus == 2) {
+        if (isWithinEpsilon) {
+          LOG.warn("Lease should not be out of date for the same trigger event since epsilon << linger for flowAction"
+                  + " {}, db eventTimestamp {}, db leaseAcquisitionTimestamp {}, linger {}", flowAction,
+              dbEventTimestamp, dbLeaseAcquisitionTimestamp, dbLinger);
+        }
+        // Use our event to acquire lease, check for previous db eventTimestamp and leaseAcquisitionTimestamp
+        ResultSet rs = withPreparedStatement(
+            String.format(CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT, this.leaseArbiterTableName,
+                this.leaseArbiterTableName, this.constantsTableName),
+            updateStatement -> {
+              completeUpdatePreparedStatement(updateStatement, flowAction, eventTimeMillis, true,
+                  true, dbEventTimestamp, dbLeaseAcquisitionTimestamp);
+              return updateStatement.executeQuery();
+            }, true);
+        return handleResultFromAttemptedLeaseObtainment(rs, eventTimeMillis);
+      } // No longer leasing this event
+        // CASE 5: Same event, no longer leasing event in db: terminate
+        if (isWithinEpsilon) {
+          return new NoLongerLeasingStatus();
+        }
+        // CASE 6: Distinct event, no longer leasing event in db
+        // Use our event to acquire lease, check for previous db eventTimestamp and NULL leaseAcquisitionTimestamp
+        ResultSet rs = withPreparedStatement(
+            String.format(CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT, this.leaseArbiterTableName,
+                this.leaseArbiterTableName, this.constantsTableName),
+            updateStatement -> {
+              completeUpdatePreparedStatement(updateStatement, flowAction, eventTimeMillis, true,
+                  false, dbEventTimestamp, null);
+              return updateStatement.executeQuery();
+            }, true);
+        return handleResultFromAttemptedLeaseObtainment(rs, eventTimeMillis);
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Attempt lease by insert or update following a read based on the condition the state of the table has not changed
+   * since the read. Parse the result to return the corresponding status based on successful insert/update or not.
+   * @param resultSet
+   * @param eventTimeMillis
+   * @return LeaseAttemptStatus
+   * @throws SQLException
+   * @throws IOException
+   */
+  protected LeaseAttemptStatus handleResultFromAttemptedLeaseObtainment(ResultSet resultSet, long eventTimeMillis)
+      throws SQLException, IOException {
+    if (!resultSet.next()) {
+      throw new IOException("Expected num rows and lease_acquisition_timestamp returned from query but received nothing");
+    }
+    int numRowsUpdated = resultSet.getInt(1);
+    long leaseAcquisitionTimeMillis = resultSet.getTimestamp(2).getTime();

Review Comment:
   yes, but it will be safeguarded by use of `leaseValidityStatus` which will have value of `3` if `lease_acquisition_status` is null which case we will not attempt to use this value. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210851134


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -70,21 +73,23 @@ public DagAction(String flowGroup, String flowName, String flowExecutionId, DagA
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException
    * @return true if we successfully delete one record, return false if the record does not exist
    */
-  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException;
+  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue) throws IOException;
 
   /***
    * Retrieve action value by the flow group, flow name and flow execution id from the {@link DagActionStore}.
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException Exception in retrieving the {@link DagAction}.
    * @throws SpecNotFoundException If {@link DagAction} being retrieved is not present in store.
    */
-  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException, SpecNotFoundException,
-                                                                                           SQLException;
+  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue)

Review Comment:
   You're right actually using `getDagAction` doesn't make sense anymore since the action is part of the primary key. Instead it may be useful to have `getDagActions(flow identifiers)` to get all pending actions associated with a flow right now. We don't have any explicit use case at the moment so I will remove this method. 
   
   Any method now with the store needs all columns that comprise the primary key, so we can actually pass `DagAction` to any of these functions but looking at how the functions are used we will end up creating a new `DagAction` object then pass to the function then unpack those values anyway so I am not certain that changing the signature is that beneficial unless we care more about encapsulating the idea that the PK is needed for all of these actions and that `DagAction` is PK. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1212444910


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -27,7 +27,9 @@
 public interface DagActionStore {
   enum DagActionValue {
     KILL,
-    RESUME
+    RESUME,
+    // TODO: potentially combine this enum with {@link SchedulerLeaseDeterminationStore.FlowActionType}
+    LAUNCH

Review Comment:
   Yes, that is correct. The host that wins the lease is only the one who passes that job along to the DagManager to launch (they do this by persisting in mysql). In the near future, only the leader host will actually run the job but eventually there should be a non-blocking, multi-active DagManager as well that will work similarly to win the lease of running the job. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210930039


##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitorFactory.java:
##########
@@ -40,12 +44,19 @@ public class DagActionStoreChangeMonitorFactory implements Provider<DagActionSto
   private final Config config;
   private DagActionStore dagActionStore;
   private DagManager dagManager;
+  private FlowCatalog flowCatalog;
+  private boolean isMultiActiveSchedulerEnabled;
 
   @Inject
-  public DagActionStoreChangeMonitorFactory(Config config, DagActionStore dagActionStore, DagManager dagManager) {
+  public DagActionStoreChangeMonitorFactory(Config config, DagActionStore dagActionStore, DagManager dagManager,
+      FlowCatalog flowCatalog, @Named(InjectionNames.WARM_STANDBY_ENABLED) boolean isMultiActiveSchedulerEnabled) {

Review Comment:
   this is a typo, it should be multiActiveScheduler enabled. These two configs are separate. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded its expiration.  why?  because each host must defer for the "linger" period, and *should only attempt to acquire after* that expires.  given that, if acquisition should fail because another won the race, that newly granted lease should be nowhere near expiration.
   
   c. with `NO_LONGER_LEASING`, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even set a reminder.  in fact, under normal operating conditions, such a self-timer should regularly conclude by the host determining "there's nothing to lease", because the trigger event has been fully handled.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1223581913


##########
gobblin-runtime/src/main/java/org/apache/gobblin/scheduler/JobScheduler.java:
##########
@@ -600,11 +600,17 @@ public static class GobblinJob extends BaseGobblinJob implements InterruptableJo
     @Override
     public void executeImpl(JobExecutionContext context)
         throws JobExecutionException {
-      LOG.info("Starting job " + context.getJobDetail().getKey());
-      JobDataMap dataMap = context.getJobDetail().getJobDataMap();
+      JobDetail jobDetail = context.getJobDetail();
+      LOG.info("Starting job " + jobDetail.getKey());
+      JobDataMap dataMap = jobDetail.getJobDataMap();
       JobScheduler jobScheduler = (JobScheduler) dataMap.get(JOB_SCHEDULER_KEY);
       Properties jobProps = (Properties) dataMap.get(PROPERTIES_KEY);
       JobListener jobListener = (JobListener) dataMap.get(JOB_LISTENER_KEY);
+      // Obtain trigger timestamp from trigger to pass to jobProps
+      Trigger trigger = context.getTrigger();
+      long triggerTimestampMillis = trigger.getPreviousFireTime().getTime();
+      jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY,
+          String.valueOf(triggerTimestampMillis));

Review Comment:
   Renaming to `EVENT_TO_TRIGGER` (latest trigger time of flow action to initiate) and `EVENT_TO_REVISIT` which _could_ be a different (previous) trigger of the same flow action whose lease should have expired by now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222373354


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java:
##########
@@ -580,7 +581,9 @@ public void run() {
 
     private void clearUpDagAction(DagId dagId) throws IOException {
       if (this.dagActionStore.isPresent()) {
-        this.dagActionStore.get().deleteDagAction(dagId.flowGroup, dagId.flowName, dagId.flowExecutionId);
+        this.dagActionStore.get().deleteDagAction(
+            new DagActionStore.DagAction(dagId.flowGroup, dagId.flowName, dagId.flowExecutionId,
+                DagActionStore.FlowActionType.KILL));

Review Comment:
   You're right this depends on the caller, I did not notice that. I modified the method to take the `FlowActionType` as a parameter to the function. As of now it should only have values `RESUME` or `KILL`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1213666867


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   I was also debating the cases you're identifying here particularly distinguishing whether the current host succeeded or the lease has gone to another. In my original code I expected the class implementing this interface to actually incorporate all the logic for setting a reminder event or persisting the flow trigger to the DagActionStore so the logic was less general. The abstraction you've described makes more sense as I generalize this to a `LeaseArbiter` for any flow event, so changing the enums to this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227685447


##########
gobblin-runtime/src/main/java/org/apache/gobblin/scheduler/JobScheduler.java:
##########
@@ -600,11 +600,17 @@ public static class GobblinJob extends BaseGobblinJob implements InterruptableJo
     @Override
     public void executeImpl(JobExecutionContext context)
         throws JobExecutionException {
-      LOG.info("Starting job " + context.getJobDetail().getKey());
-      JobDataMap dataMap = context.getJobDetail().getJobDataMap();
+      JobDetail jobDetail = context.getJobDetail();
+      LOG.info("Starting job " + jobDetail.getKey());
+      JobDataMap dataMap = jobDetail.getJobDataMap();
       JobScheduler jobScheduler = (JobScheduler) dataMap.get(JOB_SCHEDULER_KEY);
       Properties jobProps = (Properties) dataMap.get(PROPERTIES_KEY);
       JobListener jobListener = (JobListener) dataMap.get(JOB_LISTENER_KEY);
+      // Obtain trigger timestamp from trigger to pass to jobProps
+      Trigger trigger = context.getTrigger();
+      long triggerTimestampMillis = trigger.getPreviousFireTime().getTime();
+      jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY,
+          String.valueOf(triggerTimestampMillis));

Review Comment:
   nice choices!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1227693255


##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -96,19 +96,19 @@ public class ConfigurationKeys {
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
   // Scheduler lease determination store configuration
-  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
-  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";
-  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "scheduler.lease.determination.store.db.table";
-  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "gobblin_scheduler_lease_determination_store";
-  public static final String SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY = "reminderEventTimestampMillis";
-  public static final String SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY = "newEventTimestampMillis";
-  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "";
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "MysqlMultiActiveLeaseArbiter.constantsTable";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "MysqlMultiActiveLeaseArbiter.gobblin_multi_active_scheduler_constants_store";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "MysqlMultiActiveLeaseArbiter.schedulerLeaseArbiterTable";

Review Comment:
   no biggie, but for the lot of these, I suggest separately defining `MYSQL_LEASE_ARBITER_PREFIX` and then prepending that to each of the many keys



##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -96,19 +96,19 @@ public class ConfigurationKeys {
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
   // Scheduler lease determination store configuration
-  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
-  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";
-  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "scheduler.lease.determination.store.db.table";
-  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "gobblin_scheduler_lease_determination_store";
-  public static final String SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY = "reminderEventTimestampMillis";
-  public static final String SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY = "newEventTimestampMillis";
-  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "";
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "MysqlMultiActiveLeaseArbiter.constantsTable";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "MysqlMultiActiveLeaseArbiter.gobblin_multi_active_scheduler_constants_store";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "MysqlMultiActiveLeaseArbiter.schedulerLeaseArbiterTable";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "MysqlMultiActiveLeaseArbiter.gobblin_scheduler_lease_determination_store";
+  public static final String SCHEDULER_EVENT_TO_REVISIT_TIMESTAMP_MILLIS_KEY = "eventToRevisitTimestampMillis";
+  public static final String SCHEDULER_EVENT_TO_TRIGGER_TIMESTAMP_MILLIS_KEY = "triggerEventTimestampMillis";
+  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "MysqlMultiActiveLeaseArbiter.epsilonMillis";
   public static final int DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS = 100;

Review Comment:
   of course we'll tune this as we proceed.... still, on considering potential causes of late triggers, such as full GC pause, I'd imagine the need for a larger value.  I'd probably start w/ 5s



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -19,38 +19,38 @@
 
 import java.io.IOException;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import lombok.Data;
+
 
 /**
  * This interface defines a generic approach to a non-blocking, multiple active thread or host system, in which one or
- * more active instances compete over ownership of a particular flow's event. The type of flow event in question does
- * not impact the algorithm other than to uniquely identify the flow event. Each instance uses the interface to initiate
- * an attempt at ownership over the flow event and receives a response indicating the status of the attempt.
+ * more active participants compete to take responsiblity for a particular flow's event. The type of flow event in
+ * question does not impact the algorithm other than to uniquely identify the flow event. Each participant uses the
+ * interface to initiate an attempt at ownership over the flow event and receives a response indicating the status of
+ * the attempt.
  *
  * At a high level the lease arbiter works as follows:
- *  1. Multiple instances receive knowledge of a flow action event to act upon
- *  2. Each instance attempts to acquire rights or `a lease` to be the sole instance acting on the event by calling the
- *      tryAcquireLease method below and receives the resulting status. The status indicates whether this instance has
- *        a) acquired the lease -> then this instance will attempt to complete the lease
- *        b) another has acquired the lease -> then another will attempt to complete the lease
- *        c) flow event no longer needs to be acted upon -> terminal state
- *  3. If another has acquired the lease, then the instance will check back in at the time of lease expiry to see if it
- *    needs to attempt the lease again [status (b) above].
- *  4. Once the instance which acquired the lease completes its work on the flow event, it calls completeLeaseUse to
- *    indicate to all other instances that the flow event no longer needs to be acted upon [status (c) above]
+ *  1. Multiple participants independently learn of a flow action event to act upon
+ *  2. Each participant attempts to acquire rights or `a lease` to be the sole participant acting on the event by
+ *     calling the tryAcquireLease method below and receives the resulting status. The status indicates whether this
+ *     participant has
+ *        a) LeaseObtainedStatus -> this participant will attempt to carry out the required action before the lease expires
+ *        b) LeasedToAnotherStatus -> another will attempt to carry out the required action before the lease expires
+ *        c) NoLongerLeasingStatus -> flow event no longer needs to be acted upon or terminal state

Review Comment:
   nit: "...acted upon (terminal state)"



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -62,7 +62,38 @@ public interface MultiActiveLeaseArbiter {
    * leaseAcquisitionTimeMillis values have not changed since this owner acquired the lease (indicating the lease did
    * not expire).
    * @return true if successfully updated, indicating no further actions need to be taken regarding this event.
+   *         false if failed to update the lease properly, the caller should continue seeking to acquire the lease as
+   *         if any actions it did successfully accomplish, do not count
    */
-  boolean completeLeaseUse(DagActionStore.DagAction flowAction, long eventTimeMillis, long leaseAcquisitionTimeMillis)
-      throws IOException;
+  boolean recordLeaseSuccess(DagActionStore.DagAction flowAction, LeaseObtainedStatus status) throws IOException;
+
+  /*
+   Object used to encapsulate status of lease acquisition attempt and derived should contain information specific to

Review Comment:
   nits:
   this (and all those below) should really be class javadoc to show up in that tool
   a class (not an object)
   "derived [classes/types]" OR "derivations"



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -62,7 +62,38 @@ public interface MultiActiveLeaseArbiter {
    * leaseAcquisitionTimeMillis values have not changed since this owner acquired the lease (indicating the lease did
    * not expire).
    * @return true if successfully updated, indicating no further actions need to be taken regarding this event.
+   *         false if failed to update the lease properly, the caller should continue seeking to acquire the lease as
+   *         if any actions it did successfully accomplish, do not count
    */
-  boolean completeLeaseUse(DagActionStore.DagAction flowAction, long eventTimeMillis, long leaseAcquisitionTimeMillis)
-      throws IOException;
+  boolean recordLeaseSuccess(DagActionStore.DagAction flowAction, LeaseObtainedStatus status) throws IOException;

Review Comment:
   nit: to streamline this interface, I'd personally tunnel the `flowAction` as a package-protected member of `LeaseObtainedStatus`.  thus, the core identification of what to record can be derived from that single param.
   
   in a way this is also more robust, since one `status` could never be mismatched w/ an unrelated `flowAction`.



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -35,17 +36,12 @@ enum FlowActionType {
   }
 
   @Data
+  @AllArgsConstructor
   class DagAction {
     String flowGroup;
     String flowName;
     String flowExecutionId;
     FlowActionType flowActionType;

Review Comment:
   first off, this is beautiful... nearly looks like we're writing scala! ;p
   
   secondly, in what situations are we expecting the need to mutate the fields (i.e. why can't they be `final`?)?  in general `@Data` / POJOs work quite well being completely immutable.  (which is why such classes may not require `@AllArgsConstructor` in addition to `@Data`.)



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java:
##########
@@ -62,7 +62,38 @@ public interface MultiActiveLeaseArbiter {
    * leaseAcquisitionTimeMillis values have not changed since this owner acquired the lease (indicating the lease did
    * not expire).
    * @return true if successfully updated, indicating no further actions need to be taken regarding this event.
+   *         false if failed to update the lease properly, the caller should continue seeking to acquire the lease as
+   *         if any actions it did successfully accomplish, do not count
    */
-  boolean completeLeaseUse(DagActionStore.DagAction flowAction, long eventTimeMillis, long leaseAcquisitionTimeMillis)
-      throws IOException;
+  boolean recordLeaseSuccess(DagActionStore.DagAction flowAction, LeaseObtainedStatus status) throws IOException;
+
+  /*
+   Object used to encapsulate status of lease acquisition attempt and derived should contain information specific to
+   the status that results.
+   */
+  abstract class LeaseAttemptStatus {}
+
+  class NoLongerLeasingStatus extends LeaseAttemptStatus {}
+
+  /*
+  The participant calling this method acquired the lease for the event in question. The class contains the `eventTimestamp`
+  associated with the lease as well as the time the caller obtained the lease or `leaseAcquisitionTimestamp`.
+  */
+  @Data
+  class LeaseObtainedStatus extends LeaseAttemptStatus {
+    private final long eventTimestamp;
+    private final long leaseAcquisitionTimestamp;
+  }
+
+  /*
+  This flow action event already has a valid lease owned by another host.
+   */
+  @Data
+  class LeasedToAnotherStatus extends LeaseAttemptStatus {
+    // the timestamp the lease is associated with, but it may be a different timestamp for the same flow action
+    // (a previous participant of the event)
+    private final long eventTimeMillis;
+    // the minimum amount of time to wait before returning to check if the lease has completed or expired
+    private final long minimumLingerDurationMillis;

Review Comment:
   better to describe these in the class doc to cause them to show up in javadoc HTML and IDE help, etc.



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -30,62 +29,60 @@
 import org.quartz.JobKey;
 import org.quartz.SchedulerException;
 import org.quartz.Trigger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.typesafe.config.Config;
 
 import javax.inject.Inject;
+import lombok.extern.slf4j.Slf4j;
 
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.metrics.ContextAwareMeter;
 import org.apache.gobblin.metrics.MetricContext;
 import org.apache.gobblin.runtime.api.DagActionStore;
-import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
 import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.api.MysqlMultiActiveLeaseArbiter;
 import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
 import org.apache.gobblin.scheduler.JobScheduler;
 import org.apache.gobblin.scheduler.SchedulerService;
 import org.apache.gobblin.util.ConfigUtils;
-import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
-import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
 
 
 /**
  * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
- * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * {@link MysqlMultiActiveLeaseArbiter} to determine a single lease owner at a given time
  * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
  * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
  * the lease as completed by calling the
- * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * MysqlMultiActiveLeaseArbiter.recordLeaseSuccess method. Hosts that do not gain
  * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
  * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
  * cases.
  */
-public class SchedulerLeaseAlgoHandler {
-  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
-  private final int staggerUpperBoundSec;
+@Slf4j
+public class FlowTriggerHandler {
+  private final int schedulerMaxBackoffMillis;
   private static Random random = new Random();
   protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
   protected JobScheduler jobScheduler;
   protected SchedulerService schedulerService;
   protected DagActionStore dagActionStore;
   private MetricContext metricContext;
   private ContextAwareMeter numLeasesCompleted;
+
   @Inject
-  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+  // TODO: should multiActiveLeaseArbiter and DagActionStore be optional?

Review Comment:
   I wouldn't think so... but do elaborate on your thought process



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -30,62 +29,60 @@
 import org.quartz.JobKey;
 import org.quartz.SchedulerException;
 import org.quartz.Trigger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.typesafe.config.Config;
 
 import javax.inject.Inject;
+import lombok.extern.slf4j.Slf4j;
 
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.metrics.ContextAwareMeter;
 import org.apache.gobblin.metrics.MetricContext;
 import org.apache.gobblin.runtime.api.DagActionStore;
-import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
 import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.api.MysqlMultiActiveLeaseArbiter;
 import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
 import org.apache.gobblin.scheduler.JobScheduler;
 import org.apache.gobblin.scheduler.SchedulerService;
 import org.apache.gobblin.util.ConfigUtils;
-import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
-import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
 
 
 /**
  * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
- * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * {@link MysqlMultiActiveLeaseArbiter} to determine a single lease owner at a given time
  * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
  * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
  * the lease as completed by calling the
- * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * MysqlMultiActiveLeaseArbiter.recordLeaseSuccess method. Hosts that do not gain
  * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
  * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
  * cases.
  */
-public class SchedulerLeaseAlgoHandler {
-  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
-  private final int staggerUpperBoundSec;
+@Slf4j
+public class FlowTriggerHandler {
+  private final int schedulerMaxBackoffMillis;
   private static Random random = new Random();
   protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
   protected JobScheduler jobScheduler;
   protected SchedulerService schedulerService;
   protected DagActionStore dagActionStore;
   private MetricContext metricContext;
   private ContextAwareMeter numLeasesCompleted;
+
   @Inject
-  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+  // TODO: should multiActiveLeaseArbiter and DagActionStore be optional?
+  public FlowTriggerHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
       JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
-    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
-        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
-        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.schedulerMaxBackoffMillis = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_MAX_BACKOFF_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_MAX_BACKOFF_MILLIS);
     this.multiActiveLeaseArbiter = leaseDeterminationStore;
     this.jobScheduler = jobScheduler;
     this.schedulerService = schedulerService;
     this.dagActionStore = dagActionStore;
     this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
         this.getClass());
-    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_FLOWS_SUBMITTED);

Review Comment:
   looks like the constant needs renaming to `GOBBLIN_FLOW_TRIGGER_HANDLER_NUM_SUBMITTED`
   
   (shall we update `numLeasesCompleted` as well?)



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -30,62 +29,60 @@
 import org.quartz.JobKey;
 import org.quartz.SchedulerException;
 import org.quartz.Trigger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.typesafe.config.Config;
 
 import javax.inject.Inject;
+import lombok.extern.slf4j.Slf4j;
 
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.metrics.ContextAwareMeter;
 import org.apache.gobblin.metrics.MetricContext;
 import org.apache.gobblin.runtime.api.DagActionStore;
-import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
 import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.api.MysqlMultiActiveLeaseArbiter;
 import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
 import org.apache.gobblin.scheduler.JobScheduler;
 import org.apache.gobblin.scheduler.SchedulerService;
 import org.apache.gobblin.util.ConfigUtils;
-import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
-import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
 
 
 /**
  * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
- * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * {@link MysqlMultiActiveLeaseArbiter} to determine a single lease owner at a given time
  * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
  * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
  * the lease as completed by calling the
- * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * MysqlMultiActiveLeaseArbiter.recordLeaseSuccess method. Hosts that do not gain

Review Comment:
   I prefer the javadoc `@link` you had previously



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());

Review Comment:
   since this represents a gaping hole in our impl, it's actually more appropriate to scream/panic/freak out w/ a `RuntimeException`, than it is to presume to continue (as if completely shirking all responsibility for the `flowAction`)



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());
   }
 
   // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
-  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+  private boolean persistFlowAction(MultiActiveLeaseArbiter.LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
     try {
       this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
           flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
-      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
-          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
-        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
-        this.numLeasesCompleted.mark();
-        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
-            status.getMyLeaseAcquisitionTimestamp());
-      }
-    } catch (IOException | SQLException e) {
+      // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+      this.numLeasesCompleted.mark();
+      return this.multiActiveLeaseArbiter.recordLeaseSuccess(flowAction, status);
+    } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
-    return false;
   }
 
   /**
-   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
-   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
-   * expire.
+   * This method is used by FlowTriggerHandler.handleNewSchedulerEvent to schedule a reminder for itself to check on
+   * the other participant's progress to finish acting on a flow action after the time the lease should expire.
    * @param jobProps
    * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
    * @param originalEventTimeMillis the event timestamp we were originally handling
    * @param flowAction
    */
-  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+  private void scheduleReminderForEvent(Properties jobProps, MultiActiveLeaseArbiter.LeasedToAnotherStatus status,
       DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
     // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
-    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumLingerDurationMillis()
+        + random.nextInt(schedulerMaxBackoffMillis));
     jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
-    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    // Ensure we save the event timestamp that we're setting reminder for to have for debugging purposes
+    // in addition to the event we want to initiate
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_REVISIT_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(status.getEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_TRIGGER_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(originalEventTimeMillis));
     JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
-    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);
+    // Create a new trigger for the flow in job scheduler that is set to fire at the minimum reminder wait time calculated
+    Trigger trigger = this.jobScheduler.createTriggerForJob(key, jobProps);
     try {
-      LOG.info("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] -  attempting to schedule reminder for event %s in %s millis",
-          flowAction, originalEventTimeMillis, status.getReminderEventTimeMillis(), trigger.getNextFireTime());
+      log.info("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] -  attempting to schedule reminder for event %s in %s millis",

Review Comment:
   update to "Flow Trigger Handler -'



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());
   }
 
   // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
-  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+  private boolean persistFlowAction(MultiActiveLeaseArbiter.LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
     try {
       this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
           flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
-      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
-          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
-        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
-        this.numLeasesCompleted.mark();
-        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
-            status.getMyLeaseAcquisitionTimestamp());
-      }
-    } catch (IOException | SQLException e) {
+      // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+      this.numLeasesCompleted.mark();
+      return this.multiActiveLeaseArbiter.recordLeaseSuccess(flowAction, status);
+    } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
-    return false;
   }
 
   /**
-   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
-   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
-   * expire.
+   * This method is used by FlowTriggerHandler.handleNewSchedulerEvent to schedule a reminder for itself to check on
+   * the other participant's progress to finish acting on a flow action after the time the lease should expire.
    * @param jobProps
    * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
    * @param originalEventTimeMillis the event timestamp we were originally handling
    * @param flowAction
    */
-  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+  private void scheduleReminderForEvent(Properties jobProps, MultiActiveLeaseArbiter.LeasedToAnotherStatus status,
       DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
     // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
-    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumLingerDurationMillis()
+        + random.nextInt(schedulerMaxBackoffMillis));
     jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
-    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    // Ensure we save the event timestamp that we're setting reminder for to have for debugging purposes
+    // in addition to the event we want to initiate
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_REVISIT_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(status.getEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_TRIGGER_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(originalEventTimeMillis));
     JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
-    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);
+    // Create a new trigger for the flow in job scheduler that is set to fire at the minimum reminder wait time calculated
+    Trigger trigger = this.jobScheduler.createTriggerForJob(key, jobProps);
     try {
-      LOG.info("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] -  attempting to schedule reminder for event %s in %s millis",
-          flowAction, originalEventTimeMillis, status.getReminderEventTimeMillis(), trigger.getNextFireTime());
+      log.info("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] -  attempting to schedule reminder for event %s in %s millis",
+          flowAction, originalEventTimeMillis, status.getEventTimeMillis(), trigger.getNextFireTime());
       this.schedulerService.getScheduler().scheduleJob(trigger);
     } catch (SchedulerException e) {
-      LOG.warn("Failed to add job reminder due to SchedulerException for job %s trigger event %s ", key, status.getReminderEventTimeMillis(), e);
+      log.warn("Failed to add job reminder due to SchedulerException for job %s trigger event %s ", key, status.getEventTimeMillis(), e);
     }
-    LOG.info(String.format("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] - SCHEDULED REMINDER for event %s in %s millis",
-        flowAction, originalEventTimeMillis, status.getReminderEventTimeMillis(), trigger.getNextFireTime()));
+    log.info(String.format("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] - SCHEDULED REMINDER for event %s in %s millis",

Review Comment:
   (also here)



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());
   }
 
   // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
-  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+  private boolean persistFlowAction(MultiActiveLeaseArbiter.LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
     try {
       this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
           flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
-      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
-          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
-        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
-        this.numLeasesCompleted.mark();
-        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
-            status.getMyLeaseAcquisitionTimestamp());
-      }
-    } catch (IOException | SQLException e) {
+      // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+      this.numLeasesCompleted.mark();
+      return this.multiActiveLeaseArbiter.recordLeaseSuccess(flowAction, status);
+    } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
-    return false;
   }
 
   /**
-   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
-   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
-   * expire.
+   * This method is used by FlowTriggerHandler.handleNewSchedulerEvent to schedule a reminder for itself to check on
+   * the other participant's progress to finish acting on a flow action after the time the lease should expire.
    * @param jobProps
    * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
    * @param originalEventTimeMillis the event timestamp we were originally handling
    * @param flowAction
    */
-  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+  private void scheduleReminderForEvent(Properties jobProps, MultiActiveLeaseArbiter.LeasedToAnotherStatus status,

Review Comment:
   it may be necessary to overload this w/ a form taking `LeaseObtainedStatus`...



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -55,31 +55,27 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     try {
       // If an existing resume request is still pending then do not accept this request
       if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME)) {
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME,
-            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
-                + " action to be completed."));
+        this.handleException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait "

Review Comment:
   nit: `handleError` / `prepareError` (?)



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -166,8 +165,8 @@ public GobblinServiceJobScheduler(@Named(InjectionNames.SERVICE_NAME) String ser
       Config config,
       Optional<HelixManager> helixManager, Optional<FlowCatalog> flowCatalog, Optional<TopologyCatalog> topologyCatalog,
       Orchestrator orchestrator, SchedulerService schedulerService, Optional<UserQuotaManager> quotaManager, Optional<Logger> log,
-      @Named(InjectionNames.WARM_STANDBY_ENABLED) boolean warmStandbyEnabled, @Named(InjectionNames.MULTI_ACTIVE_SCHEDULER_ENABLED) boolean multiActiveSchedulerEnabled,
-      SchedulerLeaseAlgoHandler schedulerLeaseAlgoHandler) throws Exception {
+      @Named(InjectionNames.WARM_STANDBY_ENABLED) boolean warmStandbyEnabled,
+      Optional<FlowTriggerHandler> schedulerLeaseAlgoHandler) throws Exception {

Review Comment:
   rename `schedulerLeaseAlgoHandler`



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -55,31 +55,27 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     try {
       // If an existing resume request is still pending then do not accept this request
       if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME)) {
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME,
-            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
-                + " action to be completed."));
+        this.handleException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait "
+            + "for action to be completed.", HttpStatus.S_409_CONFLICT);
         return;
       }
       this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME);
     } catch (IOException | SQLException e) {
       log.warn(
           String.format("Failed to add execution resume action for flow %s %s %s to dag action store due to", flowGroup,
               flowName, flowExecutionId), e);
-      this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME, e);
+      this.handleException(e.getMessage(), HttpStatus.S_500_INTERNAL_SERVER_ERROR);
     }
 
   }
 
-  private void handleException (String flowGroup, String flowName, String flowExecutionId, DagActionStore.FlowActionType flowActionType, Exception e) {
-    try {
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, flowActionType)) {
-        throw new RestLiServiceException(HttpStatus.S_409_CONFLICT, e.getMessage());
-      } else {
-        throw new RestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, e.getMessage());
-      }
-    } catch (IOException | SQLException ex) {
-      throw new RestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, e.getMessage());
+  private void handleException(String exceptionMessage, HttpStatus errorType) {
+    if (errorType == HttpStatus.S_409_CONFLICT) {
+      throw new RestLiServiceException(HttpStatus.S_409_CONFLICT, exceptionMessage);
+    } else if (errorType == HttpStatus.S_400_BAD_REQUEST) {
+      new UpdateResponse(HttpStatus.S_400_BAD_REQUEST);

Review Comment:
   the other two throw... should this not?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -66,24 +67,37 @@
  * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
  * further leasing should be done for the event.
  */
-public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+@Slf4j
+public class MysqlMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
   /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
   @FunctionalInterface
   protected interface CheckedFunction<T, R> {
     R apply(T t) throws IOException, SQLException;
   }
 
-  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+  public static final String CONFIG_PREFIX = "MysqlMultiActiveLeaseArbiter";

Review Comment:
   better to define the prefix in `ConfigurationKeys`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -199,23 +197,27 @@ public LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, l
       int leaseValidityStatus = resultSet.getInt(4);

Review Comment:
   hopefully doesn't feel like overkill, but I'd abstract this by defining a `static` inner `@Data` class with an overloaded constructor (or `static` factory method) taking a `ResultSet`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -100,8 +114,8 @@ public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
   // Insert or update row to acquire lease if values have not changed since the previous read
   // Need to define three separate statements to handle cases where row does not exist or has null values to check
   protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
-      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
-      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?); "
+      + SELECT_AFTER_INSERT_STATEMENT;

Review Comment:
   just thinking... it may be clearer not to append this to the constant, but rather to catenate them at point of use



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java:
##########
@@ -579,11 +579,10 @@ public void run() {
       }
     }
 
-    private void clearUpDagAction(DagId dagId) throws IOException {
+    private void clearUpDagAction(DagId dagId, DagActionStore.FlowActionType flowActionType) throws IOException {

Review Comment:
   nit: "clear up" seems indirect and w/ a wide variety of connotations (from misunderstandings and weather... to skin care).
   
   what are we doing here?  "resolving" the action?  "dismissing"?  "deleting"?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/scheduler/JobScheduler.java:
##########
@@ -581,7 +581,7 @@ public void close() throws IOException {
   /**
    * Get a {@link org.quartz.Trigger} from the given job configuration properties.
    */
-  public Trigger getTrigger(JobKey jobKey, Properties jobProps) {
+  public Trigger createTriggerForJob(JobKey jobKey, Properties jobProps) {

Review Comment:
   couldn't this be `static`?  if so, let's!



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java:
##########
@@ -579,11 +579,10 @@ public void run() {
       }
     }
 
-    private void clearUpDagAction(DagId dagId) throws IOException {
+    private void clearUpDagAction(DagId dagId, DagActionStore.FlowActionType flowActionType) throws IOException {
       if (this.dagActionStore.isPresent()) {
         this.dagActionStore.get().deleteDagAction(
-            new DagActionStore.DagAction(dagId.flowGroup, dagId.flowName, dagId.flowExecutionId,
-                DagActionStore.FlowActionType.KILL));
+            new DagActionStore.DagAction(dagId.flowGroup, dagId.flowName, dagId.flowExecutionId, flowActionType));

Review Comment:
   minor, but depending on how prevalent this is, it might be reasonable to supply an additional constructor:
   ```
   public DagAction(DagId dagId, FlowActionType flowActionType) { ... }
   ```



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -106,22 +104,20 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable {
   private FlowStatusGenerator flowStatusGenerator;
 
   private UserQuotaManager quotaManager;
-  private boolean isMultiActiveSchedulerEnabled;
-  private SchedulerLeaseAlgoHandler schedulerLeaseAlgoHandler;
+  private Optional<FlowTriggerHandler> schedulerLeaseAlgoHandler;

Review Comment:
   rename `schedulerLeaseAlgoHandler`



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -317,26 +312,27 @@ public void orchestrate(Spec spec, Properties jobProps, long triggerTimestampMil
       }
 
       // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
-      if (this.isMultiActiveSchedulerEnabled) {
+      if (schedulerLeaseAlgoHandler.isPresent()) {
+        // If triggerTimestampMillis is 0, then it was not set by the job trigger handler, and we cannot handle this event
+        if (triggerTimestampMillis == 0L) {
+          _log.warn("Skipping execution of spec: {} because missing trigger timestamp in job properties",
+              jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
+          flowMetadata.put(TimingEvent.METADATA_MESSAGE, "Flow orchestration skipped because no trigger timestamp "
+              + "associated with flow action.");
+          if (this.eventSubmitter.isPresent()) {
+            new TimingEvent(this.eventSubmitter.get(), TimingEvent.FlowTimings.FLOW_FAILED).stop(flowMetadata);

Review Comment:
   tip: `ifPresent()`
   
   (note: this works easily and naturally... unless checked exceptions)



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);

Review Comment:
   shouldn't we confirm the return value to determine whether to schedule a reminder?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -378,6 +374,26 @@ public void orchestrate(Spec spec, Properties jobProps, long triggerTimestampMil
     Instrumented.updateTimer(this.flowOrchestrationTimer, System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
   }
 
+  public void submitFlowToDagManager(FlowSpec flowSpec, Optional<Dag<JobExecutionPlan>> jobExecutionPlanDag)
+      throws IOException {
+    if (!jobExecutionPlanDag.isPresent()) {
+      jobExecutionPlanDag = Optional.of(specCompiler.compileFlow(flowSpec));
+    }

Review Comment:
   while `Optional` is generally great, it's only needed when those calling are unaware of whether or not they are holding on to a `Dag<>` or not.  when it is statically known by the caller, instead clearer is to overload `submitFlowToDagManager` to both a unary and binary form.
   
   here the unary form merely compiles the DAG before forwarding/delegating to the binary one:
   ```
   public void submitFlowToDagManager(FlowSpec flowSpec) throws IOE {
     submitFlowToDagManager(flowSpec, specCompiler.compileFlow(flowSpec));
   }
   ```



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());
   }
 
   // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
-  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+  private boolean persistFlowAction(MultiActiveLeaseArbiter.LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
     try {
       this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
           flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
-      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
-          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
-        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
-        this.numLeasesCompleted.mark();
-        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
-            status.getMyLeaseAcquisitionTimestamp());
-      }
-    } catch (IOException | SQLException e) {
+      // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+      this.numLeasesCompleted.mark();
+      return this.multiActiveLeaseArbiter.recordLeaseSuccess(flowAction, status);
+    } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
-    return false;
   }
 
   /**
-   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
-   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
-   * expire.
+   * This method is used by FlowTriggerHandler.handleNewSchedulerEvent to schedule a reminder for itself to check on
+   * the other participant's progress to finish acting on a flow action after the time the lease should expire.
    * @param jobProps
    * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
    * @param originalEventTimeMillis the event timestamp we were originally handling
    * @param flowAction
    */
-  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+  private void scheduleReminderForEvent(Properties jobProps, MultiActiveLeaseArbiter.LeasedToAnotherStatus status,
       DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
     // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
-    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumLingerDurationMillis()
+        + random.nextInt(schedulerMaxBackoffMillis));
     jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
-    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    // Ensure we save the event timestamp that we're setting reminder for to have for debugging purposes
+    // in addition to the event we want to initiate
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_REVISIT_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(status.getEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_TRIGGER_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(originalEventTimeMillis));
     JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
-    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);
+    // Create a new trigger for the flow in job scheduler that is set to fire at the minimum reminder wait time calculated
+    Trigger trigger = this.jobScheduler.createTriggerForJob(key, jobProps);

Review Comment:
   would be ideal if this invocation could be `static`!



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());
   }
 
   // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
-  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+  private boolean persistFlowAction(MultiActiveLeaseArbiter.LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
     try {
       this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
           flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
-      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
-          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
-        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
-        this.numLeasesCompleted.mark();
-        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
-            status.getMyLeaseAcquisitionTimestamp());
-      }
-    } catch (IOException | SQLException e) {
+      // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+      this.numLeasesCompleted.mark();
+      return this.multiActiveLeaseArbiter.recordLeaseSuccess(flowAction, status);
+    } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
-    return false;
   }
 
   /**
-   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
-   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
-   * expire.
+   * This method is used by FlowTriggerHandler.handleNewSchedulerEvent to schedule a reminder for itself to check on

Review Comment:
   a. I like the `@link`
   b. "a reminder for itself" => "a self-reminder"



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -96,82 +93,79 @@ public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDet
    * @param eventTimeMillis
    * @throws IOException
    */
-  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
       throws IOException {
-    LeaseAttemptStatus leaseAttemptStatus =
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
         multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
     // TODO: add a log event or metric for each of these cases
-    switch (leaseAttemptStatus.getClass().getSimpleName()) {
-      case "LeaseObtainedStatus":
-        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
-        break;
-      case "LeasedToAnotherStatus":
-        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
-        break;
-      case "NoLongerLeasingStatus":
-        break;
-      default:
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      persistFlowAction((MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, flowAction,
+          eventTimeMillis);
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
     }
+    log.warn("Received type of leaseAttemptStatus: {} not handled by this method", leaseAttemptStatus.getClass().getName());
   }
 
   // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
-  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+  private boolean persistFlowAction(MultiActiveLeaseArbiter.LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
     try {
       this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
           flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
-      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
-          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
-        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
-        this.numLeasesCompleted.mark();
-        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
-            status.getMyLeaseAcquisitionTimestamp());
-      }
-    } catch (IOException | SQLException e) {
+      // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+      this.numLeasesCompleted.mark();
+      return this.multiActiveLeaseArbiter.recordLeaseSuccess(flowAction, status);
+    } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
-    return false;
   }
 
   /**
-   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
-   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
-   * expire.
+   * This method is used by FlowTriggerHandler.handleNewSchedulerEvent to schedule a reminder for itself to check on
+   * the other participant's progress to finish acting on a flow action after the time the lease should expire.
    * @param jobProps
    * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
    * @param originalEventTimeMillis the event timestamp we were originally handling
    * @param flowAction
    */
-  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+  private void scheduleReminderForEvent(Properties jobProps, MultiActiveLeaseArbiter.LeasedToAnotherStatus status,
       DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
     // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
-    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumLingerDurationMillis()
+        + random.nextInt(schedulerMaxBackoffMillis));
     jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
-    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
-    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    // Ensure we save the event timestamp that we're setting reminder for to have for debugging purposes
+    // in addition to the event we want to initiate
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_REVISIT_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(status.getEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_EVENT_TO_TRIGGER_TIMESTAMP_MILLIS_KEY,
+        String.valueOf(originalEventTimeMillis));
     JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
-    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);
+    // Create a new trigger for the flow in job scheduler that is set to fire at the minimum reminder wait time calculated
+    Trigger trigger = this.jobScheduler.createTriggerForJob(key, jobProps);
     try {
-      LOG.info("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] -  attempting to schedule reminder for event %s in %s millis",
-          flowAction, originalEventTimeMillis, status.getReminderEventTimeMillis(), trigger.getNextFireTime());
+      log.info("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] -  attempting to schedule reminder for event %s in %s millis",
+          flowAction, originalEventTimeMillis, status.getEventTimeMillis(), trigger.getNextFireTime());
       this.schedulerService.getScheduler().scheduleJob(trigger);
     } catch (SchedulerException e) {
-      LOG.warn("Failed to add job reminder due to SchedulerException for job %s trigger event %s ", key, status.getReminderEventTimeMillis(), e);
+      log.warn("Failed to add job reminder due to SchedulerException for job %s trigger event %s ", key, status.getEventTimeMillis(), e);
     }
-    LOG.info(String.format("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] - SCHEDULED REMINDER for event %s in %s millis",
-        flowAction, originalEventTimeMillis, status.getReminderEventTimeMillis(), trigger.getNextFireTime()));
+    log.info(String.format("Scheduler Lease Algo Handler - [%s, eventTimestamp: %s] - SCHEDULED REMINDER for event %s in %s millis",
+        flowAction, originalEventTimeMillis, status.getEventTimeMillis(), trigger.getNextFireTime()));
   }
 
   /**
    * These methods should only be called from the Orchestrator or JobScheduler classes as it directly adds jobs to the
    * Quartz scheduler
-   * @param delayPeriodSeconds
+   * @param delayPeriodMillis
    * @return
    */
-  protected static String createCronFromDelayPeriod(long delayPeriodSeconds) {
+  protected static String createCronFromDelayPeriod(long delayPeriodMillis) {
     LocalDateTime now = LocalDateTime.now(ZoneId.of("UTC"));
-    LocalDateTime delaySecondsLater = now.plus(delayPeriodSeconds, ChronoUnit.SECONDS);
+    LocalDateTime delaySecondsLater = now.plus(delayPeriodMillis, ChronoUnit.MILLIS);

Review Comment:
   nit: `delaySecondsLater` is neither seconds nor an interval, but an absolute time in millis



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -165,9 +161,8 @@ public Orchestrator(Config config, Optional<TopologyCatalog> topologyCatalog, Op
 
   @Inject
   public Orchestrator(Config config, FlowStatusGenerator flowStatusGenerator, Optional<TopologyCatalog> topologyCatalog,
-      Optional<DagManager> dagManager, Optional<Logger> log, @Named(InjectionNames.MULTI_ACTIVE_SCHEDULER_ENABLED) boolean multiActiveSchedulerEnabled,
-      SchedulerLeaseAlgoHandler schedulerLeaseAlgoHandler) {
-    this(config, topologyCatalog, dagManager, log, flowStatusGenerator, true, multiActiveSchedulerEnabled, schedulerLeaseAlgoHandler);
+      Optional<DagManager> dagManager, Optional<Logger> log, Optional<FlowTriggerHandler> schedulerLeaseAlgoHandler) {

Review Comment:
   need to rename `schedulerLeaseAlgoHandler`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -66,24 +67,37 @@
  * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
  * further leasing should be done for the event.
  */
-public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+@Slf4j
+public class MysqlMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
   /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
   @FunctionalInterface
   protected interface CheckedFunction<T, R> {
     R apply(T t) throws IOException, SQLException;
   }
 
-  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+  public static final String CONFIG_PREFIX = "MysqlMultiActiveLeaseArbiter";
 
   protected final DataSource dataSource;
   private final String leaseArbiterTableName;
   private final String constantsTableName;
   private final int epsilon;
   private final int linger;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "

Review Comment:
   suggest no default since probably shouldn't be possible to insert w/o a specific `event_timestamp`
   
   (the default for `lease_acq_tstamp` seems reasonable)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222657174


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    switch (leaseAttemptStatus.getClass().getSimpleName()) {
+      case "LeaseObtainedStatus":
+        finalizeLease((LeaseObtainedStatus) leaseAttemptStatus, flowAction);
+        break;
+      case "LeasedToAnotherStatus":
+        scheduleReminderForEvent(jobProps, (LeasedToAnotherStatus) leaseAttemptStatus, flowAction, eventTimeMillis);
+        break;
+      case "NoLongerLeasingStatus":
+        break;
+      default:
+    }
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean finalizeLease(LeaseObtainedStatus status, DagActionStore.DagAction flowAction) {
+    try {
+      this.dagActionStore.addDagAction(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType());
+      if (this.dagActionStore.exists(flowAction.getFlowGroup(), flowAction.getFlowName(),
+          flowAction.getFlowExecutionId(), flowAction.getFlowActionType())) {
+        // If the flow action has been persisted to the {@link DagActionStore} we can close the lease
+        this.numLeasesCompleted.mark();
+        return this.multiActiveLeaseArbiter.completeLeaseUse(flowAction, status.getEventTimestamp(),
+            status.getMyLeaseAcquisitionTimestamp());
+      }
+    } catch (IOException | SQLException e) {
+      throw new RuntimeException(e);
+    }
+    // TODO: should this return an error or print a warning log if failed to commit to dag action store?
+    return false;
+  }
+
+  /**
+   * This method is used by {@link SchedulerLeaseAlgoHandler.handleNewSchedulerEvent} to schedule a reminder for itself
+   * to check on the other participant's progress to finish acting on a flow action after the time the lease should
+   * expire.
+   * @param jobProps
+   * @param status used to extract event to be reminded for and the minimum time after which reminder should occur
+   * @param originalEventTimeMillis the event timestamp we were originally handling
+   * @param flowAction
+   */
+  private void scheduleReminderForEvent(Properties jobProps, LeasedToAnotherStatus status,
+      DagActionStore.DagAction flowAction, long originalEventTimeMillis) {
+    // Add a small randomization to the minimum reminder wait time to avoid 'thundering herd' issue
+    String cronExpression = createCronFromDelayPeriod(status.getMinimumReminderWaitMillis() + random.nextInt(staggerUpperBoundSec));
+    jobProps.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, cronExpression);
+    // Ensure we save the event timestamp that we're setting reminder for, in addition to our own event timestamp which may be different
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    jobProps.setProperty(ConfigurationKeys.SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY, String.valueOf(status.getReminderEventTimeMillis()));
+    JobKey key = new JobKey(flowAction.getFlowName(), flowAction.getFlowGroup());
+    Trigger trigger = this.jobScheduler.getTrigger(key, jobProps);

Review Comment:
   sorry, I guess I'm unfamiliar: what's the meaning/nature of this `Trigger` we get from one scheduler and give to another?
   
   (I'd love to avoid bringing in the dependency on `JobScheduler` if we can avoid it, and instead have this class depend only on the scheduler.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222665925


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/SchedulerLeaseAlgoHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sql.SQLException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.LeaseAttemptStatus;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.LeaseObtainedStatus;
+import org.apache.gobblin.runtime.api.LeasedToAnotherStatus;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link org.apache.gobblin.runtime.api.MySQLMultiActiveLeaseArbiter.completeLeaseUse} method. Hosts that do not gain
+ * the lease for the event, instead schedule a reminder using the {@link SchedulerService} to check back in on the
+ * previous lease owner's completion status after the lease should expire to ensure the event is handled in failure
+ * cases.
+ */
+public class SchedulerLeaseAlgoHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseAlgoHandler.class);
+  private final int staggerUpperBoundSec;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected JobScheduler jobScheduler;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numLeasesCompleted;
+  @Inject
+  public SchedulerLeaseAlgoHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      JobScheduler jobScheduler, SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.staggerUpperBoundSec = ConfigUtils.getInt(config,
+        ConfigurationKeys.SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_STAGGERING_UPPER_BOUND_SEC);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.jobScheduler = jobScheduler;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numLeasesCompleted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_SCHEDULER_LEASE_ALGO_HANDLER_NUM_LEASES_COMPLETED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleNewSchedulerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)

Review Comment:
   AFAICT, this is the only "real" method in the class's external interface.  revisiting the class and method naming in light of that might suggest `FlowTriggerHandler`.  (it's not really "handling" a "scheduler lease".)  WDYT?
   
   if so this method might be `handleTriggerEvent` or even `handle`, or merely `apply`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222561966


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT "
+      + "EXISTS (SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_KEY + "); " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL; " + SELECT_AFTER_INSERT_STATEMENT;
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?; " + SELECT_AFTER_INSERT_STATEMENT;
+
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "lease_acquisition_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action))";
+
+  private static final String CREATE_CONSTANTS_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %s "
+      + "(epsilon INT, linger INT), PRIMARY KEY (epsilon, linger); INSERT INTO %s (epsilon, linger) VALUES (?,?)";
+
+  @Inject
+  public MySQLMultiActiveLeaseArbiter(Config config) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MySQLMultiActiveLeaseArbiter");

Review Comment:
   minor: but I suggest inserting `CONFIG_PREFIX` into the error message, so there's no confusion there



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222591080


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MySQLMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing instances. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between instances and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+public class MySQLMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  public static final String CONFIG_PREFIX = "MySQLMultiActiveLeaseArbiter";
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=? AND event_timestamp=? AND lease_acquisition_timestamp=?";
+
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;

Review Comment:
   don't we also need to get the `event_timestamp` in case another value, different from ours was successfully inserted?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222198550


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java:
##########
@@ -54,27 +53,26 @@ public void resume(ComplexResourceKey<org.apache.gobblin.service.FlowStatusId, E
     String flowName = key.getKey().getFlowName();
     Long flowExecutionId = key.getKey().getFlowExecutionId();
     try {
-      // If an existing resume or kill request is still pending then do not accept this request
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString())) {
-        DagActionStore.DagActionValue action = this.dagActionStore.getDagAction(flowGroup, flowName, flowExecutionId.toString()).getDagActionValue();
-        this.handleException(flowGroup, flowName, flowExecutionId.toString(),
-            new RuntimeException("There is already a pending action " + action + " for this flow. Please wait to resubmit and wait for"
+      // If an existing resume request is still pending then do not accept this request
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME)) {
+        this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME,
+            new RuntimeException("There is already a pending RESUME action for this flow. Please wait to resubmit and wait for"
                 + " action to be completed."));
         return;
       }
-      this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.DagActionValue.RESUME);
-    } catch (IOException | SQLException | SpecNotFoundException e) {
+      this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME);
+    } catch (IOException | SQLException e) {
       log.warn(
           String.format("Failed to add execution resume action for flow %s %s %s to dag action store due to", flowGroup,
               flowName, flowExecutionId), e);
-      this.handleException(flowGroup, flowName, flowExecutionId.toString(), e);
+      this.handleException(flowGroup, flowName, flowExecutionId.toString(), DagActionStore.FlowActionType.RESUME, e);
     }
 
   }
 
-  private void handleException (String flowGroup, String flowName, String flowExecutionId, Exception e) {
+  private void handleException (String flowGroup, String flowName, String flowExecutionId, DagActionStore.FlowActionType flowActionType, Exception e) {
     try {
-      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId)) {
+      if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, flowActionType)) {

Review Comment:
   having a separate, subsequent, and repeated check of `exists` looks like a race condition.  instead whoever calls `handleException` should indicate the result of the first (and ideally only) call to `exists`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1222063860


##########
gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java:
##########
@@ -95,6 +95,20 @@ public class ConfigurationKeys {
   public static final int DEFAULT_LOAD_SPEC_BATCH_SIZE = 500;
   public static final String SKIP_SCHEDULING_FLOWS_AFTER_NUM_DAYS = "skip.scheduling.flows.after.num.days";
   public static final int DEFAULT_NUM_DAYS_TO_SKIP_AFTER = 365;
+  // Scheduler lease determination store configuration
+  public static final String MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY = "multi.active.scheduler.constants.db.table";
+  public static final String DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE = "multi.active.scheduler.";
+  public static final String SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY = "scheduler.lease.determination.store.db.table";
+  public static final String DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE = "gobblin_scheduler_lease_determination_store";
+  public static final String SCHEDULER_REMINDER_EVENT_TIMESTAMP_MILLIS_KEY = "reminderEventTimestampMillis";
+  public static final String SCHEDULER_NEW_EVENT_TIMESTAMP_MILLIS_KEY = "newEventTimestampMillis";
+  public static final String SCHEDULER_EVENT_EPSILON_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS = 100;
+  // Note: linger should be on the order of seconds even though we measure in millis
+  public static final String SCHEDULER_EVENT_LINGER_MILLIS_KEY = "";
+  public static final int DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS = 30000;
+  public static final String SCHEDULER_STAGGERING_UPPER_BOUND_SEC_KEY = "";

Review Comment:
   `UPPER_BOUND` => `MAX` ?
   
   `STAGGERING` => `DELAY` or `BACKOFF`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1213714880


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;

Review Comment:
   Yes you're right there is a missing close paren right before the `SELECT ROW_COUNT`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1211107701


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   we're thinking similarly, but I see a few diffs:
   
   a. the most consequential discernment is whether the current host succeeded in acquiring the lease (hence must be separated out as THE ONLY state for case 1)
   
   b. the "linger" timeout should not be so short that the current host would fail to acquire, but at that same moment discover that whoever else "has" the lease has actually exceeded its expiration.  why?  because each host must defer for the "linger" period, and *should only attempt to acquire after* that expires.  hence, if acquisition should fail because another won the race, that newly granted lease should be nowhere near expiration.
   
   c. with `NO_LONGER_LEASING`, I sought to capture the termination case, where a host realizes the trigger event has been fully handled, so there's neither a need to try to acquire nor even set a reminder.  in fact, under normal operating conditions, such a self-timer should regularly conclude by the host determining "there's nothing to lease", because the trigger event has been fully handled.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1212454064


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlSchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+
+import javax.sql.DataSource;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+public class MysqlSchedulerLeaseDeterminationStore implements SchedulerLeaseDeterminationStore {
+  public static final String CONFIG_PREFIX = "MysqlSchedulerLeaseDeterminationStore";
+
+  protected final DataSource dataSource;
+  private final DagActionStore dagActionStore;
+  private final String tableName;
+  private final long epsilon;
+  private final long linger;
+  /* TODO:
+     - define retention on this table
+     - initialize table with epsilon and linger if one already doesn't exist using these configs
+     - join with table above to ensure epsilon/linger values are consistent across hosts (in case hosts are deployed with different configs)
+   */
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=? "
+      + "AND flow_action=? AND ABS(trigger_event_timestamp-?) <= %s";
+  protected static final String ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT = "INSERT INTO %s (flow_group, "
+      + "flow_name, flow_execution_id, flow_action, trigger_event_timestamp) VALUES (?, ?, ?, ?, ?) WHERE NOT EXISTS ("
+      + "SELECT * FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW + "; SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "pursuant_timestamp FROM %s " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  protected static final String UPDATE_PURSUANT_TIMESTAMP_STATEMENT = "UPDATE %s SET pursuant_timestamp = NULL "
+      + WHERE_CLAUSE_TO_MATCH_ROW;
+  private static final String CREATE_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S (" + "flow_group varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "trigger_event_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, "
+      + "pursuant_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action,trigger_event_timestamp)";
+
+  @Inject
+  public MysqlSchedulerLeaseDeterminationStore(Config config, DagActionStore dagActionStore) throws IOException {
+    if (config.hasPath(CONFIG_PREFIX)) {
+      config = config.getConfig(CONFIG_PREFIX).withFallback(config);
+    } else {
+      throw new IOException("Please specify the config for MysqlSchedulerLeaseDeterminationStore");
+    }
+
+    this.tableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.epsilon = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getLong(config, ConfigurationKeys.SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_TRIGGER_EVENT_EPSILON_MILLIS);
+
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(CREATE_TABLE_STATEMENT, tableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + tableName, e);
+    }
+    this.dagActionStore = dagActionStore;
+  }
+
+  @Override
+  public LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis)
+      throws IOException {
+    Timestamp triggerTimestamp = new Timestamp(triggerTimeMillis);
+    try (Connection connection = this.dataSource.getConnection();
+        PreparedStatement insertStatement = connection.prepareStatement(
+            String.format(ATTEMPT_INSERT_AND_GET_PURSUANT_TIMESTAMP_STATEMENT, tableName, tableName, epsilon, tableName,
+                epsilon))) {
+      int i = 0;
+      // Values to set in new row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to check if existing row matches
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      // Values to make select statement to read row
+      insertStatement.setString(++i, flowGroup);
+      insertStatement.setString(++i, flowName);
+      insertStatement.setString(++i, flowExecutionId);
+      insertStatement.setString(++i, flowActionType.toString());
+      insertStatement.setTimestamp(++i, triggerTimestamp);
+      ResultSet resultSet = insertStatement.executeQuery();
+      connection.commit();
+
+      if (!resultSet.next()) {
+        resultSet.close();
+        throw new IOException(String.format("Unexpected error where no result returned while trying to obtain lease. "
+                + "This error indicates that no entry existed for trigger flow event for table %s flow group: %s, flow "
+                + "name: %s flow execution id: %s and trigger timestamp: %s when one should have been inserted",
+            tableName, flowGroup, flowName, flowExecutionId, triggerTimestamp));
+      }
+      // If a row was inserted, then we have obtained the lease
+      int rowsUpdated = resultSet.getInt(1);
+      if (rowsUpdated == 1) {
+        // If the pursuing flow launch has been persisted to the {@link DagActionStore} we have completed lease obtainment
+        this.dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH);
+        if (this.dagActionStore.exists(flowGroup, flowName, flowExecutionId, DagActionStore.DagActionValue.LAUNCH)) {

Review Comment:
   Yes, it is jumping in to carry out the task. I will separate the handling of the lease from the actual lease determination algorithm and instead keep this as a higher level MySQL implementation of the lease aquiring logic. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1210925266


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -440,7 +446,11 @@ public synchronized void scheduleJob(Properties jobProps, JobListener jobListene
   public void runJob(Properties jobProps, JobListener jobListener) throws JobException {
     try {
       Spec flowSpec = this.scheduledFlowSpecs.get(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
-      this.orchestrator.orchestrate(flowSpec);
+      String triggerTimestampMillis =
+          jobProps.containsKey(ConfigurationKeys.SCHEDULER_ORIGINAL_TRIGGER_TIMESTAMP_MILLIS_KEY)
+              ? jobProps.getProperty(ConfigurationKeys.SCHEDULER_ORIGINAL_TRIGGER_TIMESTAMP_MILLIS_KEY, "0L"):
+              jobProps.getProperty(ConfigurationKeys.SCHEDULER_TRIGGER_TIMESTAMP_MILLIS_KEY,"0L");
+      this.orchestrator.orchestrate(flowSpec, jobProps, Long.parseLong(triggerTimestampMillis));

Review Comment:
   agree, I can abstract this with a method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] umustafi commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "umustafi (via GitHub)" <gi...@apache.org>.
umustafi commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1228766979


##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -199,23 +197,27 @@ public LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, l
       int leaseValidityStatus = resultSet.getInt(4);

Review Comment:
   I don't see a large benefit from this since the static class will have to encode these column retrievals anyway but I made it a bit more clear by using the column name instead of index to retrieve the values so it's more readable. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1230649070


##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -364,6 +374,27 @@ public void orchestrate(Spec spec) throws Exception {
     Instrumented.updateTimer(this.flowOrchestrationTimer, System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
   }
 
+  public void submitFlowToDagManager(FlowSpec flowSpec)
+      throws IOException {
+    submitFlowToDagManager(flowSpec, specCompiler.compileFlow(flowSpec));
+  }
+  public void submitFlowToDagManager(FlowSpec flowSpec, Dag<JobExecutionPlan> jobExecutionPlanDag)

Review Comment:
   missing line between method defs



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Random;
+
+import org.quartz.JobKey;
+import org.quartz.SchedulerException;
+import org.quartz.Trigger;
+
+import com.typesafe.config.Config;
+
+import javax.inject.Inject;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMeter;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.api.MysqlMultiActiveLeaseArbiter;
+import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Handler used to coordinate multiple hosts with enabled schedulers to respond to flow action events. It uses the
+ * {@link MysqlMultiActiveLeaseArbiter} to determine a single lease owner at a given time
+ * for a flow action event. After acquiring the lease, it persists the flow action event to the {@link DagActionStore}
+ * to be eventually acted upon by the host with the active DagManager. Once it has completed this action, it will mark
+ * the lease as completed by calling the
+ * {@link MysqlMultiActiveLeaseArbiter.recordLeaseSuccess()} method. Hosts that do not gain the lease for the event,
+ * instead schedule a reminder using the {@link SchedulerService} to check back in on the previous lease owner's
+ * completion status after the lease should expire to ensure the event is handled in failure cases.
+ */
+@Slf4j
+public class FlowTriggerHandler {
+  private final int schedulerMaxBackoffMillis;
+  private static Random random = new Random();
+  protected MultiActiveLeaseArbiter multiActiveLeaseArbiter;
+  protected SchedulerService schedulerService;
+  protected DagActionStore dagActionStore;
+  private MetricContext metricContext;
+  private ContextAwareMeter numFlowsSubmitted;
+
+  @Inject
+  public FlowTriggerHandler(Config config, MultiActiveLeaseArbiter leaseDeterminationStore,
+      SchedulerService schedulerService, DagActionStore dagActionStore) {
+    this.schedulerMaxBackoffMillis = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_MAX_BACKOFF_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_MAX_BACKOFF_MILLIS);
+    this.multiActiveLeaseArbiter = leaseDeterminationStore;
+    this.schedulerService = schedulerService;
+    this.dagActionStore = dagActionStore;
+    this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(ConfigUtils.configToProperties(config)),
+        this.getClass());
+    this.numFlowsSubmitted = metricContext.contextAwareMeter(RuntimeMetrics.GOBBLIN_FLOW_TRIGGER_HANDLER_NUM_FLOWS_SUBMITTED);
+  }
+
+  /**
+   * This method is used in the multi-active scheduler case for one or more hosts to respond to a flow action event
+   * by attempting a lease for the flow event and processing the result depending on the status of the attempt.
+   * @param jobProps
+   * @param flowAction
+   * @param eventTimeMillis
+   * @throws IOException
+   */
+  public void handleTriggerEvent(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus =
+        multiActiveLeaseArbiter.tryAcquireLease(flowAction, eventTimeMillis);
+    // TODO: add a log event or metric for each of these cases
+    if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) {
+      MultiActiveLeaseArbiter.LeaseObtainedStatus leaseObtainedStatus = (MultiActiveLeaseArbiter.LeaseObtainedStatus) leaseAttemptStatus;
+      if (persistFlowAction(leaseObtainedStatus)) {
+        return;
+      }
+      // If persisting the flow action failed, then we set another trigger for this event to occur immediately to
+      // re-attempt handling the event
+      scheduleReminderForEvent(jobProps, new MultiActiveLeaseArbiter.LeasedToAnotherStatus(flowAction,
+          leaseObtainedStatus.getEventTimestamp(), 0L), eventTimeMillis);
+      return;
+    } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) {
+      scheduleReminderForEvent(jobProps, (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus,
+          eventTimeMillis);
+      return;
+    } else if (leaseAttemptStatus instanceof  MultiActiveLeaseArbiter.NoLongerLeasingStatus) {
+      return;
+    }
+    throw new RuntimeException(String.format("Received type of leaseAttemptStatus: %s not handled by this method",
+            leaseAttemptStatus.getClass().getName()));
+  }
+
+  // Called after obtaining a lease to persist the flow action to {@link DagActionStore} and mark the lease as done
+  private boolean persistFlowAction(MultiActiveLeaseArbiter.LeaseObtainedStatus status) {

Review Comment:
   minor, but IMO `lease` seems much clearer than `status`



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java:
##########
@@ -93,7 +105,7 @@ protected void assignTopicPartitions() {
 
   @Override
   /*
-  This class is multi-threaded and this message will be called by multiple threads, however any given message will be
+  This class is multithreaded and this message will be called by multiple threads, however any given message will be

Review Comment:
   "this *message* will be called", or "this *method* will be called"?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.zaxxer.hikari.HikariDataSource;
+
+import javax.sql.DataSource;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlDataSourceFactory;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * MySQL based implementation of the {@link MultiActiveLeaseArbiter} which uses a MySQL store to resolve ownership of
+ * a flow event amongst multiple competing participants. A MySQL table is used to store flow identifying information as
+ * well as the flow action associated with it. It uses two additional values of the `event_timestamp` and
+ * `lease_acquisition_timestamp` to indicate an active lease, expired lease, and state of no longer leasing. The table
+ * schema is as follows:
+ * [flow_group | flow_name | flow_execution_id | flow_action | event_timestamp | lease_acquisition_timestamp]
+ * (----------------------primary key------------------------)
+ * We also maintain another table in the database with two constants that allow us to coordinate between participants and
+ * ensure they are using the same values to base their coordination off of.
+ * [epsilon | linger]
+ * `epsilon` - time within we consider to timestamps to be the same, to account for between-host clock drift
+ * `linger` - minimum time to occur before another host may attempt a lease on a flow event. It should be much greater
+ *            than epsilon and encapsulate executor communication latency including retry attempts
+ *
+ * The `event_timestamp` is the time of the flow_action event request.
+ * ---Event consolidation---
+ * Note that for the sake of simplification, we only allow one event associated with a particular flow's flow_action
+ * (ie: only one LAUNCH for example of flow FOO, but there can be a LAUNCH, KILL, & RESUME for flow FOO at once) during
+ * the time it takes to execute the flow action. In most cases, the execution time should be so negligible that this
+ * event consolidation of duplicate flow action requests is not noticed and even during executor downtime this behavior
+ * is acceptable as the user generally expects a timely execution of the most recent request rather than one execution
+ * per request.
+ *
+ * The `lease_acquisition_timestamp` is the time a host acquired ownership of this flow action, and it is valid for
+ * `linger` period of time after which it expires and any host can re-attempt ownership. In most cases, the original
+ * host should actually complete its work while having the lease and then mark the flow action as NULL to indicate no
+ * further leasing should be done for the event.
+ */
+@Slf4j
+public class MysqlMultiActiveLeaseArbiter implements MultiActiveLeaseArbiter {
+  /** `j.u.Function` variant for an operation that may @throw IOException or SQLException: preserves method signature checked exceptions */
+  @FunctionalInterface
+  protected interface CheckedFunction<T, R> {
+    R apply(T t) throws IOException, SQLException;
+  }
+
+  protected final DataSource dataSource;
+  private final String leaseArbiterTableName;
+  private final String constantsTableName;
+  private final int epsilon;
+  private final int linger;
+
+  // TODO: define retention on this table
+  private static final String CREATE_LEASE_ARBITER_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %S ("
+      + "flow_group varchar(" + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, flow_name varchar("
+      + ServiceConfigKeys.MAX_FLOW_GROUP_LENGTH + ") NOT NULL, " + "flow_execution_id varchar("
+      + ServiceConfigKeys.MAX_FLOW_EXECUTION_ID_LENGTH + ") NOT NULL, flow_action varchar(100) NOT NULL, "
+      + "event_timestamp TIMESTAMP, "
+      + "lease_acquisition_timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP,"
+      + "PRIMARY KEY (flow_group,flow_name,flow_execution_id,flow_action))";
+  private static final String CREATE_CONSTANTS_TABLE_STATEMENT = "CREATE TABLE IF NOT EXISTS %s "
+      + "(epsilon INT, linger INT), PRIMARY KEY (epsilon, linger); INSERT INTO %s (epsilon, linger) VALUES (?,?)";
+  protected static final String WHERE_CLAUSE_TO_MATCH_KEY = "WHERE flow_group=? AND flow_name=? AND flow_execution_id=?"
+      + " AND flow_action=?";
+  protected static final String WHERE_CLAUSE_TO_MATCH_ROW = WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?";
+  protected static final String SELECT_AFTER_INSERT_STATEMENT = "SELECT ROW_COUNT() AS rows_inserted_count, "
+      + "lease_acquisition_timestamp, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+  // Does a cross join between the two tables to have epsilon and linger values available. Returns the following values:
+  // event_timestamp, lease_acquisition_timestamp, isWithinEpsilon (boolean if event_timestamp in table is within
+  // epsilon), leaseValidityStatus (1 if lease has not expired, 2 if expired, 3 if column is NULL or no longer leasing)
+  protected static final String GET_EVENT_INFO_STATEMENT = "SELECT event_timestamp, lease_acquisition_timestamp, "
+      + "abs(event_timestamp - ?) <= epsilon as isWithinEpsilon, CASE "
+      + "WHEN CURRENT_TIMESTAMP < (lease_acquisition_timestamp + linger) then 1"
+      + "WHEN CURRENT_TIMESTAMP >= (lease_acquisition_timestamp + linger) then 2"
+      + "ELSE 3 END as leaseValidityStatus, linger FROM %s, %s " + WHERE_CLAUSE_TO_MATCH_KEY;
+  // Insert or update row to acquire lease if values have not changed since the previous read
+  // Need to define three separate statements to handle cases where row does not exist or has null values to check
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT = "INSERT INTO %s "
+      + "(flow_group, flow_name, flow_execution_id, flow_action, event_timestamp) VALUES (?, ?, ?, ?, ?)";
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_FINISHED_LEASING_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_KEY
+      + " AND event_timestamp=? AND lease_acquisition_timestamp is NULL";
+  protected static final String CONDITIONALLY_ACQUIRE_LEASE_IF_MATCHING_ALL_COLS_STATEMENT = "UPDATE %s "
+      + "SET event_timestamp=?" + WHERE_CLAUSE_TO_MATCH_ROW
+      + " AND event_timestamp=? AND lease_acquisition_timestamp=?";
+  // Complete lease acquisition if values have not changed since lease was acquired
+  protected static final String CONDITIONALLY_COMPLETE_LEASE_STATEMENT = "UPDATE %s SET "
+      + "lease_acquisition_timestamp = NULL " + WHERE_CLAUSE_TO_MATCH_ROW;
+
+  @Inject
+  public MysqlMultiActiveLeaseArbiter(Config config) throws IOException {
+    if (config.hasPath(ConfigurationKeys.MYSQL_LEASE_ARBITER_PREFIX)) {
+      config = config.getConfig(ConfigurationKeys.MYSQL_LEASE_ARBITER_PREFIX).withFallback(config);
+    } else {
+      throw new IOException(String.format("Please specify the config for MysqlMultiActiveLeaseArbiter using prefix %s "
+          + "before all properties", ConfigurationKeys.MYSQL_LEASE_ARBITER_PREFIX));
+    }
+
+    this.leaseArbiterTableName = ConfigUtils.getString(config, ConfigurationKeys.SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_LEASE_DETERMINATION_STORE_DB_TABLE);
+    this.constantsTableName = ConfigUtils.getString(config, ConfigurationKeys.MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE_KEY,
+        ConfigurationKeys.DEFAULT_MULTI_ACTIVE_SCHEDULER_CONSTANTS_DB_TABLE);
+    this.epsilon = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_EPSILON_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_EPSILON_MILLIS);
+    this.linger = ConfigUtils.getInt(config, ConfigurationKeys.SCHEDULER_EVENT_LINGER_MILLIS_KEY,
+        ConfigurationKeys.DEFAULT_SCHEDULER_EVENT_LINGER_MILLIS);
+    this.dataSource = MysqlDataSourceFactory.get(config, SharedResourcesBrokerFactory.getImplicitBroker());
+    try (Connection connection = dataSource.getConnection();
+        PreparedStatement createStatement = connection.prepareStatement(String.format(
+            CREATE_LEASE_ARBITER_TABLE_STATEMENT, leaseArbiterTableName))) {
+      createStatement.executeUpdate();
+      connection.commit();
+    } catch (SQLException e) {
+      throw new IOException("Table creation failure for " + leaseArbiterTableName, e);
+    }
+    withPreparedStatement(String.format(CREATE_CONSTANTS_TABLE_STATEMENT, this.constantsTableName, this.constantsTableName),
+        createStatement -> {
+      int i = 0;
+      createStatement.setInt(++i, epsilon);
+      createStatement.setInt(++i, linger);
+      return createStatement.executeUpdate();}, true);
+  }
+
+  @Override
+  public LeaseAttemptStatus tryAcquireLease(DagActionStore.DagAction flowAction, long eventTimeMillis)
+      throws IOException {
+    // Check table for an existing entry for this flow action and event time
+    ResultSet resultSet = withPreparedStatement(
+        String.format(GET_EVENT_INFO_STATEMENT, this.leaseArbiterTableName, this.constantsTableName),
+        getInfoStatement -> {
+          int i = 0;
+          getInfoStatement.setTimestamp(++i, new Timestamp(eventTimeMillis));
+          getInfoStatement.setString(++i, flowAction.getFlowGroup());
+          getInfoStatement.setString(++i, flowAction.getFlowName());
+          getInfoStatement.setString(++i, flowAction.getFlowExecutionId());
+          getInfoStatement.setString(++i, flowAction.getFlowActionType().toString());
+          return getInfoStatement.executeQuery();
+        }, true);
+
+    try {
+      // CASE 1: If no existing row for this flow action, then go ahead and insert
+      if (!resultSet.next()) {
+        ResultSet rs = withPreparedStatement(
+            String.format(CONDITIONALLY_ACQUIRE_LEASE_IF_NEW_ROW_STATEMENT + "; " + SELECT_AFTER_INSERT_STATEMENT,

Review Comment:
   personally, for clarity's sake, I'd `format` each separately and then join them via ";"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] Will-Lo merged pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "Will-Lo (via GitHub)" <gi...@apache.org>.
Will-Lo merged PR #3700:
URL: https://github.com/apache/gobblin/pull/3700


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [gobblin] phet commented on a diff in pull request #3700: [GOBBLIN-1837] Implement multi-active, non blocking for leader host

Posted by "phet (via GitHub)" <gi...@apache.org>.
phet commented on code in PR #3700:
URL: https://github.com/apache/gobblin/pull/3700#discussion_r1207456238


##########
gobblin-metrics-libs/gobblin-metrics-base/src/main/avro/DagActionStoreChangeEvent.avsc:
##########
@@ -23,6 +23,11 @@
     "type" : "string",
     "doc" : "flow execution id for the dag action",
     "compliance" : "NONE"
+  }, {
+    "name" : "dagAction",
+    "type" : "string",

Review Comment:
   I see reasonable arguments for an enum here, as there are only a small number of actions.  what do you see as pros/cons?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java:
##########
@@ -137,12 +139,12 @@ public boolean deleteDagAction(String flowGroup, String flowName, String flowExe
       connection.commit();
       return result != 0;
     } catch (SQLException e) {
-      throw new IOException(String.format("Failure to delete action for table %s of flow with flow group:%s, flow name:%s and flow execution id:%s",
-          tableName, flowGroup, flowName, flowExecutionId), e);
+      throw new IOException(String.format("Failure to delete action for table %s of flow with flow group:%s, flow name:%s, flow execution id:%s and dagAction: %s",
+          tableName, flowGroup, flowName, flowExecutionId, dagActionValue), e);

Review Comment:
   tip: if you used `DagAction` as a POJO, you could use its `.toString()` to the same effect--w/o duplication across these messages.  e.g.:
   ```
   IOE(String.format("...of flow with %s", tableName, dagAction), e);
   ```



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -310,6 +316,17 @@ public void orchestrate(Spec spec) throws Exception {
         flowCompilationTimer.get().stop(flowMetadata);
       }
 
+      // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
+      if (this.isMultiActiveSchedulerEnabled) {
+        String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+        boolean leaseAttemptSucceeded = schedulerLeaseAlgoHandler.handleNewTriggerEvent(jobProps, flowGroup, flowName,
+            flowExecutionId, SchedulerLeaseDeterminationStore.FlowActionType.LAUNCH, triggerTimestampMillis);
+        _log.info("scheduler attempted lease on flowGroup: %s, flowName: %s, flowExecutionId: %s, LAUNCH event for "

Review Comment:
   nit: this is very conversational.  I find more regularized, even robotic text easier to parse and grok when "grepping" a large volume of log msgs.  suggestion:
   ```
   [flowGroup: 'foo-grp', flowName: 'bar-fn', execId: 1234] - multi-active LAUNCH(987654): FAILURE
   ```
   (where `987654` would be the trigger tstamp)
   
   later we might add other multi-active status messages, each with a type, like `CANCEL`, `RESUME`, `ADVANCE` (i.e. to the next job in the flow).  each could carry action-specific info in the parens next to it, which would discern exactly which instance it is  e.g.
   ```
   [g, n, id] - multi-active ADVANCE(<<job-name>>): SUCCESS
   ```
   
   looking ahead to verification and debugging, the end result is that message filtering becomes trivial and flexible on so many dimensions, including:
   * for a particular flow group+name
   * for a particular flow group+name+id
   * for a particular flow group+name+id+LAUNCH(tstamp) (e.g. across hosts)
   * related to multi-active in any way
   * related to a certain multi-active "action", such as `RESUME`
   * specifically those that are `FAILURE` or `SUCCESS`
   * etc.



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {

Review Comment:
   naming-wise: is this strictly limited to scheduling?
   
   anyway I'd encourage more evocative naming that suggests the underlying architectural role or pattern.  e.g. `MultiActiveLeaseArbiter`.  ("broker" and "controller" carry different design patterns specific connotations)



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -27,7 +27,9 @@
 public interface DagActionStore {
   enum DagActionValue {
     KILL,
-    RESUME
+    RESUME,
+    // TODO: potentially combine this enum with {@link SchedulerLeaseDeterminationStore.FlowActionType}
+    LAUNCH
   }
 
   @Getter

Review Comment:
   would it be equivalent to use the `@Data` lombok annotation, which would also synthesize the constructor?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java:
##########
@@ -120,13 +121,14 @@ public void addDagAction(String flowGroup, String flowName, String flowExecution
       insertStatement.executeUpdate();
       connection.commit();
     } catch (SQLException e) {
-      throw new IOException(String.format("Failure to adding action for table %s of flow with flow group:%s, flow name:%s and flow execution id:%s",
-          tableName, flowGroup, flowName, flowExecutionId), e);
+      throw new IOException(String.format("Failure to adding action for table %s of flow with flow group: %s, flow name"

Review Comment:
   nit: "to add" or just "adding"



##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitorFactory.java:
##########
@@ -40,12 +44,19 @@ public class DagActionStoreChangeMonitorFactory implements Provider<DagActionSto
   private final Config config;
   private DagActionStore dagActionStore;
   private DagManager dagManager;
+  private FlowCatalog flowCatalog;
+  private boolean isMultiActiveSchedulerEnabled;
 
   @Inject
-  public DagActionStoreChangeMonitorFactory(Config config, DagActionStore dagActionStore, DagManager dagManager) {
+  public DagActionStoreChangeMonitorFactory(Config config, DagActionStore dagActionStore, DagManager dagManager,
+      FlowCatalog flowCatalog, @Named(InjectionNames.WARM_STANDBY_ENABLED) boolean isMultiActiveSchedulerEnabled) {

Review Comment:
   is the mismatch in names intentional?  if so, please make sure there's a code comment somewhere, probably at the `InjectionNames` def



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java:
##########
@@ -70,21 +73,23 @@ public DagAction(String flowGroup, String flowName, String flowExecutionId, DagA
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException
    * @return true if we successfully delete one record, return false if the record does not exist
    */
-  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException;
+  boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue) throws IOException;
 
   /***
    * Retrieve action value by the flow group, flow name and flow execution id from the {@link DagActionStore}.
    * @param flowGroup flow group for the dag action
    * @param flowName flow name for the dag action
    * @param flowExecutionId flow execution for the dag action
+   * @param dagActionValue the value of the dag action
    * @throws IOException Exception in retrieving the {@link DagAction}.
    * @throws SpecNotFoundException If {@link DagAction} being retrieved is not present in store.
    */
-  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException, SpecNotFoundException,
-                                                                                           SQLException;
+  DagAction getDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue)

Review Comment:
   one thing I don't understand: a `DagAction` has exactly the four fields that are params to this method.  if so, does this method just duplicate `exists`?
   
   merely wondering: is there a use case for getting any and all actions related to a particular flow execution?
   
   relatedly w/ `deleteDagAction` (above), couldn't that take just a single param of type `DagAction`?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java:
##########
@@ -151,20 +153,21 @@ private DagAction getDagActionWithRetry(String flowGroup, String flowName, Strin
       getStatement.setString(++i, flowGroup);
       getStatement.setString(++i, flowName);
       getStatement.setString(++i, flowExecutionId);
+      getStatement.setString(++i, dagActionValue.toString());

Review Comment:
   tip: if you want to use `DagAction` as a value type/POJO, you could even add a method to encapsulate this repetitive setup:
   ```
   void parepareStatement(PreparedStatement ps, int nextVarIndex)
   ```



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java:
##########
@@ -440,7 +446,11 @@ public synchronized void scheduleJob(Properties jobProps, JobListener jobListene
   public void runJob(Properties jobProps, JobListener jobListener) throws JobException {
     try {
       Spec flowSpec = this.scheduledFlowSpecs.get(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
-      this.orchestrator.orchestrate(flowSpec);
+      String triggerTimestampMillis =
+          jobProps.containsKey(ConfigurationKeys.SCHEDULER_ORIGINAL_TRIGGER_TIMESTAMP_MILLIS_KEY)
+              ? jobProps.getProperty(ConfigurationKeys.SCHEDULER_ORIGINAL_TRIGGER_TIMESTAMP_MILLIS_KEY, "0L"):
+              jobProps.getProperty(ConfigurationKeys.SCHEDULER_TRIGGER_TIMESTAMP_MILLIS_KEY,"0L");
+      this.orchestrator.orchestrate(flowSpec, jobProps, Long.parseLong(triggerTimestampMillis));

Review Comment:
   not major, but seems worth raising the level of abstraction w/ a method like `extractTriggerTimestampMillis`



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -310,6 +316,17 @@ public void orchestrate(Spec spec) throws Exception {
         flowCompilationTimer.get().stop(flowMetadata);
       }
 
+      // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
+      if (this.isMultiActiveSchedulerEnabled) {
+        String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+        boolean leaseAttemptSucceeded = schedulerLeaseAlgoHandler.handleNewTriggerEvent(jobProps, flowGroup, flowName,
+            flowExecutionId, SchedulerLeaseDeterminationStore.FlowActionType.LAUNCH, triggerTimestampMillis);
+        _log.info("scheduler attempted lease on flowGroup: %s, flowName: %s, flowExecutionId: %s, LAUNCH event for "
+            + "triggerTimestamp: %s that was " + (leaseAttemptSucceeded ? "" : "NOT") + "successful", flowGroup,

Review Comment:
   `"NOT"` => `"NOT "`



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java:
##########
@@ -120,13 +121,14 @@ public void addDagAction(String flowGroup, String flowName, String flowExecution
       insertStatement.executeUpdate();
       connection.commit();
     } catch (SQLException e) {
-      throw new IOException(String.format("Failure to adding action for table %s of flow with flow group:%s, flow name:%s and flow execution id:%s",
-          tableName, flowGroup, flowName, flowExecutionId), e);
+      throw new IOException(String.format("Failure to adding action for table %s of flow with flow group: %s, flow name"
+              + ": %s, flow execution id: %s, and dag action: %s", tableName, flowGroup, flowName, flowExecutionId,
+          dagActionValue), e);
     }
   }
 
   @Override
-  public boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId) throws IOException {
+  public boolean deleteDagAction(String flowGroup, String flowName, String flowExecutionId, DagActionValue dagActionValue) throws IOException {
     try (Connection connection = this.dataSource.getConnection();
         PreparedStatement deleteStatement = connection.prepareStatement(String.format(DELETE_STATEMENT, tableName))) {
       int i = 0;

Review Comment:
   don't you need to add the `dagActionValue`, now that there's another "bind variable" in the statement (i.e. the fourth "?" now)?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID

Review Comment:
   I respect using an enum for the tri-state result, but the three I had imagined are more like:
   * LEASE_IS_YOURS / LEASE_OBTAINED (acquired)
   * LEASED_TO_ANOTHER / LEASE_NOT_AVAILABLE
   * NO_LONGER_LEASING / LEASE_NOT_POSSIBLE (not necessary / there's nothing to lease)
   
   did you have the same three concepts in mind, just named differently--or different idea here?



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID
+  }
+
+  // Action to take on a particular flow
+  enum FlowActionType {
+    LAUNCH,
+    RETRY,
+    CANCEL,
+    NEXT_HOP

Review Comment:
   nit: I mentioned `ADVANCE` elsewhere, but `NEXT_HOP` is fine too.  as for `RETRY`, I believe `RESUME` is the terminology we've adopted pretty widely--or do you find precendent for `RETRY`?



##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java:
##########
@@ -310,6 +316,17 @@ public void orchestrate(Spec spec) throws Exception {
         flowCompilationTimer.get().stop(flowMetadata);
       }
 
+      // If multi-active scheduler is enabled do not pass onto DagManager, otherwise scheduler forwards it directly
+      if (this.isMultiActiveSchedulerEnabled) {
+        String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+        boolean leaseAttemptSucceeded = schedulerLeaseAlgoHandler.handleNewTriggerEvent(jobProps, flowGroup, flowName,
+            flowExecutionId, SchedulerLeaseDeterminationStore.FlowActionType.LAUNCH, triggerTimestampMillis);
+        _log.info("scheduler attempted lease on flowGroup: %s, flowName: %s, flowExecutionId: %s, LAUNCH event for "
+            + "triggerTimestamp: %s that was " + (leaseAttemptSucceeded ? "" : "NOT") + "successful", flowGroup,
+            flowName, flowExecutionId, triggerTimestampMillis);
+        return;

Review Comment:
   especially as there's an `if/else` just after, why not connect this block to that, rather than adding a premature `return` here?  (these can be potentially more error prone for maintainers.)



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID
+  }
+
+  // Action to take on a particular flow
+  enum FlowActionType {
+    LAUNCH,
+    RETRY,
+    CANCEL,
+    NEXT_HOP
+  }
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow's trigger event if one does not already
+   * exist in the store for the same trigger event. Regardless of the outcome it also reads the pursuant timestamp of
+   * the entry for that trigger event (it could have pre-existed in the table or been newly added by the previous
+   * write). Based on the transaction results, it will return @LeaseAttemptStatus to determine the next action.
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param triggerTimeMillis is the time this flow is supposed to be launched
+   * @return LeaseAttemptStatus
+   * @throws IOException
+   */
+  LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,

Review Comment:
   seems misleading to name `GetPursuantTimestamp`, when that doesn't appear to be returned to the caller.  that said, won't the participant who fails to acquire the lease want to know when the lease was granted, to use that in deciding when to try again?
   
   nit: "pursuing" is slightly closer to the sense of what's happening here, than is "pursuant".  the former is also likely to be more familiar naming to maintainers.
   
   alternatively what about simply naming this `acquireLease` or `tryAcquireLease`?
   



##########
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SchedulerLeaseDeterminationStore.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.runtime.api;
+
+import java.io.IOException;
+import java.sql.Timestamp;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface defines the two basic actions required for lease determination for each FlowActionType event for a flow.
+ * It is used by the {@link SchedulerLeaseAlgoHandler} to allow multiple scheduler's on different hosts to determine
+ * which scheduler is tasked with ensuring the FlowAction is taken for the trigger.
+ */
+public interface SchedulerLeaseDeterminationStore {
+  static final Logger LOG = LoggerFactory.getLogger(SchedulerLeaseDeterminationStore.class);
+
+  // Enum is used to reason about the three possible scenarios that can result from an attempt to obtain a lease for a
+  // particular trigger event of a flow
+  enum LeaseAttemptStatus {
+    LEASE_OBTAINED,
+    PREVIOUS_LEASE_EXPIRED,
+    PREVIOUS_LEASE_VALID
+  }
+
+  // Action to take on a particular flow
+  enum FlowActionType {
+    LAUNCH,
+    RETRY,
+    CANCEL,
+    NEXT_HOP
+  }
+
+  /**
+   * This method attempts to insert an entry into store for a particular flow's trigger event if one does not already
+   * exist in the store for the same trigger event. Regardless of the outcome it also reads the pursuant timestamp of
+   * the entry for that trigger event (it could have pre-existed in the table or been newly added by the previous
+   * write). Based on the transaction results, it will return @LeaseAttemptStatus to determine the next action.
+   * @param flowGroup
+   * @param flowName
+   * @param flowExecutionId
+   * @param triggerTimeMillis is the time this flow is supposed to be launched
+   * @return LeaseAttemptStatus
+   * @throws IOException
+   */
+  LeaseAttemptStatus attemptInsertAndGetPursuantTimestamp(String flowGroup, String flowName,
+      String flowExecutionId, FlowActionType flowActionType, long triggerTimeMillis) throws IOException;
+
+  /**
+   * This method is used by `attemptInsertAndGetPursuantTimestamp` above to indicate the host has successfully completed

Review Comment:
   not having read further yet, the description here makes this sound like an internal impl detail.  does it need to be part of the public interface?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org