From 1ebc7014ae8ab33a8149441e0daeaaec038fa4a4 Mon Sep 17 00:00:00 2001 From: Meeth Gala Date: Thu, 31 Aug 2023 09:34:05 -0700 Subject: [PATCH] initial commit for refactoring dag manager --- .../gobblin/service/ServiceConfigKeys.java | 8 +- .../gobblin/service/ServiceConstants.java | 31 ++ .../gobblin/service/FlowConfigTest.java | 2 +- .../gobblin/service/FlowConfigV2Test.java | 2 +- .../gobblin/service/FlowConfigsResource.java | 2 +- .../service/FlowConfigsV2Resource.java | 2 +- .../service/FlowExecutionResource.java | 2 +- .../gobblin/service/FlowStatusResource.java | 2 +- .../gobblin/runtime/api/DagActionStore.java | 9 +- .../runtime/api/MultiActiveLeaseArbiter.java | 4 +- .../api/MysqlMultiActiveLeaseArbiter.java | 7 +- .../dag_action_store/MysqlDagActionStore.java | 23 +- .../runtime/spec_catalog/FlowCatalog.java | 5 +- .../monitoring/JobStatusRetriever.java | 67 +-- .../api/MysqlMultiActiveLeaseArbiterTest.java | 4 +- .../MysqlDagActionStoreTest.java | 6 +- .../runtime/spec_catalog/FlowCatalogTest.java | 4 +- ...ollerUserDefinedMessageHandlerFactory.java | 8 +- .../core/GobblinServiceConfiguration.java | 5 + .../core/GobblinServiceGuiceModule.java | 17 +- .../service/modules/flowgraph/Dag.java | 35 ++ .../modules/orchestration/DagManagement.java | 52 +++ .../DagManagementStateStore.java | 66 +++ .../modules/orchestration/DagManager.java | 17 +- .../orchestration/DagManagerUtils.java | 41 +- .../modules/orchestration/DagProcFactory.java | 115 +++++ .../orchestration/DagProcessingEngine.java | 123 ++++++ .../modules/orchestration/DagTaskStream.java | 141 +++++++ .../orchestration/FlowTriggerHandler.java | 63 ++- .../InMemoryDagManagementStateStore.java | 145 +++++++ .../modules/orchestration/KillDagThread.java | 169 ++++++++ .../modules/orchestration/NewDagManager.java | 398 ++++++++++++++++++ .../modules/orchestration/Orchestrator.java | 68 +-- .../orchestration/TimingEventUtils.java | 2 +- .../orchestration/proc/AdvanceDagProc.java | 139 ++++++ .../orchestration/proc/CleanUpDagProc.java | 51 +++ .../modules/orchestration/proc/DagProc.java | 77 ++++ .../orchestration/proc/KillDagProc.java | 108 +++++ .../orchestration/proc/LaunchDagProc.java | 248 +++++++++++ .../orchestration/proc/ResumeDagProc.java | 49 +++ .../orchestration/proc/RetryDagProc.java | 127 ++++++ .../orchestration/task/AdvanceDagTask.java | 44 ++ .../orchestration/task/CleanUpDagTask.java | 39 ++ .../modules/orchestration/task/DagTask.java | 53 +++ .../orchestration/task/KillDagTask.java | 40 ++ .../orchestration/task/LaunchDagTask.java | 41 ++ .../orchestration/task/ResumeDagTask.java | 35 ++ .../orchestration/task/RetryDagTask.java | 41 ++ ...bblinServiceFlowConfigResourceHandler.java | 15 +- ...ecutionResourceHandlerWithWarmStandby.java | 9 +- .../scheduler/GobblinServiceJobScheduler.java | 5 +- .../monitoring/ChangeMonitorUtils.java | 4 +- .../DagActionStoreChangeMonitor.java | 108 +++-- .../monitoring/KafkaAvroJobStatusMonitor.java | 7 +- .../monitoring/KafkaJobStatusMonitor.java | 15 +- .../KafkaJobStatusMonitorFactory.java | 12 +- .../monitoring/MysqlJobStatusRetriever.java | 5 +- .../monitoring/SpecStoreChangeMonitor.java | 23 +- .../monitoring/event/JobStatusEvent.java | 49 +++ .../orchestration/DagManagerFlowTest.java | 4 +- .../orchestration/FlowTriggerHandlerTest.java | 2 +- .../orchestration/OrchestratorTest.java | 4 +- .../GobblinServiceJobSchedulerTest.java | 8 +- .../monitoring/GitConfigMonitorTest.java | 4 +- 64 files changed, 2758 insertions(+), 253 deletions(-) create mode 100644 gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConstants.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagement.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcFactory.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagTaskStream.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/InMemoryDagManagementStateStore.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/KillDagThread.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/AdvanceDagProc.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/CleanUpDagProc.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProc.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/KillDagProc.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/LaunchDagProc.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ResumeDagProc.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/RetryDagProc.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/AdvanceDagTask.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/CleanUpDagTask.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/KillDagTask.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/LaunchDagTask.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/ResumeDagTask.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/RetryDagTask.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/event/JobStatusEvent.java diff --git a/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java b/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java index 21b32b58cc0..b5a379588c6 100644 --- a/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java +++ b/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java @@ -25,8 +25,6 @@ public class ServiceConfigKeys { public static final String GOBBLIN_SERVICE_PREFIX = "gobblin.service."; - public static final String GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS = "org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler"; - public static final String GOBBLIN_ORCHESTRATOR_LISTENER_CLASS = "org.apache.gobblin.service.modules.orchestration.Orchestrator"; // Gobblin Service Manager Keys public static final String GOBBLIN_SERVICE_TOPOLOGY_CATALOG_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "topologyCatalog.enabled"; @@ -41,6 +39,7 @@ public class ServiceConfigKeys { public static final boolean DEFAULT_GOBBLIN_SERVICE_DAG_MANAGER_ENABLED = false; public static final String GOBBLIN_SERVICE_JOB_STATUS_MONITOR_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "jobStatusMonitor.enabled"; public static final String GOBBLIN_SERVICE_WARM_STANDBY_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "warmStandby.enabled"; + public static final String GOBBLIN_SERVICE_MULTI_ACTIVE_DAG_MANAGER_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "multiActiveDagManager.enabled"; public static final String GOBBLIN_SERVICE_MULTI_ACTIVE_SCHEDULER_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "multiActiveScheduler.enabled"; // If true, will mark up/down d2 servers on leadership so that all requests will be routed to the leader node public static final String GOBBLIN_SERVICE_D2_ONLY_ANNOUNCE_LEADER = GOBBLIN_SERVICE_PREFIX + "d2.onlyAnnounceLeader"; @@ -54,11 +53,6 @@ public class ServiceConfigKeys { public static final String GOBBLIN_SERVICE_FLOWGRAPH_CLASS_KEY = GOBBLIN_SERVICE_PREFIX + "flowGraph.class"; public static final String GOBBLIN_SERVICE_FLOWGRAPH_HELPER_KEY = GOBBLIN_SERVICE_PREFIX + "flowGraphHelper.class"; - // Helix message sub types for FlowSpec - public static final String HELIX_FLOWSPEC_ADD = "FLOWSPEC_ADD"; - public static final String HELIX_FLOWSPEC_REMOVE = "FLOWSPEC_REMOVE"; - public static final String HELIX_FLOWSPEC_UPDATE = "FLOWSPEC_UPDATE"; - // Flow Compiler Keys public static final String GOBBLIN_SERVICE_FLOWCOMPILER_CLASS_KEY = GOBBLIN_SERVICE_PREFIX + "flowCompiler.class"; public static final String COMPILATION_SUCCESSFUL = "compilation.successful"; diff --git a/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConstants.java b/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConstants.java new file mode 100644 index 00000000000..4324ccee334 --- /dev/null +++ b/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConstants.java @@ -0,0 +1,31 @@ +/* + * 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; + +public class ServiceConstants { + public static final String SERVICE_NAMESPACE = "org.apache.gobblin.service"; + public static final String GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS = + "org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler"; + public static final String GOBBLIN_ORCHESTRATOR_LISTENER_CLASS = + "org.apache.gobblin.service.modules.orchestration.Orchestrator"; + + // Helix message sub types for FlowSpec + public static final String HELIX_FLOWSPEC_ADD = "FLOWSPEC_ADD"; + public static final String HELIX_FLOWSPEC_REMOVE = "FLOWSPEC_REMOVE"; + public static final String HELIX_FLOWSPEC_UPDATE = "FLOWSPEC_UPDATE"; +} diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigTest.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigTest.java index 9cd040f4451..e7e71f56c7d 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigTest.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigTest.java @@ -83,7 +83,7 @@ public void setUp() throws Exception { Config config = configBuilder.build(); final FlowCatalog flowCatalog = new FlowCatalog(config); final SpecCatalogListener mockListener = mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); + when(mockListener.getName()).thenReturn(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse("")); flowCatalog.addListener(mockListener); flowCatalog.startAsync(); diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java index b205ab244d5..cd2fabddcd5 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java @@ -109,7 +109,7 @@ public void setUp() throws Exception { Config config = configBuilder.build(); final FlowCatalog flowCatalog = new FlowCatalog(config); final SpecCatalogListener mockListener = mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); + when(mockListener.getName()).thenReturn(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); // NOTE: more general `ArgumentMatchers` (indicating compilation unsuccessful) must precede the specific when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse(null)); when(mockListener.onAddSpec(ArgumentMatchers.argThat((FlowSpec flowSpec) -> { diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResource.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResource.java index 95c0d37a9e9..381ea28e274 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResource.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResource.java @@ -41,7 +41,7 @@ /** * Resource for handling flow configuration requests */ -@RestLiCollection(name = "flowconfigs", namespace = "org.apache.gobblin.service", keyName = "id") +@RestLiCollection(name = "flowconfigs", namespace = ServiceConstants.SERVICE_NAMESPACE, keyName = "id") public class FlowConfigsResource extends ComplexKeyResourceTemplate { private static final Logger LOG = LoggerFactory.getLogger(FlowConfigsResource.class); diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java index a679d768c7c..80c676704dc 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java @@ -63,7 +63,7 @@ * Resource for handling flow configuration requests */ @Slf4j -@RestLiCollection(name = "flowconfigsV2", namespace = "org.apache.gobblin.service", keyName = "id") +@RestLiCollection(name = "flowconfigsV2", namespace = ServiceConstants.SERVICE_NAMESPACE, keyName = "id") public class FlowConfigsV2Resource extends ComplexKeyResourceTemplate { private static final Logger LOG = LoggerFactory.getLogger(FlowConfigsV2Resource.class); public static final String INJECT_READY_TO_USE = "v2ReadyToUse"; diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java index 0f276b5ba97..b52889c7745 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java @@ -39,7 +39,7 @@ /** * Resource for handling flow execution requests */ -@RestLiCollection(name = "flowexecutions", namespace = "org.apache.gobblin.service", keyName = "id") +@RestLiCollection(name = "flowexecutions", namespace = ServiceConstants.SERVICE_NAMESPACE, keyName = "id") public class FlowExecutionResource extends ComplexKeyResourceTemplate { @Inject diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowStatusResource.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowStatusResource.java index 483648f7905..9257a108688 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowStatusResource.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowStatusResource.java @@ -38,7 +38,7 @@ /** * Resource for handling flow status requests */ -@RestLiCollection(name = "flowstatuses", namespace = "org.apache.gobblin.service", keyName = "id") +@RestLiCollection(name = "flowstatuses", namespace = ServiceConstants.SERVICE_NAMESPACE, keyName = "id") public class FlowStatusResource extends ComplexKeyResourceTemplate { public static final String MESSAGE_SEPARATOR = ", "; diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java index 4f3442597fb..3260373d94a 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/DagActionStore.java @@ -39,7 +39,7 @@ enum FlowActionType { class DagAction { final String flowGroup; final String flowName; - final String flowExecutionId; + final long flowExecutionId; final FlowActionType flowActionType; public FlowId getFlowId() { @@ -50,8 +50,7 @@ public FlowId getFlowId() { * Replace flow execution id with agreed upon event time to easily track the flow */ public DagAction updateFlowExecutionId(long eventTimeMillis) { - return new DagAction(this.getFlowGroup(), this.getFlowName(), - String.valueOf(eventTimeMillis), this.getFlowActionType()); + return new DagAction(this.getFlowGroup(), this.getFlowName(), eventTimeMillis, this.getFlowActionType()); } } @@ -64,7 +63,7 @@ public DagAction updateFlowExecutionId(long eventTimeMillis) { * @param flowActionType the value of the dag action * @throws IOException */ - boolean exists(String flowGroup, String flowName, String flowExecutionId, FlowActionType flowActionType) throws IOException, SQLException; + boolean exists(String flowGroup, String flowName, long flowExecutionId, FlowActionType flowActionType) throws IOException, SQLException; /** * Persist the dag action in {@link DagActionStore} for durability @@ -74,7 +73,7 @@ public DagAction updateFlowExecutionId(long eventTimeMillis) { * @param flowActionType the value of the dag action * @throws IOException */ - void addDagAction(String flowGroup, String flowName, String flowExecutionId, FlowActionType flowActionType) throws IOException; + void addDagAction(String flowGroup, String flowName, long flowExecutionId, FlowActionType flowActionType) throws IOException; /** * delete the dag action from {@link DagActionStore} diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java index 253db49ba92..17a264e6d1b 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MultiActiveLeaseArbiter.java @@ -92,7 +92,7 @@ class LeaseObtainedStatus extends LeaseAttemptStatus { * @return event time in millis since epoch for the event of this lease acquisition */ public long getEventTimeMillis() { - return Long.parseLong(flowAction.getFlowExecutionId()); + return flowAction.getFlowExecutionId(); } } @@ -113,7 +113,7 @@ class LeasedToAnotherStatus extends LeaseAttemptStatus { * @return */ public long getEventTimeMillis() { - return Long.parseLong(flowAction.getFlowExecutionId()); + return flowAction.getFlowExecutionId(); } } } diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java index 05449767cf1..aac8e5aa21f 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiter.java @@ -434,13 +434,8 @@ protected SelectInfoResult getRowInfo(DagActionStore.DagAction flowAction) throw return dbStatementExecutor.withPreparedStatement(thisTableSelectAfterInsertStatement, selectStatement -> { completeWhereClauseMatchingKeyPreparedStatement(selectStatement, flowAction); - ResultSet resultSet = selectStatement.executeQuery(); - try { + try (ResultSet resultSet = selectStatement.executeQuery()) { return createSelectInfoResult(resultSet); - } finally { - if (resultSet != null) { - resultSet.close(); - } } }, true); } diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java index 894d0a3004c..b6b97d016ea 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStore.java @@ -97,12 +97,13 @@ public MysqlDagActionStore(Config config) throws IOException { } @Override - public boolean exists(String flowGroup, String flowName, String flowExecutionId, FlowActionType flowActionType) throws IOException, SQLException { - return dbStatementExecutor.withPreparedStatement(String.format(EXISTS_STATEMENT, tableName), existStatement -> { + public boolean exists(String flowGroup, String flowName, long flowExecutionId, FlowActionType flowActionType) throws IOException, SQLException { + try (Connection connection = this.dataSource.getConnection(); + PreparedStatement existStatement = connection.prepareStatement(String.format(EXISTS_STATEMENT, tableName))) { int i = 0; existStatement.setString(++i, flowGroup); existStatement.setString(++i, flowName); - existStatement.setString(++i, flowExecutionId); + existStatement.setString(++i, String.valueOf(flowExecutionId)); existStatement.setString(++i, flowActionType.toString()); ResultSet rs = null; try { @@ -117,18 +118,18 @@ public boolean exists(String flowGroup, String flowName, String flowExecutionId, rs.close(); } } - }, true); + } } @Override - public void addDagAction(String flowGroup, String flowName, String flowExecutionId, FlowActionType flowActionType) + public void addDagAction(String flowGroup, String flowName, long flowExecutionId, FlowActionType flowActionType) throws IOException { dbStatementExecutor.withPreparedStatement(String.format(INSERT_STATEMENT, tableName), insertStatement -> { try { int i = 0; insertStatement.setString(++i, flowGroup); insertStatement.setString(++i, flowName); - insertStatement.setString(++i, flowExecutionId); + insertStatement.setString(++i, String.valueOf(flowExecutionId)); insertStatement.setString(++i, flowActionType.toString()); return insertStatement.executeUpdate(); } catch (SQLException e) { @@ -144,7 +145,7 @@ public boolean deleteDagAction(DagAction dagAction) throws IOException { int i = 0; deleteStatement.setString(++i, dagAction.getFlowGroup()); deleteStatement.setString(++i, dagAction.getFlowName()); - deleteStatement.setString(++i, dagAction.getFlowExecutionId()); + deleteStatement.setString(++i, String.valueOf(dagAction.getFlowExecutionId())); deleteStatement.setString(++i, dagAction.getFlowActionType().toString()); int result = deleteStatement.executeUpdate(); return result != 0; @@ -155,17 +156,17 @@ public boolean deleteDagAction(DagAction dagAction) throws IOException { } // TODO: later change this to getDagActions relating to a particular flow execution if it makes sense - private DagAction getDagActionWithRetry(String flowGroup, String flowName, String flowExecutionId, FlowActionType flowActionType, ExponentialBackoff exponentialBackoff) + private DagAction getDagActionWithRetry(String flowGroup, String flowName, long flowExecutionId, FlowActionType flowActionType, ExponentialBackoff exponentialBackoff) throws IOException, SQLException { return dbStatementExecutor.withPreparedStatement(String.format(GET_STATEMENT, tableName), getStatement -> { int i = 0; getStatement.setString(++i, flowGroup); getStatement.setString(++i, flowName); - getStatement.setString(++i, flowExecutionId); + getStatement.setString(++i, String.valueOf(flowExecutionId)); getStatement.setString(++i, flowActionType.toString()); try (ResultSet rs = getStatement.executeQuery()) { if (rs.next()) { - return new DagAction(rs.getString(1), rs.getString(2), rs.getString(3), FlowActionType.valueOf(rs.getString(4))); + return new DagAction(rs.getString(1), rs.getString(2), Long.parseLong(rs.getString(3)), FlowActionType.valueOf(rs.getString(4))); } else if (exponentialBackoff.awaitNextRetryIfAvailable()) { return getDagActionWithRetry(flowGroup, flowName, flowExecutionId, flowActionType, exponentialBackoff); } else { @@ -186,7 +187,7 @@ public Collection getDagActions() throws IOException { HashSet result = new HashSet<>(); try (ResultSet rs = getAllStatement.executeQuery()) { while (rs.next()) { - result.add(new DagAction(rs.getString(1), rs.getString(2), rs.getString(3), FlowActionType.valueOf(rs.getString(4)))); + result.add(new DagAction(rs.getString(1), rs.getString(2), Long.parseLong(rs.getString(3)), FlowActionType.valueOf(rs.getString(4)))); } return result; } catch (SQLException e) { diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java index 0fe6f2583e1..ee53fb454a6 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java @@ -32,6 +32,7 @@ import org.apache.commons.lang3.reflect.ConstructorUtils; import org.apache.gobblin.configuration.ConfigurationKeys; import org.apache.gobblin.runtime.util.InjectionNames; +import org.apache.gobblin.service.ServiceConstants; import org.apache.gobblin.util.ExponentialBackoff; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -421,10 +422,10 @@ private Map updateOrAddSpecHelper(Spec spec, boolean tr } AddSpecResponse compileResponse; if (isWarmStandbyEnabled) { - compileResponse = responseMap.getOrDefault(ServiceConfigKeys.GOBBLIN_ORCHESTRATOR_LISTENER_CLASS, new AddSpecResponse<>(null)); + compileResponse = responseMap.getOrDefault(ServiceConstants.GOBBLIN_ORCHESTRATOR_LISTENER_CLASS, new AddSpecResponse<>(null)); //todo: do we check quota here? or in compiler? Quota manager need dag to check quota which is not accessable from this class } else { - compileResponse = responseMap.getOrDefault(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS, new AddSpecResponse<>(null)); + compileResponse = responseMap.getOrDefault(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS, new AddSpecResponse<>(null)); } responseMap.put(ServiceConfigKeys.COMPILATION_RESPONSE, compileResponse); diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/service/monitoring/JobStatusRetriever.java b/gobblin-runtime/src/main/java/org/apache/gobblin/service/monitoring/JobStatusRetriever.java index e5845fe373e..c4993abf767 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/service/monitoring/JobStatusRetriever.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/service/monitoring/JobStatusRetriever.java @@ -108,6 +108,26 @@ public Iterator getLatestJobStatusByFlowNameAndGroup(String flowName, * @return deserialize {@link State} into a {@link JobStatus}. */ protected JobStatus getJobStatus(State jobState) { + JobStatus.JobStatusBuilder jobStatusBuilder = createJobStatusBuilderFromState(jobState); + + String contextId = TroubleshooterUtils.getContextIdForJob(jobState.getProperties()); + + Supplier> jobIssues = Suppliers.memoize(() -> { + List issues; + try { + issues = issueRepository.getAll(contextId); + } catch (TroubleshooterException e) { + log.warn("Cannot retrieve job issues", e); + issues = Collections.emptyList(); + } + return issues; + }); + + jobStatusBuilder.issues(jobIssues); + return jobStatusBuilder.build(); + } + + public static JobStatus.JobStatusBuilder createJobStatusBuilderFromState(State jobState) { String flowGroup = getFlowGroup(jobState); String flowName = getFlowName(jobState); long flowExecutionId = getFlowExecutionId(jobState); @@ -130,48 +150,35 @@ protected JobStatus getJobStatus(State jobState) { int progressPercentage = jobState.getPropAsInt(TimingEvent.JOB_COMPLETION_PERCENTAGE, 0); long lastProgressEventTime = jobState.getPropAsLong(TimingEvent.JOB_LAST_PROGRESS_EVENT_TIME, 0); - String contextId = TroubleshooterUtils.getContextIdForJob(jobState.getProperties()); - - Supplier> jobIssues = Suppliers.memoize(() -> { - List issues; - try { - issues = issueRepository.getAll(contextId); - } catch (TroubleshooterException e) { - log.warn("Cannot retrieve job issues", e); - issues = Collections.emptyList(); - } - return issues; - }); - - return JobStatus.builder().flowName(flowName).flowGroup(flowGroup).flowExecutionId(flowExecutionId). - jobName(jobName).jobGroup(jobGroup).jobTag(jobTag).jobExecutionId(jobExecutionId).eventName(eventName). - lowWatermark(lowWatermark).highWatermark(highWatermark).orchestratedTime(orchestratedTime).startTime(startTime).endTime(endTime). - message(message).processedCount(processedCount).maxAttempts(maxAttempts).currentAttempts(currentAttempts).currentGeneration(currentGeneration). - shouldRetry(shouldRetry).progressPercentage(progressPercentage).lastProgressEventTime(lastProgressEventTime). - issues(jobIssues).build(); + return JobStatus.builder().flowName(flowName).flowGroup(flowGroup).flowExecutionId(flowExecutionId).jobName(jobName) + .jobGroup(jobGroup).jobTag(jobTag).jobExecutionId(jobExecutionId).eventName(eventName).lowWatermark(lowWatermark) + .highWatermark(highWatermark).orchestratedTime(orchestratedTime).startTime(startTime).endTime(endTime) + .message(message).processedCount(processedCount).maxAttempts(maxAttempts).currentAttempts(currentAttempts) + .currentGeneration(currentGeneration).shouldRetry(shouldRetry).progressPercentage(progressPercentage) + .lastProgressEventTime(lastProgressEventTime); } - protected final String getFlowGroup(State jobState) { + protected static final String getFlowGroup(State jobState) { return jobState.getProp(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD); } - protected final String getFlowName(State jobState) { + protected static final String getFlowName(State jobState) { return jobState.getProp(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD); } - protected final long getFlowExecutionId(State jobState) { + protected static final long getFlowExecutionId(State jobState) { return Long.parseLong(jobState.getProp(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD)); } - protected final String getJobGroup(State jobState) { + protected static final String getJobGroup(State jobState) { return jobState.getProp(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD); } - protected final String getJobName(State jobState) { + protected static final String getJobName(State jobState) { return jobState.getProp(TimingEvent.FlowEventConstants.JOB_NAME_FIELD); } - protected final long getJobExecutionId(State jobState) { + protected static final long getJobExecutionId(State jobState) { return Long.parseLong(jobState.getProp(TimingEvent.FlowEventConstants.JOB_EXECUTION_ID_FIELD, "0")); } @@ -183,7 +190,9 @@ protected List asFlowStatuses(List fl return flowExecutionGroupings.stream().map(exec -> { List jobStatuses = ImmutableList.copyOf(asJobStatuses(exec.getJobStates().stream().sorted( // rationalized order, to facilitate test assertions - Comparator.comparing(this::getJobGroup).thenComparing(this::getJobName).thenComparing(this::getJobExecutionId) + Comparator.comparing(JobStatusRetriever::getJobGroup) + .thenComparing(JobStatusRetriever::getJobName) + .thenComparing(JobStatusRetriever::getJobExecutionId) ).collect(Collectors.toList()))); return new FlowStatus(exec.getFlowName(), exec.getFlowGroup(), exec.getFlowExecutionId(), jobStatuses.iterator(), getFlowStatusFromJobStatuses(dagManagerEnabled, jobStatuses.iterator())); @@ -201,10 +210,8 @@ protected static class FlowExecutionJobStateGrouping { protected List groupByFlowExecutionAndRetainLatest( String flowGroup, List jobStatusStates, int maxCountPerFlowName) { - Map>> statesByFlowExecutionIdByName = - jobStatusStates.stream().collect(Collectors.groupingBy( - this::getFlowName, - Collectors.groupingBy(this::getFlowExecutionId))); + Map>> statesByFlowExecutionIdByName = jobStatusStates.stream().collect( + Collectors.groupingBy(JobStatusRetriever::getFlowName, Collectors.groupingBy(JobStatusRetriever::getFlowExecutionId))); return statesByFlowExecutionIdByName.entrySet().stream().sorted(Map.Entry.comparingByKey()).flatMap(flowNameEntry -> { String flowName = flowNameEntry.getKey(); diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiterTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiterTest.java index 08630ab3661..3399cb6098a 100644 --- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiterTest.java +++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/api/MysqlMultiActiveLeaseArbiterTest.java @@ -43,7 +43,7 @@ public class MysqlMultiActiveLeaseArbiterTest { private static final String TABLE = "mysql_multi_active_lease_arbiter_store"; private static final String flowGroup = "testFlowGroup"; private static final String flowName = "testFlowName"; - private static final String flowExecutionId = "12345677"; + private static final long flowExecutionId = 12345677L; // The following are considered unique because they correspond to different flow action types private static DagActionStore.DagAction launchDagAction = new DagActionStore.DagAction(flowGroup, flowName, flowExecutionId, DagActionStore.FlowActionType.LAUNCH); @@ -91,7 +91,7 @@ public void testAcquireLeaseSingleParticipant() throws Exception { Assert.assertTrue(firstObtainedStatus.getEventTimeMillis() <= firstObtainedStatus.getLeaseAcquisitionTimestamp()); Assert.assertTrue(firstObtainedStatus.getFlowAction().equals( - new DagActionStore.DagAction(flowGroup, flowName, String.valueOf(firstObtainedStatus.getEventTimeMillis()), + new DagActionStore.DagAction(flowGroup, flowName, firstObtainedStatus.getEventTimeMillis(), DagActionStore.FlowActionType.LAUNCH))); // Verify that different DagAction types for the same flow can have leases at the same time diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStoreTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStoreTest.java index 255dd07898f..fdd9f6e288a 100644 --- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStoreTest.java +++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/dag_action_store/MysqlDagActionStoreTest.java @@ -40,9 +40,9 @@ public class MysqlDagActionStoreTest { private static final String TABLE = "dag_action_store"; private static final String flowGroup = "testFlowGroup"; private static final String flowName = "testFlowName"; - private static final String flowExecutionId = "12345677"; - private static final String flowExecutionId_2 = "12345678"; - private static final String flowExecutionId_3 = "12345679"; + private static final long flowExecutionId = 12345677L; + private static final long flowExecutionId_2 = 12345678L; + private static final long flowExecutionId_3 = 12345679L; private MysqlDagActionStore mysqlDagActionStore; @BeforeClass diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalogTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalogTest.java index e17fc04ebcb..f8148479cda 100644 --- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalogTest.java +++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalogTest.java @@ -37,7 +37,7 @@ import org.apache.gobblin.runtime.api.SpecNotFoundException; import org.apache.gobblin.runtime.app.ServiceBasedAppLauncher; import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor; -import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; import org.apache.gobblin.util.ConfigUtils; import org.apache.gobblin.util.PathUtils; import org.apache.hadoop.fs.Path; @@ -82,7 +82,7 @@ public void setup() throws Exception { Optional.of(logger)); this.mockListener = mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); + when(mockListener.getName()).thenReturn(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse("")); this.flowCatalog.addListener(mockListener); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/ControllerUserDefinedMessageHandlerFactory.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/ControllerUserDefinedMessageHandlerFactory.java index 6e9cff7828d..28f4c3076d4 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/ControllerUserDefinedMessageHandlerFactory.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/ControllerUserDefinedMessageHandlerFactory.java @@ -39,7 +39,7 @@ import org.apache.gobblin.service.FlowConfigResourceLocalHandler; import org.apache.gobblin.service.FlowConfigsResourceHandler; import org.apache.gobblin.service.FlowId; -import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; import org.apache.gobblin.service.modules.restli.FlowConfigUtils; import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; @@ -171,11 +171,11 @@ public HelixTaskResult handleMessage() String msg = _message.getAttribute(Message.Attributes.INNER_MESSAGE); log.info("{} ControllerUserDefinedMessage received : {}, type {}", this.serviceName, msg, _message.getMsgSubType()); try { - if (_message.getMsgSubType().equals(ServiceConfigKeys.HELIX_FLOWSPEC_ADD)) { + if (_message.getMsgSubType().equals(ServiceConstants.HELIX_FLOWSPEC_ADD)) { handleAdd(msg); - } else if (_message.getMsgSubType().equals(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE)) { + } else if (_message.getMsgSubType().equals(ServiceConstants.HELIX_FLOWSPEC_REMOVE)) { handleDelete(msg); - } else if (_message.getMsgSubType().equals(ServiceConfigKeys.HELIX_FLOWSPEC_UPDATE)) { + } else if (_message.getMsgSubType().equals(ServiceConstants.HELIX_FLOWSPEC_UPDATE)) { handleUpdate(msg); } } catch (IOException e) { diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceConfiguration.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceConfiguration.java index 03081b3ba2d..ba5668522cd 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceConfiguration.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceConfiguration.java @@ -28,6 +28,7 @@ import lombok.ToString; import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.modules.orchestration.DagProcessingEngine; import org.apache.gobblin.util.ConfigUtils; @@ -79,6 +80,9 @@ public class GobblinServiceConfiguration { @Getter private final boolean onlyAnnounceLeader; + @Getter + private final boolean isDagProcessingEngineEnabled; + @Getter private final Config innerConfig; @@ -124,5 +128,6 @@ public GobblinServiceConfiguration(String serviceName, String serviceId, Config this.isTopologySpecFactoryEnabled = ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_TOPOLOGY_SPEC_FACTORY_ENABLED_KEY, true); this.onlyAnnounceLeader = ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_D2_ONLY_ANNOUNCE_LEADER, false); + this.isDagProcessingEngineEnabled = ConfigUtils.getBoolean(config, DagProcessingEngine.GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX, false); } } diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceGuiceModule.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceGuiceModule.java index af0c3461adf..6448c455940 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceGuiceModule.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceGuiceModule.java @@ -23,7 +23,12 @@ import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter; import org.apache.gobblin.runtime.api.MysqlMultiActiveLeaseArbiter; import org.apache.gobblin.runtime.dag_action_store.MysqlDagActionStore; +import org.apache.gobblin.service.modules.orchestration.DagManagementStateStore; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.DagProcessingEngine; +import org.apache.gobblin.service.modules.orchestration.DagTaskStream; import org.apache.gobblin.service.modules.orchestration.FlowTriggerHandler; +import org.apache.gobblin.service.modules.orchestration.InMemoryDagManagementStateStore; import org.apache.gobblin.service.modules.orchestration.UserQuotaManager; import org.apache.gobblin.service.modules.restli.GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby; import org.apache.gobblin.service.modules.restli.GobblinServiceFlowExecutionResourceHandlerWithWarmStandby; @@ -168,16 +173,26 @@ public void configure(Binder binder) { OptionalBinder.newOptionalBinder(binder, MultiActiveLeaseArbiter.class); OptionalBinder.newOptionalBinder(binder, FlowTriggerHandler.class); + OptionalBinder.newOptionalBinder(binder, DagProcFactory.class); + OptionalBinder.newOptionalBinder(binder, DagProcessingEngine.class); + OptionalBinder.newOptionalBinder(binder, DagManagementStateStore.class); + OptionalBinder.newOptionalBinder(binder, DagTaskStream.class); + if (serviceConfig.isMultiActiveSchedulerEnabled()) { binder.bind(MultiActiveLeaseArbiter.class).to(MysqlMultiActiveLeaseArbiter.class); binder.bind(FlowTriggerHandler.class); + if(serviceConfig.isDagProcessingEngineEnabled()) { + binder.bind(DagManagementStateStore.class).to(InMemoryDagManagementStateStore.class); + binder.bind(DagProcFactory.class).in(Singleton.class); + binder.bind(DagProcessingEngine.class).in(Singleton.class); + binder.bind(DagTaskStream.class).in(Singleton.class); + } } binder.bind(FlowConfigsResource.class); binder.bind(FlowConfigsV2Resource.class); binder.bind(FlowStatusResource.class); binder.bind(FlowExecutionResource.class); - binder.bind(FlowConfigResourceLocalHandler.class); binder.bind(FlowConfigV2ResourceLocalHandler.class); binder.bind(FlowExecutionResourceLocalHandler.class); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flowgraph/Dag.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flowgraph/Dag.java index edb19d8d145..dfac181e1eb 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flowgraph/Dag.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flowgraph/Dag.java @@ -27,13 +27,20 @@ import java.util.Set; import java.util.stream.Collectors; +import com.google.common.base.Joiner; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.typesafe.config.Config; +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.Setter; import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + /** * An implementation of Dag. Assumes that nodes have unique values. Nodes with duplicate values will produce @@ -255,10 +262,12 @@ public static class DagNode { private T value; //List of parent Nodes that are dependencies of this Node. private List> parentNodes; + private DagNodeId id; //Constructor public DagNode(T value) { this.value = value; + this.id = createId(((JobExecutionPlan) this.getValue()).getJobSpec().getConfig()); } public void addParentNode(DagNode node) { @@ -285,6 +294,32 @@ public boolean equals(Object o) { public int hashCode() { return this.getValue().hashCode(); } + + private static DagNodeId createId(Config jobConfig) { + String flowGroup = jobConfig.getString(ConfigurationKeys.FLOW_GROUP_KEY); + String flowName =jobConfig.getString(ConfigurationKeys.FLOW_NAME_KEY); + long flowExecutionId = jobConfig.getLong(ConfigurationKeys.FLOW_EXECUTION_ID_KEY); + String jobName = jobConfig.getString(ConfigurationKeys.JOB_NAME_KEY); + String jobGroup = jobConfig.getString(ConfigurationKeys.JOB_GROUP_KEY); + + return new DagNodeId(flowGroup, flowName, flowExecutionId, jobGroup, jobName); + } + + @Getter + @EqualsAndHashCode + @AllArgsConstructor + public static class DagNodeId { + String flowGroup; + String flowName; + long flowExecutionId; + String jobGroup; + String jobName; + + @Override + public String toString() { + return Joiner.on("_").join(flowGroup, flowName, flowExecutionId, jobGroup, jobName); + } + } } /** diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagement.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagement.java new file mode 100644 index 00000000000..f04968bc6d7 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagement.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.io.IOException; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.runtime.api.DagActionStore; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; + +/** + * Responsible for defining the behavior of {@link DagTask} handling scenarios for launch, resume, kill, job start + * and flow completion deadlines + * + */ +@Alpha +public interface DagManagement { + + void launchFlow(String flowGroup, String flowName, long eventTimestamp); + + /** + * Handles the resume of a {@link Dag} request via REST client or triggered by the scheduler. + * @param resumeAction + * @param eventTimestamp + * @throws IOException + */ + void resumeFlow(DagActionStore.DagAction resumeAction, long eventTimestamp) throws IOException; + + /** + * Currently, it is handling just the kill/cancel of a {@link Dag} request via REST client + * @param killAction + * @param eventTimestamp + * @throws IOException + */ + void killFlow(DagActionStore.DagAction killAction, long eventTimestamp) throws IOException; +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java new file mode 100644 index 00000000000..b1b296261ad --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.io.IOException; +import java.util.LinkedList; + +import com.google.common.base.Optional; + +import org.apache.gobblin.runtime.api.DagActionStore; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + + +/** + * An interface to provide abstractions for managing {@link Dag} and {@link org.apache.gobblin.service.modules.flowgraph.Dag.DagNode} states + * and allows add/delete and other functions + */ +public interface DagManagementStateStore { + +// public void addDag(Dag dag); + + public void addJobState(String dagId, Dag.DagNode dagNode); + + public void deleteJobState(String dagId, Dag.DagNode dagNode); + + public boolean hasRunningJobs(String dagId); + + public void removeDagActionFromStore(DagManager.DagId dagId, DagActionStore.FlowActionType flowActionType) throws IOException; + +// public void addDagStartDeadline(String dagId, Long flowStartSla); +// public Long getDagStartDeadline(String dagId); + public void addDagDeadline(String dagId, Long flowSla); + public Long getDagDeadline(String dagId); + + public Optional> getDag(String dagId); + + public LinkedList> getJobs(String dagId) throws IOException; + + public boolean addFailedDag(String dagId); + + public boolean existsFailedDag(String dagId); + + public boolean addCleanUpDag(String dagId); + + public boolean checkCleanUpDag(String dagId); + + + //TODO: Add get methods for dags and jobs + + } diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java index c1553da39ca..2a7096de930 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManager.java @@ -75,6 +75,7 @@ import org.apache.gobblin.service.ExecutionStatus; import org.apache.gobblin.service.FlowId; import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; import org.apache.gobblin.service.modules.flow.SpecCompiler; import org.apache.gobblin.service.modules.flowgraph.Dag; import org.apache.gobblin.service.modules.flowgraph.Dag.DagNode; @@ -174,8 +175,8 @@ public String toString() { public static class DagId { String flowGroup; String flowName; - String flowExecutionId; - public DagId(String flowGroup, String flowName, String flowExecutionId) { + long flowExecutionId; + public DagId(String flowGroup, String flowName, long flowExecutionId) { this.flowGroup = flowGroup; this.flowName = flowName; this.flowExecutionId = flowExecutionId; @@ -238,7 +239,7 @@ public DagManager(Config config, JobStatusRetriever jobStatusRetriever, MetricContext metricContext = null; if (instrumentationEnabled) { metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()), getClass()); - this.eventSubmitter = Optional.of(new EventSubmitter.Builder(metricContext, "org.apache.gobblin.service").build()); + this.eventSubmitter = Optional.of(new EventSubmitter.Builder(metricContext, ServiceConstants.SERVICE_NAMESPACE).build()); } else { this.eventSubmitter = Optional.absent(); } @@ -272,7 +273,7 @@ DagStateStore createDagStateStore(Config config, Map topology } // Initializes and returns an array of Queue of size numThreads - private static LinkedBlockingDeque[] initializeDagQueue(int numThreads) { + static LinkedBlockingDeque[] initializeDagQueue(int numThreads) { LinkedBlockingDeque[] queue = new LinkedBlockingDeque[numThreads]; for (int i=0; i< numThreads; i++) { @@ -460,10 +461,10 @@ public synchronized void setActive(boolean active) { for (DagActionStore.DagAction action : dagActions) { switch (action.getFlowActionType()) { case KILL: - this.handleKillFlowEvent(new KillFlowEvent(action.getFlowGroup(), action.getFlowName(), Long.parseLong(action.getFlowExecutionId()))); + this.handleKillFlowEvent(new KillFlowEvent(action.getFlowGroup(), action.getFlowName(), action.getFlowExecutionId())); break; case RESUME: - this.handleResumeFlowEvent(new ResumeFlowEvent(action.getFlowGroup(), action.getFlowName(), Long.parseLong(action.getFlowExecutionId()))); + this.handleResumeFlowEvent(new ResumeFlowEvent(action.getFlowGroup(), action.getFlowName(), action.getFlowExecutionId())); break; case LAUNCH: this.handleLaunchFlowEvent(action); @@ -590,7 +591,7 @@ public static class DagManagerThread implements Runnable { if (instrumentationEnabled) { this.metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()), getClass()); - this.eventSubmitter = Optional.of(new EventSubmitter.Builder(this.metricContext, "org.apache.gobblin.service").build()); + this.eventSubmitter = Optional.of(new EventSubmitter.Builder(this.metricContext, ServiceConstants.SERVICE_NAMESPACE).build()); this.jobStatusPolledTimer = Optional.of(this.metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER)); ContextAwareGauge orchestrationDelayMetric = metricContext.newContextAwareGauge(ServiceMetricNames.FLOW_ORCHESTRATION_DELAY, orchestrationDelay::get); @@ -838,7 +839,7 @@ private void initialize(Dag dag) /** * Proceed the execution of each dag node based on job status. */ - private void pollAndAdvanceDag() throws IOException, ExecutionException, InterruptedException { + private void pollAndAdvanceDag() { Map>> nextSubmitted = Maps.newHashMap(); List> nodesToCleanUp = Lists.newArrayList(); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagerUtils.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagerUtils.java index 152dca00d65..44fe720c58d 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagerUtils.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagerUtils.java @@ -48,6 +48,7 @@ import org.apache.gobblin.service.modules.flowgraph.Dag.DagNode; import org.apache.gobblin.service.modules.orchestration.DagManager.FailureOption; import org.apache.gobblin.service.modules.spec.JobExecutionPlan; +import org.apache.gobblin.service.monitoring.event.JobStatusEvent; import org.apache.gobblin.util.ConfigUtils; @@ -55,7 +56,7 @@ public class DagManagerUtils { static long DEFAULT_FLOW_SLA_MILLIS = TimeUnit.HOURS.toMillis(24); static String QUOTA_KEY_SEPERATOR = ","; - static FlowId getFlowId(Dag dag) { + public static FlowId getFlowId(Dag dag) { return getFlowId(dag.getStartNodes().get(0)); } @@ -66,7 +67,7 @@ static FlowId getFlowId(DagNode dagNode) { return new FlowId().setFlowGroup(flowGroup).setFlowName(flowName); } - static long getFlowExecId(Dag dag) { + public static long getFlowExecId(Dag dag) { return getFlowExecId(dag.getStartNodes().get(0)); } @@ -97,19 +98,23 @@ private static DagManager.DagId generateDagId(Config jobConfig) { String flowName = jobConfig.getString(ConfigurationKeys.FLOW_NAME_KEY); long flowExecutionId = jobConfig.getLong(ConfigurationKeys.FLOW_EXECUTION_ID_KEY); - return new DagManager.DagId(flowGroup, flowName, String.valueOf(flowExecutionId)); + return new DagManager.DagId(flowGroup, flowName, flowExecutionId); } - static DagManager.DagId generateDagId(Dag.DagNode dagNode) { + public static DagManager.DagId generateDagId(Dag.DagNode dagNode) { return generateDagId(dagNode.getValue().getJobSpec().getConfig()); } - static DagManager.DagId generateDagId(String flowGroup, String flowName, long flowExecutionId) { - return generateDagId(flowGroup, flowName, String.valueOf(flowExecutionId)); + public static DagManager.DagId generateDagId(String flowGroup, String flowName, long flowExecutionId) { + return new DagManager.DagId(flowGroup, flowName, flowExecutionId); } - static DagManager.DagId generateDagId(String flowGroup, String flowName, String flowExecutionId) { - return new DagManager.DagId(flowGroup, flowName, flowExecutionId); + public static DagNode.DagNodeId generateDagNodeId(String flowGroup, String flowName, long flowExecutionId, String jobGroup, String jobName) { + return new DagNode.DagNodeId(flowGroup, flowName, flowExecutionId, jobGroup, jobName); + } + + public static DagNode.DagNodeId generateDagNodeId(JobStatusEvent jobStatusEvent) { + return generateDagNodeId(jobStatusEvent.getFlowGroup(), jobStatusEvent.getFlowName(), jobStatusEvent.getFlowExecutionId(), jobStatusEvent.getJobGroup(), jobStatusEvent.getJobName()); } /** @@ -117,7 +122,7 @@ static DagManager.DagId generateDagId(String flowGroup, String flowName, String * @param dag * @return fully qualified name of the underlying {@link Dag}. */ - static String getFullyQualifiedDagName(Dag dag) { + public static String getFullyQualifiedDagName(Dag dag) { FlowId flowid = getFlowId(dag); long flowExecutionId = getFlowExecId(dag); return "(flowGroup: " + flowid.getFlowGroup() + ", flowName: " + flowid.getFlowName() + ", flowExecutionId: " + flowExecutionId + ")"; @@ -134,7 +139,7 @@ static String getFullyQualifiedDagName(DagNode dagNode) { return "(flowGroup: " + flowid.getFlowGroup() + ", flowName: " + flowid.getFlowName() + ", flowExecutionId: " + flowExecutionId + ")"; } - static String getJobName(DagNode dagNode) { + public static String getJobName(DagNode dagNode) { return dagNode.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.JOB_NAME_KEY); } @@ -143,7 +148,7 @@ static String getJobName(DagNode dagNode) { * @param dagNode * @return a fully qualified name of the underlying job. */ - static String getFullyQualifiedJobName(DagNode dagNode) { + public static String getFullyQualifiedJobName(DagNode dagNode) { Config jobConfig = dagNode.getValue().getJobSpec().getConfig(); String flowGroup = ConfigUtils.getString(jobConfig, ConfigurationKeys.FLOW_GROUP_KEY, ""); @@ -154,7 +159,7 @@ static String getFullyQualifiedJobName(DagNode dagNode) { return "(flowGroup: " + flowGroup + ", flowName: " + flowName + ", flowExecutionId: " + flowExecutionId + ", jobName: " + jobName + ")"; } - static JobExecutionPlan getJobExecutionPlan(DagNode dagNode) { + public static JobExecutionPlan getJobExecutionPlan(DagNode dagNode) { return dagNode.getValue(); } @@ -173,12 +178,12 @@ static Config getJobConfig(DagNode dagNode) { return dagNode.getValue().getJobSpec().getConfig(); } - static SpecProducer getSpecProducer(DagNode dagNode) + public static SpecProducer getSpecProducer(DagNode dagNode) throws ExecutionException, InterruptedException { return dagNode.getValue().getSpecExecutor().getProducer().get(); } - static ExecutionStatus getExecutionStatus(DagNode dagNode) { + public static ExecutionStatus getExecutionStatus(DagNode dagNode) { return dagNode.getValue().getExecutionStatus(); } @@ -187,7 +192,7 @@ static ExecutionStatus getExecutionStatus(DagNode dagNode) { * identifies each node yet to be executed and for which each of its parent nodes is in the {@link ExecutionStatus#COMPLETE} * state. */ - static Set> getNext(Dag dag) { + public static Set> getNext(Dag dag) { Set> nextNodesToExecute = new HashSet<>(); LinkedList> nodesToExpand = Lists.newLinkedList(dag.getStartNodes()); FailureOption failureOption = getFailureOption(dag); @@ -235,7 +240,7 @@ static FailureOption getFailureOption(Dag dag) { return FailureOption.valueOf(failureOption); } - static String getSpecExecutorUri(DagNode dagNode) { + public static String getSpecExecutorUri(DagNode dagNode) { return dagNode.getValue().getSpecExecutor().getUri().toString(); } @@ -253,7 +258,7 @@ static String getFlowGroupQuotaKey(String flowGroup, DagNode d /** * Increment the value of {@link JobExecutionPlan#currentAttempts} */ - static void incrementJobAttempt(DagNode dagNode) { + public static void incrementJobAttempt(DagNode dagNode) { dagNode.getValue().setCurrentAttempts(dagNode.getValue().getCurrentAttempts() + 1); } @@ -332,7 +337,7 @@ static String getSpecExecutorName(DagNode dagNode) { return dagNode.getValue().getSpecExecutor().getUri().toString(); } - static void emitFlowEvent(Optional eventSubmitter, Dag dag, String flowEvent) { + public static void emitFlowEvent(Optional eventSubmitter, Dag dag, String flowEvent) { if (eventSubmitter.isPresent() && !dag.isEmpty()) { // Every dag node will contain the same flow metadata Config config = getDagJobConfig(dag); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcFactory.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcFactory.java new file mode 100644 index 00000000000..d89c628a366 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcFactory.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.util.Optional; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.instrumented.Instrumented; +import org.apache.gobblin.metrics.MetricContext; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; +import org.apache.gobblin.service.ServiceConstants; +import org.apache.gobblin.service.modules.flow.SpecCompiler; +import org.apache.gobblin.service.modules.orchestration.proc.AdvanceDagProc; +import org.apache.gobblin.service.modules.orchestration.proc.CleanUpDagProc; +import org.apache.gobblin.service.modules.orchestration.proc.DagProc; +import org.apache.gobblin.service.modules.orchestration.proc.KillDagProc; +import org.apache.gobblin.service.modules.orchestration.proc.LaunchDagProc; +import org.apache.gobblin.service.modules.orchestration.proc.ResumeDagProc; +import org.apache.gobblin.service.modules.orchestration.proc.RetryDagProc; +import org.apache.gobblin.service.modules.orchestration.task.AdvanceDagTask; +import org.apache.gobblin.service.modules.orchestration.task.CleanUpDagTask; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; +import org.apache.gobblin.service.modules.orchestration.task.KillDagTask; +import org.apache.gobblin.service.modules.orchestration.task.LaunchDagTask; +import org.apache.gobblin.service.modules.orchestration.task.ResumeDagTask; +import org.apache.gobblin.service.modules.orchestration.task.RetryDagTask; +import org.apache.gobblin.service.modules.utils.FlowCompilationValidationHelper; +import org.apache.gobblin.service.monitoring.FlowStatusGenerator; +import org.apache.gobblin.service.monitoring.JobStatusRetriever; +import org.apache.gobblin.util.ConfigUtils; + + +/** + * Factory for creating {@link DagProc} based on the visitor type for a given {@link DagTask}. + */ + +@Alpha +@Slf4j +public class DagProcFactory {//implements DagTaskVisitor { + + + // check what all fields are needed by DagProc implementations + public NewDagManager dagManager; + private JobStatusRetriever jobStatusRetriever; + private FlowStatusGenerator flowStatusGenerator; + private UserQuotaManager quotaManager; + private SpecCompiler specCompiler; + private FlowCatalog flowCatalog; + private FlowCompilationValidationHelper flowCompilationValidationHelper; + private Config config; + public final DagProcessingEngine dagProcessingEngine; + Optional eventSubmitter; + + // arjun - we may need to pass these objects to different DagProcs + + public DagProcFactory(Config config, NewDagManager dagManager, JobStatusRetriever jobStatusRetriever, + FlowStatusGenerator flowStatusGenerator, UserQuotaManager quotaManager, SpecCompiler specCompiler, FlowCatalog flowCatalog, + FlowCompilationValidationHelper flowCompilationValidationHelper, boolean instrumentationEnabled, DagProcessingEngine dagProcessingEngine) { + + this.config = config; + this.dagManager = dagManager; + this.jobStatusRetriever = jobStatusRetriever; + this.flowStatusGenerator = flowStatusGenerator; + this.quotaManager = quotaManager; + this.specCompiler = specCompiler; + this.flowCatalog = flowCatalog; + this.flowCompilationValidationHelper = flowCompilationValidationHelper; + this.dagProcessingEngine = dagProcessingEngine; + if (instrumentationEnabled) { + MetricContext metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()), getClass()); + this.eventSubmitter = Optional.of(new EventSubmitter.Builder(metricContext, ServiceConstants.SERVICE_NAMESPACE).build()); + } else { + this.eventSubmitter = Optional.empty(); + } + } + + public DagProc getDagProcFor(DagTask dagTask) { + if (dagTask instanceof LaunchDagTask) { + return new LaunchDagProc((LaunchDagTask) dagTask, this); // check what all fields are needed by DagProc implementations + } else if (dagTask instanceof KillDagTask) { + return new KillDagProc((KillDagTask) dagTask, this); + } else if (dagTask instanceof ResumeDagTask) { + return new ResumeDagProc((ResumeDagTask) dagTask, this); + } else if (dagTask instanceof AdvanceDagTask) { + return new AdvanceDagProc((AdvanceDagTask) dagTask, this); + } else if (dagTask instanceof RetryDagTask) { + return new RetryDagProc((RetryDagTask) dagTask, this); + }else if (dagTask instanceof CleanUpDagTask) { + return new CleanUpDagProc((CleanUpDagTask) dagTask, this); + } + throw new UnsupportedOperationException("Invalid dagTask " + dagTask); + } +} + diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java new file mode 100644 index 00000000000..cb2a8b7684a --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcessingEngine.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.io.IOException; +import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.instrumented.Instrumented; +import org.apache.gobblin.metrics.MetricContext; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter; +import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.proc.DagProc; +import org.apache.gobblin.service.modules.orchestration.task.AdvanceDagTask; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; +import org.apache.gobblin.service.modules.orchestration.task.LaunchDagTask; +import org.apache.gobblin.service.modules.orchestration.task.RetryDagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; +import org.apache.gobblin.util.ConfigUtils; + + +/** + * Responsible for polling {@link DagTask}s from {@link DagTaskStream} and processing the {@link org.apache.gobblin.service.modules.flowgraph.Dag} + * based on the type of {@link DagTask} which is determined by the {@link org.apache.gobblin.runtime.api.DagActionStore.DagAction}. + * Each {@link DagTask} acquires a lease for the {@link org.apache.gobblin.runtime.api.DagActionStore.DagAction}. + * The {@link DagProcFactory} then provides the appropriate {@link DagProc} associated with the {@link DagTask}. + * The actual work or processing is done by the {@link DagProc#process(DagManagementStateStore, int, long)} + */ + +@Alpha +@Slf4j +public class DagProcessingEngine { + public static final String GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX = ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "dagProcessingEngine."; + public static final String GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_ENABLED_KEY = GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "enabled"; + public static final String NUM_THREADS_KEY = GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "numThreads"; + public static final String JOB_STATUS_POLLING_INTERVAL_KEY = GOBBLIN_SERVICE_DAG_PROCESSING_ENGINE_PREFIX + "pollingInterval"; + private static final Integer DEFAULT_NUM_THREADS = 3; + private static final Integer DEFAULT_JOB_STATUS_POLLING_INTERVAL = 10; + + private final DagTaskStream dagTaskStream; + Optional eventSubmitter; + + public DagProcessingEngine(Config config, DagTaskStream dagTaskStream, DagProcFactory dagProcFactory, DagManagementStateStore dagManagementStateStore, MultiActiveLeaseArbiter multiActiveLeaseArbiter) { + MetricContext metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()), getClass()); + this.eventSubmitter = Optional.of(new EventSubmitter.Builder(metricContext, ServiceConstants.SERVICE_NAMESPACE).build()); + Integer numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY, DEFAULT_NUM_THREADS); + ScheduledExecutorService scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads); + Integer pollingInterval = ConfigUtils.getInt(config, JOB_STATUS_POLLING_INTERVAL_KEY, DEFAULT_JOB_STATUS_POLLING_INTERVAL); + this.dagTaskStream = dagTaskStream; + + for (int i=0; i < numThreads; i++) { + DagProcEngineThread dagProcEngineThread = new DagProcEngineThread(dagTaskStream, dagProcFactory); + scheduledExecutorPool.scheduleAtFixedRate(dagProcEngineThread, 0, pollingInterval, TimeUnit.SECONDS); + } + } + + public void addNewDag(Dag dag) { + DagTask dagTask = new LaunchDagTask(dag); + this.dagTaskStream.addDagTask(dagTask); + } + + public void addDagNodeToRetry(Dag.DagNode dagNode) { + this.dagTaskStream.addDagTask(new RetryDagTask(dagNode)); + } + + public void addAdvanceDagTask(Dag.DagNode dagNode) { + this.dagTaskStream.addDagTask(new AdvanceDagTask(dagNode)); + } + + public void addDagTask(DagTask dagTask) { + this.dagTaskStream.addDagTask(dagTask); + } + + @AllArgsConstructor + private static class DagProcEngineThread implements Runnable { + + private DagTaskStream dagTaskStream; + private DagProcFactory dagProcFactory; + + @Override + public void run() { + for (DagTaskStream it = dagTaskStream; it.hasNext(); ) { + DagTask dagTask = it.next(); + DagProc dagProc = dagProcFactory.getDagProcFor(dagTask); +// dagProc.process(eventSubmitter.get(), maxRetryAttempts, delayRetryMillis); + try { + dagProc.process(dagTask); + } catch (IOException e) { + throw new RuntimeException(e); + } + // todo mark lease success and releases it + //dagTaskStream.complete(dagTask); + } + } + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagTaskStream.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagTaskStream.java new file mode 100644 index 00000000000..e03c3295ff8 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagTaskStream.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Properties; +import java.util.concurrent.BlockingQueue; +import com.google.common.base.Optional; +import com.typesafe.config.Config; + +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.runtime.api.DagActionStore; +import org.apache.gobblin.service.ExecutionStatus; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; +import org.apache.gobblin.service.monitoring.JobStatus; +import org.apache.gobblin.service.monitoring.JobStatusRetriever; + + +/** + * Holds a stream of {@link DagTask}s that {@link DagProcessingEngine} would pull from for processing. + * It provides an implementation for {@link DagManagement} that defines the rules for a flow and job. + * Implements {@link Iterable} to provide {@link DagTask}s as soon as it's available to {@link DagProcessingEngine} + */ + +@Alpha +@Slf4j +@AllArgsConstructor + +// change to iterable +public class DagTaskStream implements Iterator{ + + private final BlockingQueue dagActionQueue; + private final FlowTriggerHandler flowTriggerHandler; + private final DagManagementStateStore dagManagementStateStore; + + @Override + public boolean hasNext() { + return !this.dagActionQueue.isEmpty(); + } + + @Override + public DagTask next() { + DagTask dagTask = this.dagActionQueue.poll(); + assert dagTask != null; + try { + // todo reconsider the use of MultiActiveLeaseArbiter +// MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus = +// flowTriggerHandler.getLeaseOnDagAction(jobProps, dagAction, System.currentTimeMillis()); +// if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) { + // can it return null? is this iterator allowed to return null? + return dagTask; + //} + } catch (Exception e) { + //TODO: need to handle exceptions gracefully + log.error("Error creating DagTask", e); + return null; + } + } + + /** + * Enforce cancel on the job if the job has been "orphaned". A job is orphaned if has been in ORCHESTRATED + * {@link ExecutionStatus} for some specific amount of time determined by config. + * @param node {@link Dag.DagNode} representing the job + * @param jobStatus current {@link JobStatus} of the job + * @return true if the total time that the job remains in the ORCHESTRATED state exceeds + * {@value ConfigurationKeys#GOBBLIN_JOB_START_SLA_TIME}. + */ + + protected void complete(DagTask dagTask) throws IOException { + dagTask.conclude(this.flowTriggerHandler.getMultiActiveLeaseArbiter()); + } + + /** + * Retrieve the {@link JobStatus} from the {@link JobExecutionPlan}. + */ + + // arjun add job status retriever somewhere.... dagProcEngine or some new component? and add it there. find where do we need retrieve status + + protected JobStatus retrieveJobStatus(Dag.DagNode dagNode) { + Config jobConfig = dagNode.getValue().getJobSpec().getConfig(); + String flowGroup = jobConfig.getString(ConfigurationKeys.FLOW_GROUP_KEY); + String flowName = jobConfig.getString(ConfigurationKeys.FLOW_NAME_KEY); + long flowExecutionId = jobConfig.getLong(ConfigurationKeys.FLOW_EXECUTION_ID_KEY); + String jobGroup = jobConfig.getString(ConfigurationKeys.JOB_GROUP_KEY); + String jobName = jobConfig.getString(ConfigurationKeys.JOB_NAME_KEY); + + return getStatus(flowGroup, flowName, flowExecutionId, jobGroup, jobName); + } + + /** + * Retrieve the flow's {@link JobStatus} (i.e. job status with {@link JobStatusRetriever#NA_KEY} as job name/group) from a dag + */ + protected JobStatus retrieveFlowStatus(Dag dag) { + if (dag == null || dag.isEmpty()) { + return null; + } + Config jobConfig = dag.getNodes().get(0).getValue().getJobSpec().getConfig(); + String flowGroup = jobConfig.getString(ConfigurationKeys.FLOW_GROUP_KEY); + String flowName = jobConfig.getString(ConfigurationKeys.FLOW_NAME_KEY); + long flowExecutionId = jobConfig.getLong(ConfigurationKeys.FLOW_EXECUTION_ID_KEY); + + return getStatus(flowGroup, flowName, flowExecutionId, JobStatusRetriever.NA_KEY, JobStatusRetriever.NA_KEY); + } + + private JobStatus getStatus(String flowGroup, String flowName, long flowExecutionId, String jobGroup, String jobName) { + throw new UnsupportedOperationException("Currently retrieval of flow/job status is not supported"); + } + + private Properties getJobProperties(DagActionStore.DagAction dagAction) { + String dagId = DagManagerUtils.generateDagId( + dagAction.getFlowGroup(), dagAction.getFlowName(), dagAction.getFlowExecutionId()).toString(); + Optional> dag = dagManagementStateStore.getDag(dagId); + return dag.isPresent() ? dag.get().getStartNodes().get(0).getValue().getJobSpec().getConfigAsProperties() : new Properties(); + } + + public void addDagTask(DagTask dagTask) { + this.dagActionQueue.add(dagTask); + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java index af65390ec50..eea4455c1b5 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandler.java @@ -17,9 +17,6 @@ package org.apache.gobblin.service.modules.orchestration; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; -import com.typesafe.config.Config; import java.io.IOException; import java.time.LocalDateTime; import java.time.ZoneId; @@ -29,8 +26,21 @@ import java.util.Locale; import java.util.Properties; import java.util.Random; + +import org.quartz.JobDataMap; +import org.quartz.JobDetail; +import org.quartz.JobKey; +import org.quartz.SchedulerException; +import org.quartz.Trigger; +import org.quartz.impl.JobDetailImpl; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +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.ContextAwareCounter; @@ -44,12 +54,6 @@ import org.apache.gobblin.scheduler.SchedulerService; import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; import org.apache.gobblin.util.ConfigUtils; -import org.quartz.JobDataMap; -import org.quartz.JobDetail; -import org.quartz.JobKey; -import org.quartz.SchedulerException; -import org.quartz.Trigger; -import org.quartz.impl.JobDetailImpl; /** @@ -66,7 +70,7 @@ public class FlowTriggerHandler { private final int schedulerMaxBackoffMillis; private static Random random = new Random(); - protected Optional multiActiveLeaseArbiter; + private Optional multiActiveLeaseArbiter; protected SchedulerService schedulerService; protected Optional dagActionStore; private MetricContext metricContext; @@ -324,4 +328,43 @@ protected static long getUTCTimeFromDelayPeriod(long delayPeriodMillis) { Date date = Date.from(localDateTime.atZone(ZoneId.of("UTC")).toInstant()); return GobblinServiceJobScheduler.utcDateAsUTCEpochMillis(date); } + + /** + * Attempts to acquire lease for a given {@link org.apache.gobblin.runtime.api.DagActionStore.DagAction} + * through lease arbitration and if it fails, it will create and schedule a reminder trigger to check back again. + * @param jobProps + * @param flowAction + * @param eventTimeMillis + * @return optionally leaseObtainedStatus if acquired; otherwise schedule reminder to check back again. + * @throws IOException + */ + public MultiActiveLeaseArbiter.LeaseAttemptStatus getLeaseOnDagAction(Properties jobProps, DagActionStore.DagAction flowAction, long eventTimeMillis) throws IOException { + + if (multiActiveLeaseArbiter.isPresent()) { + boolean isReminderEvent = + Boolean.parseBoolean(jobProps.getProperty(ConfigurationKeys.FLOW_IS_REMINDER_EVENT_KEY, "false")); + MultiActiveLeaseArbiter.LeaseAttemptStatus leaseAttemptStatus = + multiActiveLeaseArbiter.get().tryAcquireLease(flowAction, eventTimeMillis, isReminderEvent); + if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeaseObtainedStatus) { + this.leaseObtainedCount.inc(); + log.info("Successfully acquired lease for dag action: {}", flowAction); + } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.LeasedToAnotherStatus) { + this.leasedToAnotherStatusCount.inc(); + scheduleReminderForEvent(jobProps, + (MultiActiveLeaseArbiter.LeasedToAnotherStatus) leaseAttemptStatus, eventTimeMillis); + } else if (leaseAttemptStatus instanceof MultiActiveLeaseArbiter.NoLongerLeasingStatus) { + this.noLongerLeasingStatusCount.inc(); + log.info("Received type of leaseAttemptStatus: [{}, eventTimestamp: {}] ", leaseAttemptStatus.getClass().getName(), + eventTimeMillis); + } + return leaseAttemptStatus; + } else { + throw new RuntimeException("Multi-active scheduler is not enabled so trigger event should not be " + + "handled with this method."); + } + } + + public MultiActiveLeaseArbiter getMultiActiveLeaseArbiter() { + return this.multiActiveLeaseArbiter.get(); + } } diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/InMemoryDagManagementStateStore.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/InMemoryDagManagementStateStore.java new file mode 100644 index 00000000000..80c355a8900 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/InMemoryDagManagementStateStore.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gobblin.service.modules.orchestration; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import com.google.common.base.Optional; +import com.google.common.collect.Lists; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.runtime.api.DagActionStore; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + + +/** + * An implementation of {@link DagManagementStateStore} to provide information about dags, dag nodes and their job states. + * This store maintains and utilizes in-memory references about dags and their job states and is used + * to determine what the current status of the {@link Dag} and/or {@link Dag.DagNode} is and what actions needs to be + * taken next likewise mark it as: complete, failed, sla breached or simply clean up after completion. + * Going forward, each of these in-memory references will be read/write from MySQL store. + * Thus, the {@link DagManager} would then be stateless and operate independently. + */ +@Alpha +@Slf4j +public class InMemoryDagManagementStateStore implements DagManagementStateStore { + private final Map, Dag> jobToDag = new HashMap<>(); + private final Map> dagIdToDags = new HashMap<>(); + private final Set failedDagIds = new HashSet<>(); + private final Map> dagIdToResumingDags = new HashMap<>(); + // dagToJobs holds a map of dagId to running jobs of that dag + final Map>> dagToJobs = new HashMap<>(); + final Map dagToDeadline = new HashMap<>(); + private final Set dagIdstoClean = new HashSet<>(); + private DagActionStore dagActionStore; + + @Override + public synchronized void deleteJobState(String dagId, Dag.DagNode dagNode) { + this.jobToDag.remove(dagNode); + this.dagToJobs.get(dagId).remove(dagNode); + this.dagToDeadline.remove(dagId); + } + +// @Override +// public void addDag(Dag dag) { +// +// } + + @Override + public synchronized void addJobState(String dagId, Dag.DagNode dagNode) { + Dag dag = this.dagIdToDags.get(dagId); + this.jobToDag.put(dagNode, dag); + if (this.dagToJobs.containsKey(dagId)) { + this.dagToJobs.get(dagId).add(dagNode); + } else { + LinkedList> dagNodeList = Lists.newLinkedList(); + dagNodeList.add(dagNode); + this.dagToJobs.put(dagId, dagNodeList); + } + } + + @Override + public synchronized boolean hasRunningJobs(String dagId) { + List> dagNodes = this.dagToJobs.get(dagId); + return dagNodes != null && !dagNodes.isEmpty(); + } + + @Override + public synchronized void removeDagActionFromStore(DagManager.DagId dagId, DagActionStore.FlowActionType flowActionType) throws IOException { + this.dagActionStore.deleteDagAction(new DagActionStore.DagAction(dagId.flowGroup, dagId.flowName, dagId.flowExecutionId, flowActionType)); + } + + @Override + public void addDagDeadline(String dagId, Long flowSla) { + this.dagToDeadline.putIfAbsent(dagId, flowSla); + } + + @Override + public Long getDagDeadline(String dagId) { + if(this.dagToDeadline.containsKey(dagId)) { + return this.dagToDeadline.get(dagId); + } + return null; + } + + @Override + public Optional> getDag(String dagId) { + if(this.dagIdToDags.containsKey(dagId)) { + return Optional.of(this.dagIdToDags.get(dagId)); + } else { + log.error("Dag {} not found in dagIdToDags map.", dagId); + return Optional.absent(); + } + } + + @Override + public LinkedList> getJobs(String dagId) throws IOException { + if(this.dagToJobs.containsKey(dagId)) { + return this.dagToJobs.get(dagId); + } + throw new IOException("Dag Id: " + dagId + "is not present"); + } + + @Override + public boolean addFailedDag(String dagId) { + return this.failedDagIds.add(dagId); + } + + @Override + public boolean existsFailedDag(String dagId) { + return this.failedDagIds.contains(dagId); + } + + @Override + public boolean addCleanUpDag(String dagId) { + return this.dagIdstoClean.add(dagId); + } + + @Override + public boolean checkCleanUpDag(String dagId) { + return this.dagIdstoClean.contains(dagId); + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/KillDagThread.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/KillDagThread.java new file mode 100644 index 00000000000..5bd4d0f6611 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/KillDagThread.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigException; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.instrumented.Instrumented; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.service.ExecutionStatus; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.task.KillDagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; +import org.apache.gobblin.service.monitoring.JobStatus; + +import static org.apache.gobblin.service.ExecutionStatus.CANCELLED; +import static org.apache.gobblin.service.ExecutionStatus.ORCHESTRATED; +import static org.apache.gobblin.service.ExecutionStatus.valueOf; + + +@Slf4j +public class KillDagThread implements Runnable { + private final Long defaultJobStartSlaTimeMillis; + NewDagManager dagManager; + public KillDagThread(Long defaultJobStartSlaTimeMillis, NewDagManager newDagManager) { + this.defaultJobStartSlaTimeMillis = defaultJobStartSlaTimeMillis; + this.dagManager = newDagManager; + } + + @Override + public void run() { + for (Dag.DagNode node : this.dagManager.getJobToDag().keySet()) { + boolean flowKilled = enforceFlowStartDeadline(node); + boolean jobKilled = false; + + if (!flowKilled) { + JobStatus jobStatus = pollJobStatus(node); + try { + jobKilled = enforceJobCompletionDeadline(node, jobStatus); + } catch (ExecutionException | InterruptedException e) { + log.warn("Error getting status for dag node " + node.getId()); + continue; + } + } + + if (flowKilled || jobKilled) { + JobExecutionPlan jobExecutionPlan = DagManagerUtils.getJobExecutionPlan(node); + jobExecutionPlan.setExecutionStatus(CANCELLED); + try { + this.dagManager.onJobFinish(node); + } catch (IOException e) { + throw new RuntimeException(e); + } + String dagId = DagManagerUtils.generateDagId(node).toString(); + this.dagManager.deleteJobState(dagId, node); + } + } + } + + /** + * Retrieve the {@link JobStatus} from the {@link JobExecutionPlan}. + */ + private JobStatus pollJobStatus(Dag.DagNode dagNode) { + Config jobConfig = dagNode.getValue().getJobSpec().getConfig(); + String flowGroup = jobConfig.getString(ConfigurationKeys.FLOW_GROUP_KEY); + String flowName = jobConfig.getString(ConfigurationKeys.FLOW_NAME_KEY); + long flowExecutionId = jobConfig.getLong(ConfigurationKeys.FLOW_EXECUTION_ID_KEY); + String jobGroup = jobConfig.getString(ConfigurationKeys.JOB_GROUP_KEY); + String jobName = jobConfig.getString(ConfigurationKeys.JOB_NAME_KEY); + + return pollStatus(flowGroup, flowName, flowExecutionId, jobGroup, jobName); + } + + private JobStatus pollStatus(String flowGroup, String flowName, long flowExecutionId, String jobGroup, String jobName) { + long pollStartTime = System.nanoTime(); + Iterator jobStatusIterator = + this.dagManager.getJobStatusRetriever().getJobStatusesForFlowExecution(flowName, flowGroup, flowExecutionId, jobName, jobGroup); + Instrumented.updateTimer(this.dagManager.getJobStatusPolledTimer(), System.nanoTime() - pollStartTime, TimeUnit.NANOSECONDS); + + if (jobStatusIterator.hasNext()) { + return jobStatusIterator.next(); + } else { + return null; + } + } + + private boolean enforceJobCompletionDeadline(Dag.DagNode node, JobStatus jobStatus) + throws ExecutionException, InterruptedException { + if (jobStatus == null) { + return false; + } + ExecutionStatus executionStatus = valueOf(jobStatus.getEventName()); + long timeOutForJobStart = DagManagerUtils.getJobStartSla(node, this.defaultJobStartSlaTimeMillis); + long jobOrchestratedTime = jobStatus.getOrchestratedTime(); + if (executionStatus == ORCHESTRATED && System.currentTimeMillis() - jobOrchestratedTime > timeOutForJobStart) { + log.info("Job {} of flow {} exceeded the job start SLA of {} ms. Killing the job now...", + DagManagerUtils.getJobName(node), + DagManagerUtils.getFullyQualifiedDagName(node), + timeOutForJobStart); + this.dagManager.getDagManagerMetrics().incrementCountsStartSlaExceeded(node); + this.dagManager.getDagProcessingEngine().addDagTask(new KillDagTask(node.getId())); + String dagId = DagManagerUtils.generateDagId(node).toString(); + this.dagManager.getDags().get(dagId).setFlowEvent(TimingEvent.FlowTimings.FLOW_START_DEADLINE_EXCEEDED); + this.dagManager.getDags().get(dagId).setMessage("Flow killed because no update received for " + timeOutForJobStart + " ms after orchestration"); + return true; + } else { + return false; + } + } + + private boolean enforceFlowStartDeadline(Dag.DagNode node) { + long flowStartTime = DagManagerUtils.getFlowStartTime(node); + long currentTime = System.currentTimeMillis(); + String dagId = DagManagerUtils.generateDagId(node).toString(); + + long flowSla; + if (this.dagManager.getDagToSLA().containsKey(dagId)) { + flowSla = this.dagManager.getDagToSLA().get(dagId); + } else { + try { + flowSla = DagManagerUtils.getFlowSLA(node); + } catch (ConfigException e) { + log.warn("Flow SLA for flowGroup: {}, flowName: {} is given in invalid format, using default SLA of {}", + node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_GROUP_KEY), + node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY), + DagManagerUtils.DEFAULT_FLOW_SLA_MILLIS); + flowSla = DagManagerUtils.DEFAULT_FLOW_SLA_MILLIS; + } + this.dagManager.getDagToSLA().put(dagId, flowSla); + } + + if (currentTime > flowStartTime + flowSla) { + log.info("Flow {} exceeded the SLA of {} ms. Killing the job {} now...", + node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY), flowSla, + node.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.JOB_NAME_KEY)); + this.dagManager.getDagManagerMetrics().incrementExecutorSlaExceeded(node); + this.dagManager.getDagProcessingEngine().addDagTask(new KillDagTask(node.getId())); + + this.dagManager.getDags().get(dagId).setFlowEvent(TimingEvent.FlowTimings.FLOW_RUN_DEADLINE_EXCEEDED); + this.dagManager.getDags().get(dagId).setMessage("Flow killed due to exceeding SLA of " + flowSla + " ms"); + + return true; + } + return false; + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java new file mode 100644 index 00000000000..2f615b10e7b --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/NewDagManager.java @@ -0,0 +1,398 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import com.codahale.metrics.Timer; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.eventbus.EventBus; +import com.google.common.util.concurrent.AbstractIdleService; +import com.google.inject.Inject; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.instrumented.Instrumented; +import org.apache.gobblin.metrics.MetricContext; +import org.apache.gobblin.metrics.ServiceMetricNames; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.runtime.api.DagActionStore; +import org.apache.gobblin.runtime.api.TopologySpec; +import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; +import org.apache.gobblin.service.ExecutionStatus; +import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; +import org.apache.gobblin.service.modules.flow.SpecCompiler; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; +import org.apache.gobblin.service.modules.utils.FlowCompilationValidationHelper; +import org.apache.gobblin.service.modules.utils.SharedFlowMetricsSingleton; +import org.apache.gobblin.service.monitoring.FlowStatusGenerator; +import org.apache.gobblin.service.monitoring.JobStatus; +import org.apache.gobblin.service.monitoring.JobStatusRetriever; +import org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor; +import org.apache.gobblin.service.monitoring.event.JobStatusEvent; +import org.apache.gobblin.util.ConfigUtils; +import org.apache.gobblin.util.reflection.GobblinConstructorUtils; + +import static org.apache.gobblin.service.ExecutionStatus.*; +import static org.apache.gobblin.service.ExecutionStatus.PENDING_RETRY; + + +/** + * NewDagManager manages dags in memory and various mappings. + */ +@Slf4j +public class NewDagManager extends AbstractIdleService { + public static final String DAG_MANAGER_PREFIX = "gobblin.service.dagManager."; + + private static final Integer DEFAULT_JOB_STATUS_POLLING_INTERVAL = 10; + public static final Integer DEFAULT_NUM_THREADS = 3; + private static final Integer TERMINATION_TIMEOUT = 30; + public static final String NUM_THREADS_KEY = DAG_MANAGER_PREFIX + "numThreads"; + public static final String JOB_STATUS_POLLING_INTERVAL_KEY = DAG_MANAGER_PREFIX + "pollingInterval"; + private static final String DAG_STATESTORE_CLASS_KEY = DAG_MANAGER_PREFIX + "dagStateStoreClass"; + private static final String FAILED_DAG_STATESTORE_PREFIX = "failedDagStateStore"; + private static final String FAILED_DAG_RETENTION_TIME_UNIT = FAILED_DAG_STATESTORE_PREFIX + ".retention.timeUnit"; + private static final String DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT = "DAYS"; + private static final String FAILED_DAG_RETENTION_TIME = FAILED_DAG_STATESTORE_PREFIX + ".retention.time"; + private static final long DEFAULT_FAILED_DAG_RETENTION_TIME = 7L; + // Re-emit the final flow status if not detected within 5 minutes + public static final String FAILED_DAG_POLLING_INTERVAL = FAILED_DAG_STATESTORE_PREFIX + ".retention.pollingIntervalMinutes"; + public static final Integer DEFAULT_FAILED_DAG_POLLING_INTERVAL = 60; + public static final String DAG_MANAGER_HEARTBEAT = ServiceMetricNames.GOBBLIN_SERVICE_PREFIX_WITH_DELIMITER + "dagManager.heartbeat-%s"; + // Default job start SLA time if configured, measured in minutes. Default is 10 minutes + private static final String JOB_START_SLA_TIME = DAG_MANAGER_PREFIX + ConfigurationKeys.GOBBLIN_JOB_START_SLA_TIME; + private static final String JOB_START_SLA_UNITS = DAG_MANAGER_PREFIX + ConfigurationKeys.GOBBLIN_JOB_START_SLA_TIME_UNIT; + private static final int INITIAL_HOUSEKEEPING_THREAD_DELAY = 2; + + public void addDag(Dag dag) { + // TODO : implement it, get some code from old dag manager + } + + /** + * Action to be performed on a {@link Dag}, in case of a job failure. Currently, we allow 2 modes: + *
    + *
  • FINISH_RUNNING, which allows currently running jobs to finish.
  • + *
  • FINISH_ALL_POSSIBLE, which allows every possible job in the Dag to finish, as long as all the dependencies + * of the job are successful.
  • + *
+ */ + public enum FailureOption { + FINISH_RUNNING("FINISH_RUNNING"), + CANCEL("CANCEL"), + FINISH_ALL_POSSIBLE("FINISH_ALL_POSSIBLE"); + + private final String failureOption; + + FailureOption(final String failureOption) { + this.failureOption = failureOption; + } + + @Override + public String toString() { + return this.failureOption; + } + } + + @Getter + @EqualsAndHashCode + public static class DagId { + String flowGroup; + String flowName; + long flowExecutionId; + public DagId(String flowGroup, String flowName, long flowExecutionId) { + this.flowGroup = flowGroup; + this.flowName = flowName; + this.flowExecutionId = flowExecutionId; + } + + @Override + public String toString() { + return Joiner.on("_").join(flowGroup, flowName, flowExecutionId); + } + } + + @Getter private final Map> dags = new HashMap<>(); + @Getter private final Map, Dag> jobToDag = new HashMap<>(); + @Getter private final Map> dagNodes = new HashMap<>(); + @Getter private final Map>> dagToJobs = new HashMap<>(); + @Getter final Map dagToSLA = new HashMap<>(); + DagManager.DagManagerThread[] dagManagerThreads; + + private final ScheduledExecutorService scheduledExecutorPool; + private Map topologySpecMap = new HashMap<>(); + private final boolean instrumentationEnabled; + @Getter private DagStateStore dagStateStore; + private int houseKeepingThreadInitialDelay = INITIAL_HOUSEKEEPING_THREAD_DELAY; + @Getter + private final Integer numThreads; + protected final Long defaultJobStartSlaTimeMillis; + @Getter + private final JobStatusRetriever jobStatusRetriever; + @Getter private final UserQuotaManager quotaManager; + private final SpecCompiler specCompiler; + private final boolean isFlowConcurrencyEnabled; + private final FlowCompilationValidationHelper flowCompilationValidationHelper; + @Getter private final Optional jobStatusPolledTimer; + @Getter private final Optional eventSubmitter; + // todo implement + private final long failedDagRetentionTime; + @Getter private final DagManagerMetrics dagManagerMetrics; + @Getter private final AtomicLong orchestrationDelay = new AtomicLong(0); + @Getter private final DagProcessingEngine dagProcessingEngine; + + + @Inject(optional=true) + @Getter protected Optional dagActionStore; + + protected final EventBus eventBus; + + public NewDagManager(Config config, JobStatusRetriever jobStatusRetriever, + SharedFlowMetricsSingleton sharedFlowMetricsSingleton, FlowStatusGenerator flowStatusGenerator, + FlowCatalog flowCatalog, Optional dagActionStore, boolean instrumentationEnabled, DagProcessingEngine dagProcessingEngine) { + this.numThreads = ConfigUtils.getInt(config, NUM_THREADS_KEY, DEFAULT_NUM_THREADS); + this.dagActionStore = dagActionStore; + this.scheduledExecutorPool = Executors.newScheduledThreadPool(numThreads); +// this.retentionPollingInterval = ConfigUtils.getInt(config, FAILED_DAG_POLLING_INTERVAL, DEFAULT_FAILED_DAG_POLLING_INTERVAL); + this.instrumentationEnabled = instrumentationEnabled; + this.dagStateStore = createDagStateStore(config, topologySpecMap); + this.eventBus = KafkaJobStatusMonitor.getEventBus(); + this.eventBus.register(this); + this.dagProcessingEngine = dagProcessingEngine; + MetricContext metricContext; + if (instrumentationEnabled) { + metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()), getClass()); + this.jobStatusPolledTimer = Optional.of(metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER)); + this.eventSubmitter = Optional.of(new EventSubmitter.Builder(metricContext, ServiceConstants.SERVICE_NAMESPACE).build()); + } else { + this.jobStatusPolledTimer = Optional.absent(); + this.eventSubmitter = Optional.absent(); + } + this.dagManagerMetrics = new DagManagerMetrics(); + TimeUnit jobStartTimeUnit = TimeUnit.valueOf(ConfigUtils.getString(config, JOB_START_SLA_UNITS, ConfigurationKeys.FALLBACK_GOBBLIN_JOB_START_SLA_TIME_UNIT)); + this.defaultJobStartSlaTimeMillis = jobStartTimeUnit.toMillis(ConfigUtils.getLong(config, JOB_START_SLA_TIME, ConfigurationKeys.FALLBACK_GOBBLIN_JOB_START_SLA_TIME)); + this.jobStatusRetriever = jobStatusRetriever; + this.specCompiler = GobblinConstructorUtils.invokeConstructor(SpecCompiler.class, ConfigUtils.getString(config, + ServiceConfigKeys.GOBBLIN_SERVICE_FLOWCOMPILER_CLASS_KEY, + ServiceConfigKeys.DEFAULT_GOBBLIN_SERVICE_FLOWCOMPILER_CLASS), config); + this.isFlowConcurrencyEnabled = ConfigUtils.getBoolean(config, ServiceConfigKeys.FLOW_CONCURRENCY_ALLOWED, + ServiceConfigKeys.DEFAULT_FLOW_CONCURRENCY_ALLOWED); + this.quotaManager = GobblinConstructorUtils.invokeConstructor(UserQuotaManager.class, + ConfigUtils.getString(config, ServiceConfigKeys.QUOTA_MANAGER_CLASS, ServiceConfigKeys.DEFAULT_QUOTA_MANAGER), + config); + this.flowCompilationValidationHelper = new FlowCompilationValidationHelper(sharedFlowMetricsSingleton, specCompiler, + quotaManager, eventSubmitter, flowStatusGenerator, isFlowConcurrencyEnabled); + TimeUnit timeUnit = TimeUnit.valueOf(ConfigUtils.getString(config, FAILED_DAG_RETENTION_TIME_UNIT, DEFAULT_FAILED_DAG_RETENTION_TIME_UNIT)); + this.failedDagRetentionTime = timeUnit.toMillis(ConfigUtils.getLong(config, FAILED_DAG_RETENTION_TIME, DEFAULT_FAILED_DAG_RETENTION_TIME)); + KillDagThread killDagThread = new KillDagThread(defaultJobStartSlaTimeMillis, this); + this.scheduledExecutorPool.scheduleAtFixedRate(killDagThread, 100L, 60L, TimeUnit.SECONDS); + } + + // todo call it from orchestrator + public synchronized void setTopologySpecMap(Map topologySpecMap) { + this.topologySpecMap = topologySpecMap; + } + + DagStateStore createDagStateStore(Config config, Map topologySpecMap) { + try { + Class dagStateStoreClass = Class.forName(ConfigUtils.getString(config, DAG_STATESTORE_CLASS_KEY, FSDagStateStore.class.getName())); + return (DagStateStore) GobblinConstructorUtils.invokeLongestConstructor(dagStateStoreClass, config, topologySpecMap); + } catch (ReflectiveOperationException e) { + throw new RuntimeException(e); + } + } + + // @Subscribe todo uncomment it when new dag manager is ready + public void handleJobStatusEvent(JobStatusEvent jobStatusEvent) { + Map>> nextSubmitted = Maps.newHashMap(); + List> nodesToCleanUp = Lists.newArrayList(); + + ExecutionStatus executionStatus = jobStatusEvent.getStatus(); + JobStatus jobStatus = jobStatusEvent.getJobStatus(); + + String dagNodeId = DagManagerUtils.generateDagNodeId(jobStatusEvent).toString(); + Dag.DagNode dagNode = this.dagNodes.get(dagNodeId); + + JobExecutionPlan jobExecutionPlan = DagManagerUtils.getJobExecutionPlan(dagNode); + + try { + switch (executionStatus) { + case COMPLETE: + jobExecutionPlan.setExecutionStatus(COMPLETE); + nextSubmitted.putAll(onJobFinish(dagNode)); + nodesToCleanUp.add(dagNode); + break; + case FAILED: + jobExecutionPlan.setExecutionStatus(FAILED); + nextSubmitted.putAll(onJobFinish(dagNode)); + nodesToCleanUp.add(dagNode); + break; + case CANCELLED: + jobExecutionPlan.setExecutionStatus(CANCELLED); + nextSubmitted.putAll(onJobFinish(dagNode)); + nodesToCleanUp.add(dagNode); + break; + case PENDING: + jobExecutionPlan.setExecutionStatus(PENDING); + break; + case PENDING_RETRY: + jobExecutionPlan.setExecutionStatus(PENDING_RETRY); + break; + default: + jobExecutionPlan.setExecutionStatus(RUNNING); + break; + } + + if (jobStatus != null && jobStatus.isShouldRetry()) { + log.info("Retrying job: {}, current attempts: {}, max attempts: {}", DagManagerUtils.getFullyQualifiedJobName(dagNode), + jobStatus.getCurrentAttempts(), jobStatus.getMaxAttempts()); + this.jobToDag.get(dagNode).setFlowEvent(null); + + this.dagProcessingEngine.addDagNodeToRetry(dagNode); + } + } catch (Exception e) { + // Error occurred while processing dag, continue processing other dags assigned to this thread + log.error(String.format("Exception caught in DagManager while processing dag %s due to ", + DagManagerUtils.getFullyQualifiedDagName(dagNode)), e); + } + + for (Map.Entry>> entry: nextSubmitted.entrySet()) { + String nextDagId = entry.getKey(); + Set> dagNodes = entry.getValue(); + for (Dag.DagNode nextDagNode: dagNodes) { + addJobState(nextDagId, nextDagNode); + } + } + + for (Dag.DagNode dagNodeToClean: nodesToCleanUp) { + String dagId = DagManagerUtils.generateDagId(dagNodeToClean).toString(); + deleteJobState(dagId, dagNodeToClean); + } + } + + void deleteJobState(String dagId, Dag.DagNode dagNode) { + this.jobToDag.remove(dagNode); + this.dagToJobs.get(dagId).remove(dagNode); + this.dagToSLA.remove(dagId); + } + + private void addJobState(String dagId, Dag.DagNode dagNode) { + Dag dag = this.dags.get(dagId); + this.jobToDag.put(dagNode, dag); + if (this.dagToJobs.containsKey(dagId)) { + this.dagToJobs.get(dagId).add(dagNode); + } else { + LinkedList> dagNodeList = Lists.newLinkedList(); + dagNodeList.add(dagNode); + this.dagToJobs.put(dagId, dagNodeList); + } + } + + /** + * Method that defines the actions to be performed when a job finishes either successfully or with failure. + * This method updates the state of the dag and performs clean up actions as necessary. + */ + Map>> onJobFinish(Dag.DagNode dagNode) + throws IOException { + Dag dag = this.jobToDag.get(dagNode); + String dagId = DagManagerUtils.generateDagId(dag).toString(); + String jobName = DagManagerUtils.getFullyQualifiedJobName(dagNode); + ExecutionStatus jobStatus = DagManagerUtils.getExecutionStatus(dagNode); + log.info("Job {} of Dag {} has finished with status {}", jobName, dagId, jobStatus.name()); + // Only decrement counters and quota for jobs that actually ran on the executor, not from a GaaS side failure/skip event + if (quotaManager.releaseQuota(dagNode)) { + dagManagerMetrics.decrementRunningJobMetrics(dagNode); + } + + switch (jobStatus) { + case FAILED: + dag.setMessage("Flow failed because job " + jobName + " failed"); + dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_FAILED); + dagManagerMetrics.incrementExecutorFailed(dagNode); + return Maps.newHashMap(); + case CANCELLED: + dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_CANCELLED); + return Maps.newHashMap(); + case COMPLETE: + dagManagerMetrics.incrementExecutorSuccess(dagNode); + return submitNext(dagId); + default: + log.warn("It should not reach here. Job status is unexpected."); + return Maps.newHashMap(); + } + } + + /** + * Submit next set of Dag nodes in the Dag identified by the provided dagId + * @param dagId The dagId that should be processed. + * @return + * @throws IOException + */ + synchronized Map>> submitNext(String dagId) throws IOException { + Dag dag = this.dags.get(dagId); + Set> nextNodes = DagManagerUtils.getNext(dag); + List nextJobNames = new ArrayList<>(); + + //Submit jobs from the dag ready for execution. + for (Dag.DagNode dagNode : nextNodes) { + this.dagProcessingEngine.addAdvanceDagTask(dagNode); + nextJobNames.add(DagManagerUtils.getJobName(dagNode)); + } + log.info("Submitting next nodes for dagId {}, where next jobs to be submitted are {}", dagId, nextJobNames); + //Checkpoint the dag state + this.dagStateStore.writeCheckpoint(dag); + + Map>> dagIdToNextJobs = Maps.newHashMap(); + dagIdToNextJobs.put(dagId, nextNodes); + return dagIdToNextJobs; + } + + @Override + protected void startUp() + throws Exception { + // do nothing + } + + @Override + protected void shutDown() + throws Exception { + this.scheduledExecutorPool.shutdown(); + this.scheduledExecutorPool.awaitTermination(TERMINATION_TIMEOUT, TimeUnit.SECONDS); + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java index bcb1743200a..41a70ee985c 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java @@ -57,6 +57,7 @@ import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse; import org.apache.gobblin.runtime.spec_catalog.TopologyCatalog; import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; import org.apache.gobblin.service.modules.flow.SpecCompiler; import org.apache.gobblin.service.modules.flowgraph.Dag; import org.apache.gobblin.service.modules.spec.JobExecutionPlan; @@ -83,6 +84,7 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable { protected final SpecCompiler specCompiler; protected final Optional topologyCatalog; protected final Optional dagManager; + protected final Optional dagProcessingEngine; protected final MetricContext metricContext; @@ -107,12 +109,14 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable { private final ClassAliasResolver aliasResolver; public Orchestrator(Config config, Optional topologyCatalog, Optional dagManager, - Optional log, FlowStatusGenerator flowStatusGenerator, boolean instrumentationEnabled, - Optional flowTriggerHandler, SharedFlowMetricsSingleton sharedFlowMetricsSingleton) { + Optional dagProcessingEngine, Optional log, FlowStatusGenerator flowStatusGenerator, + boolean instrumentationEnabled, Optional flowTriggerHandler, + SharedFlowMetricsSingleton sharedFlowMetricsSingleton) { _log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass()); this.aliasResolver = new ClassAliasResolver<>(SpecCompiler.class); this.topologyCatalog = topologyCatalog; this.dagManager = dagManager; + this.dagProcessingEngine = dagProcessingEngine; this.flowStatusGenerator = flowStatusGenerator; this.flowTriggerHandler = flowTriggerHandler; this.sharedFlowMetricsSingleton = sharedFlowMetricsSingleton; @@ -140,7 +144,7 @@ public Orchestrator(Config config, Optional topologyCatalog, Op this.flowOrchestrationFailedMeter = Optional.of(this.metricContext.meter(ServiceMetricNames.FLOW_ORCHESTRATION_FAILED_METER)); this.flowOrchestrationTimer = Optional.of(this.metricContext.timer(ServiceMetricNames.FLOW_ORCHESTRATION_TIMER)); this.flowFailedForwardToDagManagerCounter = Optional.of(this.metricContext.counter(ServiceMetricNames.FLOW_FAILED_FORWARD_TO_DAG_MANAGER_COUNT)); - this.eventSubmitter = Optional.of(new EventSubmitter.Builder(this.metricContext, "org.apache.gobblin.service").build()); + this.eventSubmitter = Optional.of(new EventSubmitter.Builder(this.metricContext, ServiceConstants.SERVICE_NAMESPACE).build()); } else { this.metricContext = null; this.flowOrchestrationSuccessFulMeter = Optional.absent(); @@ -160,10 +164,10 @@ public Orchestrator(Config config, Optional topologyCatalog, Op @Inject public Orchestrator(Config config, FlowStatusGenerator flowStatusGenerator, Optional topologyCatalog, - Optional dagManager, Optional log, Optional flowTriggerHandler, - SharedFlowMetricsSingleton sharedFlowMetricsSingleton) { - this(config, topologyCatalog, dagManager, log, flowStatusGenerator, true, flowTriggerHandler, - sharedFlowMetricsSingleton); + Optional dagManager, Optional dagProcessingEngine, Optional log, + Optional flowTriggerHandler, SharedFlowMetricsSingleton sharedFlowMetricsSingleton) { + this(config, topologyCatalog, dagManager, dagProcessingEngine, log, flowStatusGenerator, true, + flowTriggerHandler, sharedFlowMetricsSingleton); } @@ -266,7 +270,7 @@ public void orchestrate(Spec spec, Properties jobProps, long triggerTimestampMil return; } - String flowExecutionId = flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD); + long flowExecutionId = Long.parseLong(flowMetadata.get(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD)); DagActionStore.DagAction flowAction = new DagActionStore.DagAction(flowGroup, flowName, flowExecutionId, DagActionStore.FlowActionType.LAUNCH); flowTriggerHandler.get().handleTriggerEvent(jobProps, flowAction, triggerTimestampMillis, isReminderEvent); @@ -274,7 +278,7 @@ public void orchestrate(Spec spec, Properties jobProps, long triggerTimestampMil flowAction, isReminderEvent ? "reminder" : "original", triggerTimestampMillis); } else { Dag jobExecutionPlanDag = jobExecutionPlanDagOptional.get(); - if (jobExecutionPlanDag == null || jobExecutionPlanDag.isEmpty()) { + if (jobExecutionPlanDag.isEmpty()) { FlowCompilationValidationHelper.populateFlowCompilationFailedEventMessage(eventSubmitter, spec, flowMetadata); Instrumented.markMeter(this.flowOrchestrationFailedMeter); sharedFlowMetricsSingleton.conditionallyUpdateFlowGaugeSpecState(spec, @@ -290,42 +294,7 @@ public void orchestrate(Spec spec, Properties jobProps, long triggerTimestampMil flowCompilationTimer.get().stop(flowMetadata); } - // Depending on if DagManager is present, handle execution - if (this.dagManager.isPresent()) { - submitFlowToDagManager((FlowSpec) spec, jobExecutionPlanDag); - } else { - // Schedule all compiled JobSpecs on their respective Executor - for (Dag.DagNode dagNode : jobExecutionPlanDag.getNodes()) { - DagManagerUtils.incrementJobAttempt(dagNode); - JobExecutionPlan jobExecutionPlan = dagNode.getValue(); - - // Run this spec on selected executor - SpecProducer producer = null; - try { - producer = jobExecutionPlan.getSpecExecutor().getProducer().get(); - Spec jobSpec = jobExecutionPlan.getJobSpec(); - - if (!((JobSpec) jobSpec).getConfig().hasPath(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)) { - _log.warn("JobSpec does not contain flowExecutionId."); - } - - Map jobMetadata = TimingEventUtils.getJobMetadata(flowMetadata, jobExecutionPlan); - _log.info(String.format("Going to orchestrate JobSpec: %s on Executor: %s", jobSpec, producer)); - - Optional jobOrchestrationTimer = this.eventSubmitter.transform( - submitter -> new TimingEvent(submitter, TimingEvent.LauncherTimings.JOB_ORCHESTRATED)); - - producer.addSpec(jobSpec); - - if (jobOrchestrationTimer.isPresent()) { - jobOrchestrationTimer.get().stop(jobMetadata); - } - } catch (Exception e) { - _log.error("Cannot successfully setup spec: " + jobExecutionPlan.getJobSpec() + " on executor: " + producer - + " for flow: " + spec, e); - } - } - } + submitFlowToDagManager((FlowSpec) spec, jobExecutionPlanDag); } } else { Instrumented.markMeter(this.flowOrchestrationFailedMeter); @@ -348,9 +317,14 @@ public void submitFlowToDagManager(FlowSpec flowSpec) throws IOException, Interr public void submitFlowToDagManager(FlowSpec flowSpec, Dag jobExecutionPlanDag) throws IOException { + // arjun, here instead add to dagprocengine / dagtaskstream try { - //Send the dag to the DagManager. - this.dagManager.get().addDag(jobExecutionPlanDag, true, true); + if (this.dagProcessingEngine.isPresent()) { + this.dagProcessingEngine.get().addNewDag(jobExecutionPlanDag); + } else { + //Send the dag to the DagManager. + this.dagManager.get().addDag(jobExecutionPlanDag, true, true); + } } catch (Exception ex) { String failureMessage = "Failed to add Job Execution Plan due to: " + ex.getMessage(); _log.warn("Orchestrator call - " + failureMessage, ex); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/TimingEventUtils.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/TimingEventUtils.java index 99661305fd3..dc04e0a53f5 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/TimingEventUtils.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/TimingEventUtils.java @@ -47,7 +47,7 @@ static Map getFlowMetadata(Config flowConfig) { return metadata; } - static Map getJobMetadata(Map flowMetadata, JobExecutionPlan jobExecutionPlan) { + public static Map getJobMetadata(Map flowMetadata, JobExecutionPlan jobExecutionPlan) { Map jobMetadata = Maps.newHashMap(); JobSpec jobSpec = jobExecutionPlan.getJobSpec(); SpecExecutor specExecutor = jobExecutionPlan.getSpecExecutor(); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/AdvanceDagProc.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/AdvanceDagProc.java new file mode 100644 index 00000000000..43778f6c1bf --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/AdvanceDagProc.java @@ -0,0 +1,139 @@ +/* + * 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.proc; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.Future; + +import com.google.common.base.Optional; +import com.google.common.collect.Maps; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.runtime.api.JobSpec; +import org.apache.gobblin.runtime.api.Spec; +import org.apache.gobblin.runtime.api.SpecProducer; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.DagManagerUtils; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.TimingEventUtils; +import org.apache.gobblin.service.modules.orchestration.task.AdvanceDagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + +import static org.apache.gobblin.service.ExecutionStatus.RUNNING; + + +/** + * An implementation of {@link DagProc} dealing with advancing to the next node in the {@link Dag}. + * This Dag Procedure will deal with pending Job statuses such as: PENDING, PENDING_RESUME, PENDING_RETRY + * as well jobs that have reached an end state with statuses such as: COMPLETED, FAILED and CANCELLED. + * Primarily, it will be responsible for polling the flow and job statuses and advancing to the next node in the dag. + * + */ +@Slf4j +@Alpha +public class AdvanceDagProc extends DagProc { + private AdvanceDagTask advanceDagTask; + + public AdvanceDagProc(AdvanceDagTask advanceDagTask, DagProcFactory dagProcFactory) { + super(dagProcFactory); + this.advanceDagTask = advanceDagTask; + } + +// @Override +// protected Object initialize(DagManagementStateStore dagManagementStateStore) throws MaybeRetryableException, IOException { +// throw new UnsupportedOperationException("Not supported"); +// } +// +// @Override +// protected Object act(Object state, DagManagementStateStore dagManagementStateStore) throws MaybeRetryableException, Exception { +// throw new UnsupportedOperationException("Not supported"); +// } +// +// @Override +// protected void sendNotification(Object result, EventSubmitter eventSubmitter) throws MaybeRetryableException, IOException { +// throw new UnsupportedOperationException("Not supported"); +// } + + @Override + public void process(AdvanceDagTask dagTask) { + // send dag node to orchestrater + // sla kill if needed + // find next dag nodes to submit and create next AdvanceDagTasks for them + submitJob(this.advanceDagTask.getDagNode()); } + private void submitJob(Dag.DagNode dagNode) { + // de duplicate it with LaunchDagProc + DagManagerUtils.incrementJobAttempt(dagNode); + JobExecutionPlan jobExecutionPlan = DagManagerUtils.getJobExecutionPlan(dagNode); + jobExecutionPlan.setExecutionStatus(RUNNING); + JobSpec jobSpec = DagManagerUtils.getJobSpec(dagNode); + Map jobMetadata = TimingEventUtils.getJobMetadata(Maps.newHashMap(), jobExecutionPlan); + + String specExecutorUri = DagManagerUtils.getSpecExecutorUri(dagNode); + + // Run this spec on selected executor + SpecProducer producer; + try { + this.quotaManager.checkQuota(Collections.singleton(dagNode)); + + producer = DagManagerUtils.getSpecProducer(dagNode); + TimingEvent jobOrchestrationTimer = this.eventSubmitter.isPresent() ? this.eventSubmitter.get(). + getTimingEvent(TimingEvent.LauncherTimings.JOB_ORCHESTRATED) : null; + + // Increment job count before submitting the job onto the spec producer, in case that throws an exception. + // By this point the quota is allocated, so it's imperative to increment as missing would introduce the potential to decrement below zero upon quota release. + // Quota release is guaranteed, despite failure, because exception handling within would mark the job FAILED. + // When the ensuing kafka message spurs DagManager processing, the quota is released and the counts decremented + // Ensure that we do not double increment for flows that are retried + if (dagNode.getValue().getCurrentAttempts() == 1) { + dagManagerMetrics.incrementRunningJobMetrics(dagNode); + } + // Submit the job to the SpecProducer, which in turn performs the actual job submission to the SpecExecutor instance. + // The SpecProducer implementations submit the job to the underlying executor and return when the submission is complete, + // either successfully or unsuccessfully. To catch any exceptions in the job submission, the DagManagerThread + // blocks (by calling Future#get()) until the submission is completed. + Future addSpecFuture = producer.addSpec(jobSpec); + dagNode.getValue().setJobFuture(Optional.of(addSpecFuture)); + //Persist the dag + this.dagStateStore.writeCheckpoint(this.dags.get(DagManagerUtils.generateDagId(dagNode).toString())); + + addSpecFuture.get(); + + jobMetadata.put(TimingEvent.METADATA_MESSAGE, producer.getExecutionLink(addSpecFuture, specExecutorUri)); + // Add serialized job properties as part of the orchestrated job event metadata + jobMetadata.put(JobExecutionPlan.JOB_PROPS_KEY, dagNode.getValue().toString()); + if (jobOrchestrationTimer != null) { + jobOrchestrationTimer.stop(jobMetadata); + } + log.info("Orchestrated job: {} on Executor: {}", DagManagerUtils.getFullyQualifiedJobName(dagNode), specExecutorUri); + this.dagManagerMetrics.incrementJobsSentToExecutor(dagNode); + } catch (Exception e) { + TimingEvent jobFailedTimer = this.eventSubmitter.isPresent() ? this.eventSubmitter.get(). + getTimingEvent(TimingEvent.LauncherTimings.JOB_FAILED) : null; + String message = "Cannot submit job " + DagManagerUtils.getFullyQualifiedJobName(dagNode) + " on executor " + specExecutorUri; + log.error(message, e); + jobMetadata.put(TimingEvent.METADATA_MESSAGE, message + " due to " + e.getMessage()); + if (jobFailedTimer != null) { + jobFailedTimer.stop(jobMetadata); + } + } + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/CleanUpDagProc.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/CleanUpDagProc.java new file mode 100644 index 00000000000..3dc1daa7acc --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/CleanUpDagProc.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration.proc; + +import java.io.IOException; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.task.CleanUpDagTask; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; + + +/** + * An implementation of {@link DagProc} that is responsible for cleaning up {@link Dag} that has reached an end state + * i.e. FAILED, COMPLETE or CANCELED + * + */ +@Slf4j +@Alpha +public class CleanUpDagProc extends DagProc { + + private CleanUpDagTask cleanUpDagTask; + + public CleanUpDagProc(CleanUpDagTask cleanUpDagTask, DagProcFactory dagProcFactory) { + super(dagProcFactory); + this.cleanUpDagTask = cleanUpDagTask; + } + + @Override + public void process(DagTask dagTask) throws IOException { + + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProc.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProc.java new file mode 100644 index 00000000000..b9160b93c72 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProc.java @@ -0,0 +1,77 @@ +/* + * 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.proc; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import com.google.common.base.Optional; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.DagManagerMetrics; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.DagStateStore; +import org.apache.gobblin.service.modules.orchestration.NewDagManager; +import org.apache.gobblin.service.modules.orchestration.UserQuotaManager; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + + +/** + * Responsible to performing the actual work for a given {@link DagTask}. + * It processes the {@link DagTask} by first initializing its state, performing actions + * based on the type of {@link DagTask} and finally submitting an event to the executor. + */ +@Alpha +@Slf4j +@Getter +public abstract class DagProc { + protected final DagProcFactory dagProcFactory; + + public final Map> dags; + private final Map, Dag> jobToDag; + private final Map>> dagToJobs; + protected final UserQuotaManager quotaManager; + protected final Optional eventSubmitter; + protected DagStateStore dagStateStore; + private final AtomicLong orchestrationDelay; + private final NewDagManager dagManager; + protected final DagManagerMetrics dagManagerMetrics = new DagManagerMetrics(); + + public DagProc(DagProcFactory dagProcFactory) { + // todo make it cleaner + this.dagProcFactory = dagProcFactory; + this.dagManager = dagProcFactory.dagManager; + this.dags = this.dagManager.getDags(); + this.jobToDag = this.dagManager.getJobToDag(); + this.dagToJobs = this.dagManager.getDagToJobs(); + this.quotaManager = this.dagManager.getQuotaManager(); + this.eventSubmitter = this.dagManager.getEventSubmitter(); + this.dagStateStore = this.dagManager.getDagStateStore(); + this.orchestrationDelay = this.dagManager.getOrchestrationDelay(); + } + + abstract public void process(T dagTask) throws IOException; +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/KillDagProc.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/KillDagProc.java new file mode 100644 index 00000000000..0d33f13b21c --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/KillDagProc.java @@ -0,0 +1,108 @@ +/* + * 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.proc; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +import com.google.common.collect.Maps; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.DagManagerUtils; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.TimingEventUtils; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; +import org.apache.gobblin.service.modules.orchestration.task.KillDagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + +import static org.apache.gobblin.service.ExecutionStatus.CANCELLED; + + +/** + * An implementation of {@link DagProc} for killing {@link DagTask}. + */ +@Slf4j +@Alpha +public final class KillDagProc extends DagProc { + + private KillDagTask killDagTask; + + public KillDagProc(KillDagTask killDagTask, DagProcFactory dagProcFactory) { + super(dagProcFactory); + this.killDagTask = killDagTask; + } + + @Override + public void process(KillDagTask dagTask) throws IOException { + if (dagTask.getDagId() != null) { + if (this.dagProcFactory.dagManager.getDagToJobs().containsKey(dagTask.getDagId().toString())) { + List> dagNodesToCancel = + this.dagProcFactory.dagManager.getDagToJobs().get(dagTask.getDagId().toString()); + for (Dag.DagNode dagNodeToCancel : dagNodesToCancel) { + cancelDagNode(dagNodeToCancel); + } + } else { + log.warn("Dag to cancel " + dagTask.getDagId().toString() + " not found."); + } + } + if (dagTask.getDagNodeId() != null) { + if (this.dagProcFactory.dagManager.getDagNodes().containsKey(dagTask.getDagNodeId().toString())) { + cancelDagNode(this.dagProcFactory.dagManager.getDagNodes().get(dagTask.getDagNodeId().toString())); + } else { + log.warn("Dag node to cancel " + dagTask.getDagNodeId().toString() + " not found."); + } + } + } + + private void cancelDagNode(Dag.DagNode dagNodeToCancel) throws IOException { + Properties props = new Properties(); + try { + if (dagNodeToCancel.getValue().getJobFuture().isPresent()) { + Future future = dagNodeToCancel.getValue().getJobFuture().get(); + String serializedFuture = DagManagerUtils.getSpecProducer(dagNodeToCancel).serializeAddSpecResponse(future); + props.put(ConfigurationKeys.SPEC_PRODUCER_SERIALIZED_FUTURE, serializedFuture); + sendCancellationEvent(dagNodeToCancel.getValue()); + } + if (dagNodeToCancel.getValue().getJobSpec().getConfig().hasPath(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)) { + props.setProperty(ConfigurationKeys.FLOW_EXECUTION_ID_KEY, + dagNodeToCancel.getValue().getJobSpec().getConfig().getString(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)); + } + DagManagerUtils.getSpecProducer(dagNodeToCancel).cancelJob(dagNodeToCancel.getValue().getJobSpec().getUri(), props); + } catch (ExecutionException | InterruptedException e) { + throw new IOException(e); + } + } + + + private void sendCancellationEvent(JobExecutionPlan jobExecutionPlan) { + Map jobMetadata = TimingEventUtils.getJobMetadata(Maps.newHashMap(), jobExecutionPlan); + this.eventSubmitter.get().getTimingEvent(TimingEvent.LauncherTimings.JOB_CANCEL).stop(jobMetadata); + jobExecutionPlan.setExecutionStatus(CANCELLED); + } + +} + diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/LaunchDagProc.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/LaunchDagProc.java new file mode 100644 index 00000000000..5875dbdd26d --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/LaunchDagProc.java @@ -0,0 +1,248 @@ +/* + * 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.proc; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicLong; + +import com.google.common.base.Optional; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.metrics.event.EventSubmitter; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.runtime.api.JobSpec; +import org.apache.gobblin.runtime.api.Spec; +import org.apache.gobblin.runtime.api.SpecProducer; +import org.apache.gobblin.service.FlowId; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.DagManager; +import org.apache.gobblin.service.modules.orchestration.DagManagerUtils; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.DagStateStore; +import org.apache.gobblin.service.modules.orchestration.TimingEventUtils; +import org.apache.gobblin.service.modules.orchestration.UserQuotaManager; +import org.apache.gobblin.service.modules.orchestration.task.AdvanceDagTask; +import org.apache.gobblin.service.modules.orchestration.task.LaunchDagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + +import static org.apache.gobblin.service.ExecutionStatus.RUNNING; + + +/** + * An implementation of {@link DagProc} for launching {@link org.apache.gobblin.service.modules.orchestration.task.DagTask}. + */ +@Slf4j +@Alpha +public final class LaunchDagProc extends DagProc { + private LaunchDagTask launchDagTask; + + public final Map> dags; + private final Map, Dag> jobToDag; + private final Map>> dagToJobs; + private final UserQuotaManager quotaManager; + private final Optional eventSubmitter; + private DagStateStore dagStateStore; + private final AtomicLong orchestrationDelay; + + public LaunchDagProc(LaunchDagTask launchDagTask, DagProcFactory dagProcFactory) { + super(dagProcFactory); + this.launchDagTask = launchDagTask; + this.dags = dagProcFactory.dagManager.getDags(); + this.jobToDag = dagProcFactory.dagManager.getJobToDag(); + this.dagToJobs = dagProcFactory.dagManager.getDagToJobs(); + this.quotaManager = dagProcFactory.dagManager.getQuotaManager(); + this.eventSubmitter = dagProcFactory.dagManager.getEventSubmitter(); + this.dagStateStore = dagProcFactory.dagManager.getDagStateStore(); + this.orchestrationDelay = dagProcFactory.dagManager.getOrchestrationDelay(); + } + + @Override + public void process(LaunchDagTask dagTask) throws IOException { + initializeDag(dagTask.getDag()); + persistInDagStore(this.launchDagTask.getDag()); + for (Dag.DagNode dagNode : this.launchDagTask.getDag().getStartNodes()) { + this.dagProcFactory.dagProcessingEngine.addDagTask(new AdvanceDagTask(dagNode)); + } + } + + private void initializeDag(Dag dag) + throws IOException { + //Add Dag to the map of running dags + String dagId = DagManagerUtils.generateDagId(dag).toString(); + log.info("Initializing Dag {}", DagManagerUtils.getFullyQualifiedDagName(dag)); + if (this.dags.containsKey(dagId)) { + log.warn("Already tracking a dag with dagId {}, skipping.", dagId); + return; + } + + this.dags.put(dagId, dag); + log.debug("Dag {} - determining if any jobs are already running.", DagManagerUtils.getFullyQualifiedDagName(dag)); + + //A flag to indicate if the flow is already running. + boolean isDagRunning = false; + //Are there any jobs already in the running state? This check is for Dags already running + //before a leadership change occurs. + for (Dag.DagNode dagNode : dag.getNodes()) { + if (DagManagerUtils.getExecutionStatus(dagNode) == RUNNING) { + addJobState(dagId, dagNode); + //Update the running jobs counter. + dagManagerMetrics.incrementRunningJobMetrics(dagNode); + isDagRunning = true; + } + } + + FlowId flowId = DagManagerUtils.getFlowId(dag); + this.dagManagerMetrics.registerFlowMetric(flowId, dag); + + log.debug("Dag {} submitting jobs ready for execution.", DagManagerUtils.getFullyQualifiedDagName(dag)); + //Determine the next set of jobs to run and submit them for execution + Map>> nextSubmitted = submitNext(dagId); + for (Dag.DagNode dagNode : nextSubmitted.get(dagId)) { + addJobState(dagId, dagNode); + } + + // Set flow status to running + DagManagerUtils.emitFlowEvent(this.eventSubmitter, dag, TimingEvent.FlowTimings.FLOW_RUNNING); + dagManagerMetrics.conditionallyMarkFlowAsState(flowId, DagManager.FlowState.RUNNING); + + // Report the orchestration delay the first time the Dag is initialized. Orchestration delay is defined as + // the time difference between the instant when a flow first transitions to the running state and the instant + // when the flow is submitted to Gobblin service. + if (!isDagRunning) { + this.orchestrationDelay.set(System.currentTimeMillis() - DagManagerUtils.getFlowExecId(dag)); + } + + log.info("Dag {} Initialization complete.", DagManagerUtils.getFullyQualifiedDagName(dag)); + } + + private void addJobState(String dagId, Dag.DagNode dagNode) { + Dag dag = this.dags.get(dagId); + this.jobToDag.put(dagNode, dag); + if (this.dagToJobs.containsKey(dagId)) { + this.dagToJobs.get(dagId).add(dagNode); + } else { + LinkedList> dagNodeList = Lists.newLinkedList(); + dagNodeList.add(dagNode); + this.dagToJobs.put(dagId, dagNodeList); + } + } + + /** + * Submit next set of Dag nodes in the Dag identified by the provided dagId + * @param dagId The dagId that should be processed. + * @return + * @throws IOException + */ + synchronized Map>> submitNext(String dagId) + throws IOException { + Dag dag = this.dags.get(dagId); + Set> nextNodes = DagManagerUtils.getNext(dag); + List nextJobNames = new ArrayList<>(); + + //Submit jobs from the dag ready for execution. + for (Dag.DagNode dagNode : nextNodes) { + submitJob(dagNode); + nextJobNames.add(DagManagerUtils.getJobName(dagNode)); + } + + log.info("Submitting next nodes for dagId {}, where next jobs to be submitted are {}", dagId, nextJobNames); + //Checkpoint the dag state + this.dagStateStore.writeCheckpoint(dag); + + Map>> dagIdToNextJobs = Maps.newHashMap(); + dagIdToNextJobs.put(dagId, nextNodes); + return dagIdToNextJobs; + } + + + /** + * Submits a {@link JobSpec} to a {@link org.apache.gobblin.runtime.api.SpecExecutor}. + */ + private void submitJob(Dag.DagNode dagNode) { + DagManagerUtils.incrementJobAttempt(dagNode); + JobExecutionPlan jobExecutionPlan = DagManagerUtils.getJobExecutionPlan(dagNode); + jobExecutionPlan.setExecutionStatus(RUNNING); + JobSpec jobSpec = DagManagerUtils.getJobSpec(dagNode); + Map jobMetadata = TimingEventUtils.getJobMetadata(Maps.newHashMap(), jobExecutionPlan); + + String specExecutorUri = DagManagerUtils.getSpecExecutorUri(dagNode); + + // Run this spec on selected executor + SpecProducer producer; + try { + quotaManager.checkQuota(Collections.singleton(dagNode)); + + producer = DagManagerUtils.getSpecProducer(dagNode); + TimingEvent jobOrchestrationTimer = this.eventSubmitter.isPresent() ? this.eventSubmitter.get(). + getTimingEvent(TimingEvent.LauncherTimings.JOB_ORCHESTRATED) : null; + + // Increment job count before submitting the job onto the spec producer, in case that throws an exception. + // By this point the quota is allocated, so it's imperative to increment as missing would introduce the potential to decrement below zero upon quota release. + // Quota release is guaranteed, despite failure, because exception handling within would mark the job FAILED. + // When the ensuing kafka message spurs DagManager processing, the quota is released and the counts decremented + // Ensure that we do not double increment for flows that are retried + if (dagNode.getValue().getCurrentAttempts() == 1) { + dagManagerMetrics.incrementRunningJobMetrics(dagNode); + } + // Submit the job to the SpecProducer, which in turn performs the actual job submission to the SpecExecutor instance. + // The SpecProducer implementations submit the job to the underlying executor and return when the submission is complete, + // either successfully or unsuccessfully. To catch any exceptions in the job submission, the DagManagerThread + // blocks (by calling Future#get()) until the submission is completed. + Future addSpecFuture = producer.addSpec(jobSpec); + dagNode.getValue().setJobFuture(Optional.of(addSpecFuture)); + //Persist the dag + this.dagStateStore.writeCheckpoint(this.dags.get(DagManagerUtils.generateDagId(dagNode).toString())); + + addSpecFuture.get(); + + jobMetadata.put(TimingEvent.METADATA_MESSAGE, producer.getExecutionLink(addSpecFuture, specExecutorUri)); + // Add serialized job properties as part of the orchestrated job event metadata + jobMetadata.put(JobExecutionPlan.JOB_PROPS_KEY, dagNode.getValue().toString()); + if (jobOrchestrationTimer != null) { + jobOrchestrationTimer.stop(jobMetadata); + } + log.info("Orchestrated job: {} on Executor: {}", DagManagerUtils.getFullyQualifiedJobName(dagNode), specExecutorUri); + this.dagManagerMetrics.incrementJobsSentToExecutor(dagNode); + } catch (Exception e) { + TimingEvent jobFailedTimer = this.eventSubmitter.isPresent() ? this.eventSubmitter.get(). + getTimingEvent(TimingEvent.LauncherTimings.JOB_FAILED) : null; + String message = "Cannot submit job " + DagManagerUtils.getFullyQualifiedJobName(dagNode) + " on executor " + specExecutorUri; + log.error(message, e); + jobMetadata.put(TimingEvent.METADATA_MESSAGE, message + " due to " + e.getMessage()); + if (jobFailedTimer != null) { + jobFailedTimer.stop(jobMetadata); + } + } + } + + private void persistInDagStore(Dag dag) { + // TODO : implement it, get some code from old dag manager + this.dagProcFactory.dagManager.addDag(dag); + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ResumeDagProc.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ResumeDagProc.java new file mode 100644 index 00000000000..dd08a6dc2ac --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ResumeDagProc.java @@ -0,0 +1,49 @@ +/* + * 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.proc; + +import java.io.IOException; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.task.DagTask; +import org.apache.gobblin.service.modules.orchestration.task.ResumeDagTask; + + +/** + * An implementation of {@link DagProc} for resuming {@link org.apache.gobblin.service.modules.orchestration.task.DagTask}. + */ + +@Alpha +@Slf4j +public final class ResumeDagProc extends DagProc { + + private ResumeDagTask resumeDagTask; + + public ResumeDagProc(ResumeDagTask resumeDagTask, DagProcFactory dagProcFactory) { + super(dagProcFactory); + this.resumeDagTask = resumeDagTask; + } + + @Override + public void process(DagTask dagTask) throws IOException { + // todo implement it + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/RetryDagProc.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/RetryDagProc.java new file mode 100644 index 00000000000..8dd1a8056c4 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/RetryDagProc.java @@ -0,0 +1,127 @@ +/* + * 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.proc; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.Future; + +import com.google.common.base.Optional; +import com.google.common.collect.Maps; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.runtime.api.JobSpec; +import org.apache.gobblin.runtime.api.Spec; +import org.apache.gobblin.runtime.api.SpecProducer; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.DagManagerUtils; +import org.apache.gobblin.service.modules.orchestration.DagProcFactory; +import org.apache.gobblin.service.modules.orchestration.TimingEventUtils; +import org.apache.gobblin.service.modules.orchestration.task.AdvanceDagTask; +import org.apache.gobblin.service.modules.orchestration.task.RetryDagTask; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + +import static org.apache.gobblin.service.ExecutionStatus.RUNNING; + + +/** + * An implementation of {@link DagProc} for launching {@link org.apache.gobblin.service.modules.orchestration.task.DagTask}. + */ +@Slf4j +@Alpha +public final class RetryDagProc extends DagProc { + + private RetryDagTask retryDagTask; + + + public RetryDagProc(RetryDagTask retryDagTask, DagProcFactory dagProcFactory) { + super(dagProcFactory); + this.retryDagTask = retryDagTask; + } + + @Override + public void process(RetryDagTask dagTask) throws IOException { + submitJob(this.retryDagTask.getDagNode()); + this.dagProcFactory.dagProcessingEngine.addDagTask(new AdvanceDagTask(this.retryDagTask.getDagNode())); + } + + /** + * Submits a {@link JobSpec} to a {@link org.apache.gobblin.runtime.api.SpecExecutor}. + */ + private void submitJob(Dag.DagNode dagNode) { + // de duplicate it with LaunchDagProc + DagManagerUtils.incrementJobAttempt(dagNode); + JobExecutionPlan jobExecutionPlan = DagManagerUtils.getJobExecutionPlan(dagNode); + jobExecutionPlan.setExecutionStatus(RUNNING); + JobSpec jobSpec = DagManagerUtils.getJobSpec(dagNode); + Map jobMetadata = TimingEventUtils.getJobMetadata(Maps.newHashMap(), jobExecutionPlan); + + String specExecutorUri = DagManagerUtils.getSpecExecutorUri(dagNode); + + // Run this spec on selected executor + SpecProducer producer; + try { + quotaManager.checkQuota(Collections.singleton(dagNode)); + + producer = DagManagerUtils.getSpecProducer(dagNode); + TimingEvent jobOrchestrationTimer = this.eventSubmitter.isPresent() ? this.eventSubmitter.get(). + getTimingEvent(TimingEvent.LauncherTimings.JOB_ORCHESTRATED) : null; + + // Increment job count before submitting the job onto the spec producer, in case that throws an exception. + // By this point the quota is allocated, so it's imperative to increment as missing would introduce the potential to decrement below zero upon quota release. + // Quota release is guaranteed, despite failure, because exception handling within would mark the job FAILED. + // When the ensuing kafka message spurs DagManager processing, the quota is released and the counts decremented + // Ensure that we do not double increment for flows that are retried + if (dagNode.getValue().getCurrentAttempts() == 1) { + dagManagerMetrics.incrementRunningJobMetrics(dagNode); + } + // Submit the job to the SpecProducer, which in turn performs the actual job submission to the SpecExecutor instance. + // The SpecProducer implementations submit the job to the underlying executor and return when the submission is complete, + // either successfully or unsuccessfully. To catch any exceptions in the job submission, the DagManagerThread + // blocks (by calling Future#get()) until the submission is completed. + Future addSpecFuture = producer.addSpec(jobSpec); + dagNode.getValue().setJobFuture(Optional.of(addSpecFuture)); + //Persist the dag + this.dagStateStore.writeCheckpoint(this.dags.get(DagManagerUtils.generateDagId(dagNode).toString())); + + addSpecFuture.get(); + + jobMetadata.put(TimingEvent.METADATA_MESSAGE, producer.getExecutionLink(addSpecFuture, specExecutorUri)); + // Add serialized job properties as part of the orchestrated job event metadata + jobMetadata.put(JobExecutionPlan.JOB_PROPS_KEY, dagNode.getValue().toString()); + if (jobOrchestrationTimer != null) { + jobOrchestrationTimer.stop(jobMetadata); + } + log.info("Orchestrated job: {} on Executor: {}", DagManagerUtils.getFullyQualifiedJobName(dagNode), specExecutorUri); + this.dagManagerMetrics.incrementJobsSentToExecutor(dagNode); + } catch (Exception e) { + TimingEvent jobFailedTimer = this.eventSubmitter.isPresent() ? this.eventSubmitter.get(). + getTimingEvent(TimingEvent.LauncherTimings.JOB_FAILED) : null; + String message = "Cannot submit job " + DagManagerUtils.getFullyQualifiedJobName(dagNode) + " on executor " + specExecutorUri; + log.error(message, e); + jobMetadata.put(TimingEvent.METADATA_MESSAGE, message + " due to " + e.getMessage()); + if (jobFailedTimer != null) { + jobFailedTimer.stop(jobMetadata); + } + } + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/AdvanceDagTask.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/AdvanceDagTask.java new file mode 100644 index 00000000000..b781913e4c4 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/AdvanceDagTask.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration.task; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + + +/** + * An implementation of {@link DagTask} that is responsible for advancing the dag to the next node based + * on its current flow and job status. It is added to the {@link org.apache.gobblin.service.modules.orchestration.DagTaskStream} + * by the {@link org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor} after it consumes the appropriate + * {@link org.apache.gobblin.metrics.GobblinTrackingEvent} for the {@link org.apache.gobblin.service.modules.flowgraph.Dag} + */ + +@Alpha +@Slf4j +public class AdvanceDagTask extends DagTask { + @Getter + private final Dag.DagNode dagNode; + + public AdvanceDagTask(Dag.DagNode dagNode) { + this.dagNode = dagNode; + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/CleanUpDagTask.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/CleanUpDagTask.java new file mode 100644 index 00000000000..796c0db8293 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/CleanUpDagTask.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.service.modules.orchestration.task; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.service.modules.flowgraph.Dag; + + +/** + * An implementation of {@link org.apache.gobblin.service.modules.orchestration.task.DagTask} + * that is responsible for clean up {@link Dag} that has been completed or has reached an end state likewise: + * FAILED, COMPLETE or CANCELED. It is added to the {@link org.apache.gobblin.service.modules.orchestration.DagTaskStream} + * by the {@link org.apache.gobblin.service.monitoring.KafkaJobStatusMonitor} after it consumes the appropriate + * {@link org.apache.gobblin.metrics.GobblinTrackingEvent}. + * + */ + +@Alpha +@Slf4j +public class CleanUpDagTask extends DagTask { + // todo +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java new file mode 100644 index 00000000000..5da32abb0fe --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration.task; + +import java.io.IOException; + +import lombok.Getter; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter; +import org.apache.gobblin.service.modules.orchestration.DagManagementStateStore; +import org.apache.gobblin.service.modules.orchestration.DagManager; +import org.apache.gobblin.service.modules.orchestration.proc.DagProc; + + +/** + * Defines an individual task in a Dag. + * Upon completion of the {@link DagProc#process(DagManagementStateStore)} it will mark the lease + * acquired by {@link org.apache.gobblin.runtime.api.MultiActiveLeaseArbiter} as complete + */ + +@Alpha +public abstract class DagTask { + @Getter protected DagManager.DagId dagId; + //private MultiActiveLeaseArbiter.LeaseObtainedStatus leaseObtainedStatus; + + /** + * Currently, conclusion of {@link DagTask} marks and records a successful release of lease. + * It is invoked after {@link DagProc#process(DagManagementStateStore)} is completed successfully. + * @param multiActiveLeaseArbiter + * @throws IOException + */ + public void conclude(MultiActiveLeaseArbiter multiActiveLeaseArbiter) throws IOException { + // todo + //multiActiveLeaseArbiter.recordLeaseSuccess(leaseObtainedStatus); + } + +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/KillDagTask.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/KillDagTask.java new file mode 100644 index 00000000000..94abebb5a00 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/KillDagTask.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration.task; + +import lombok.Getter; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.orchestration.DagManager; + + +/** + * A {@link DagTask} responsible to handle kill tasks. + */ +@Alpha +public class KillDagTask extends DagTask { + @Getter Dag.DagNode.DagNodeId dagNodeId; + public KillDagTask(DagManager.DagId dagId) { + this.dagId = dagId; + } + + public KillDagTask(Dag.DagNode.DagNodeId dagNodeId) { + this.dagNodeId = dagNodeId; + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/LaunchDagTask.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/LaunchDagTask.java new file mode 100644 index 00000000000..0b3853551c1 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/LaunchDagTask.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration.task; + + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + + +/** + * A {@link DagTask} responsible to handle launch tasks. + */ + +@Slf4j +@Alpha +public class LaunchDagTask extends DagTask { + @Getter private final Dag dag; + + public LaunchDagTask(Dag dag) { + this.dag = dag; + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/ResumeDagTask.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/ResumeDagTask.java new file mode 100644 index 00000000000..0a1dd5b92df --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/ResumeDagTask.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration.task; + + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; + + +/** + * A {@link DagTask} responsible to handle resume tasks. + */ + +@Slf4j +@Alpha +public class ResumeDagTask extends DagTask { + +// todo +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/RetryDagTask.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/RetryDagTask.java new file mode 100644 index 00000000000..4aef56ea098 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/RetryDagTask.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration.task; + + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.annotation.Alpha; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + + +/** + * A {@link DagTask} responsible to handle launch tasks. + */ + +@Slf4j +@Alpha +public class RetryDagTask extends DagTask { + @Getter private final Dag.DagNode dagNode; + + public RetryDagTask(Dag.DagNode dagNode) { + this.dagNode = dagNode; + } +} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigResourceHandler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigResourceHandler.java index f0db8b5826f..b12b6036322 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigResourceHandler.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigResourceHandler.java @@ -48,6 +48,7 @@ import org.apache.gobblin.service.FlowConfigsResourceHandler; import org.apache.gobblin.service.FlowId; import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; import org.apache.gobblin.service.modules.utils.HelixUtils; import org.apache.gobblin.runtime.util.InjectionNames; @@ -107,7 +108,7 @@ public Collection getAllFlowConfigs(int start, int count) { /** * Adding {@link FlowConfig} should check if current node is active (master). * If current node is active, call {@link FlowConfigResourceLocalHandler#createFlowConfig(FlowConfig)} directly. - * If current node is standby, forward {@link ServiceConfigKeys#HELIX_FLOWSPEC_ADD} to active. The remote active will + * If current node is standby, forward {@link ServiceConstants#HELIX_FLOWSPEC_ADD} to active. The remote active will * then call {@link FlowConfigResourceLocalHandler#createFlowConfig(FlowConfig)}. * * Please refer to {@link org.apache.gobblin.service.modules.core.ControllerUserDefinedMessageHandlerFactory} for remote handling. @@ -128,7 +129,7 @@ public CreateResponse createFlowConfig(FlowConfig flowConfig) null); } - checkHelixConnection(ServiceConfigKeys.HELIX_FLOWSPEC_ADD, flowName, flowGroup); + checkHelixConnection(ServiceConstants.HELIX_FLOWSPEC_ADD, flowName, flowGroup); if (forceLeader) { HelixUtils.throwErrorIfNotLeader(helixManager); @@ -144,7 +145,7 @@ public CreateResponse createFlowConfig(FlowConfig flowConfig) if (!flowConfig.hasExplain() || !flowConfig.isExplain()) { //Forward the message to master only if it is not an "explain" request. - forwardMessage(ServiceConfigKeys.HELIX_FLOWSPEC_ADD, FlowConfigUtils.serializeFlowConfig(flowConfig), flowName, flowGroup); + forwardMessage(ServiceConstants.HELIX_FLOWSPEC_ADD, FlowConfigUtils.serializeFlowConfig(flowConfig), flowName, flowGroup); } // Do actual work on remote node, directly return success @@ -182,7 +183,7 @@ public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig) "flowName and flowGroup cannot be changed in update", null); } - checkHelixConnection(ServiceConfigKeys.HELIX_FLOWSPEC_UPDATE, flowName, flowGroup); + checkHelixConnection(ServiceConstants.HELIX_FLOWSPEC_UPDATE, flowName, flowGroup); if (forceLeader) { HelixUtils.throwErrorIfNotLeader(helixManager); @@ -196,7 +197,7 @@ public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig) this.localHandler.updateFlowConfig(flowId, flowConfig, false); } - forwardMessage(ServiceConfigKeys.HELIX_FLOWSPEC_UPDATE, FlowConfigUtils.serializeFlowConfig(flowConfig), flowName, flowGroup); + forwardMessage(ServiceConstants.HELIX_FLOWSPEC_UPDATE, FlowConfigUtils.serializeFlowConfig(flowConfig), flowName, flowGroup); // Do actual work on remote node, directly return success log.info("Forwarding update flowConfig [flowName=" + flowName + " flowGroup=" + flowGroup + "]"); @@ -242,7 +243,7 @@ public UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) String flowName = flowId.getFlowName(); String flowGroup = flowId.getFlowGroup(); - checkHelixConnection(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE, flowName, flowGroup); + checkHelixConnection(ServiceConstants.HELIX_FLOWSPEC_REMOVE, flowName, flowGroup); if (forceLeader) { HelixUtils.throwErrorIfNotLeader(helixManager); @@ -256,7 +257,7 @@ public UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) this.localHandler.deleteFlowConfig(flowId, header, false); } - forwardMessage(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE, FlowConfigUtils.serializeFlowId(flowId), flowName, flowGroup); + forwardMessage(ServiceConstants.HELIX_FLOWSPEC_REMOVE, FlowConfigUtils.serializeFlowId(flowId), flowName, flowGroup); return new UpdateResponse(HttpStatus.S_200_OK); } else { diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java index 0b5d1cdc7e9..08b6cab8d39 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java @@ -52,14 +52,15 @@ public void resume(ComplexResourceKey helixManager, Optional flowCatalog, Optional topologyCatalog, - Optional dagManager, Optional quotaManager, SchedulerService schedulerService, + Optional dagManager, Optional dagProcessingEngine, Optional quotaManager, SchedulerService schedulerService, Optional log, boolean warmStandbyEnabled, Optional flowTriggerHandler, SharedFlowMetricsSingleton sharedFlowMetricsSingleton) throws Exception { this(serviceName, config, helixManager, flowCatalog, topologyCatalog, - new Orchestrator(config, flowStatusGenerator, topologyCatalog, dagManager, log, flowTriggerHandler, + new Orchestrator(config, flowStatusGenerator, topologyCatalog, dagManager, dagProcessingEngine, log, flowTriggerHandler, sharedFlowMetricsSingleton), schedulerService, quotaManager, log, warmStandbyEnabled, flowTriggerHandler); } diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/ChangeMonitorUtils.java b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/ChangeMonitorUtils.java index 9e121c8b778..d6210184c0a 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/ChangeMonitorUtils.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/ChangeMonitorUtils.java @@ -34,7 +34,7 @@ private ChangeMonitorUtils() { * should not be processed. Returns true if the pre-conditions above don't apply, and we should proceed processing * the change event */ - public static boolean isValidAndUniqueMessage(String changeIdentifier, String operation, String timestamp, + public static boolean isValidAndUniqueMessage(String changeIdentifier, OperationType operation, String timestamp, LoadingCache cache, ContextAwareMeter duplicateMessagesMeter, ContextAwareMeter heartbeatMessagesMeter) { // If we've already processed a message with this timestamp and key before then skip duplicate message @@ -45,7 +45,7 @@ public static boolean isValidAndUniqueMessage(String changeIdentifier, String op } // If event is a heartbeat type then log it and skip processing - if (operation.equals("HEARTBEAT")) { + if (operation == OperationType.HEARTBEAT) { log.debug("Received heartbeat message from time {}", timestamp); heartbeatMessagesMeter.mark(); return false; diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java index bbe1e47ab20..4e3c8d34fad 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/DagActionStoreChangeMonitor.java @@ -28,6 +28,9 @@ import java.net.URISyntaxException; import java.util.UUID; import java.util.concurrent.TimeUnit; + +import org.jetbrains.annotations.NotNull; + import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.gobblin.configuration.ConfigurationKeys; @@ -41,8 +44,13 @@ import org.apache.gobblin.runtime.metrics.RuntimeMetrics; import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; import org.apache.gobblin.service.FlowId; +import org.apache.gobblin.service.ServiceConfigKeys; import org.apache.gobblin.service.modules.orchestration.DagManager; +import org.apache.gobblin.service.modules.orchestration.DagManagerUtils; +import org.apache.gobblin.service.modules.orchestration.DagTaskStream; import org.apache.gobblin.service.modules.orchestration.Orchestrator; +import org.apache.gobblin.service.modules.orchestration.task.KillDagTask; +import org.apache.gobblin.util.ConfigUtils; /** @@ -51,7 +59,7 @@ * connector between the API and execution layers of GaaS. */ @Slf4j -public class DagActionStoreChangeMonitor extends HighLevelConsumer { +public class DagActionStoreChangeMonitor extends HighLevelConsumer { public static final String DAG_ACTION_CHANGE_MONITOR_PREFIX = "dagActionChangeStore"; // Metrics @@ -70,7 +78,7 @@ public class DagActionStoreChangeMonitor extends HighLevelConsumer { protected CacheLoader cacheLoader = new CacheLoader() { @Override - public String load(String key) throws Exception { + public String load(@NotNull String key) { return key; } }; @@ -87,6 +95,8 @@ public String load(String key) throws Exception { @Getter @VisibleForTesting protected FlowCatalog flowCatalog; + private DagTaskStream dagTaskStream; + private final boolean isMultiLeaderDagManagerEnabled; // 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. @@ -101,6 +111,9 @@ public DagActionStoreChangeMonitor(String topic, Config config, DagActionStore d this.flowCatalog = flowCatalog; this.orchestrator = orchestrator; this.isMultiActiveSchedulerEnabled = isMultiActiveSchedulerEnabled; + // instantiating using default ctor; subsequent PR will handle instantiating with multi-args ctor +// this.dagTaskStream = new DagTaskStream(); + this.isMultiLeaderDagManagerEnabled = ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_MULTI_ACTIVE_DAG_MANAGER_ENABLED_KEY, false); } @Override @@ -115,7 +128,7 @@ protected void assignTopicPartitions() { This class is multithreaded and this method will be called by multiple threads, however any given message will be partitioned and processed by only one thread (and corresponding queue). */ - protected void processMessage(DecodeableKafkaRecord message) { + protected void processMessage(DecodeableKafkaRecord message) { // This will also include the heathCheck message so that we can rely on this to monitor the health of this Monitor messageProcessedMeter.mark(); String key = (String) message.getKey(); @@ -123,10 +136,10 @@ protected void processMessage(DecodeableKafkaRecord message) { String tid = value.getChangeEventIdentifier().getTxId(); Long produceTimestamp = value.getChangeEventIdentifier().getProduceTimestampMillis(); - String operation = value.getChangeEventIdentifier().getOperationType().name(); + OperationType operation = value.getChangeEventIdentifier().getOperationType(); String flowGroup = value.getFlowGroup(); String flowName = value.getFlowName(); - String flowExecutionId = value.getFlowExecutionId(); + long flowExecutionId = Long.parseLong(value.getFlowExecutionId()); produceToConsumeDelayValue = calcMillisSince(produceTimestamp); log.debug("Processing Dag Action message for flow group: {} name: {} executionId: {} tid: {} operation: {} lag: {}", @@ -147,63 +160,80 @@ protected void processMessage(DecodeableKafkaRecord message) { DagActionStore.FlowActionType dagActionType = DagActionStore.FlowActionType.valueOf(value.getDagAction().toString()); // Used to easily log information to identify the dag action - DagActionStore.DagAction dagAction = new DagActionStore.DagAction(flowGroup, flowName, flowExecutionId, - dagActionType); + DagActionStore.DagAction dagAction = new DagActionStore.DagAction(flowGroup, flowName, flowExecutionId, dagActionType); // We only expect INSERT and DELETE operations done to this table. INSERTs correspond to any type of // {@link DagActionStore.FlowActionType} flow requests that have to be processed. DELETEs require no action. try { - if (operation.equals("INSERT")) { - log.info("DagAction change ({}) received for flow: {}", dagActionType, dagAction); - if (dagActionType.equals(DagActionStore.FlowActionType.RESUME)) { - dagManager.handleResumeFlowRequest(flowGroup, flowName,Long.parseLong(flowExecutionId)); - this.resumesInvoked.mark(); - } else if (dagActionType.equals(DagActionStore.FlowActionType.KILL)) { - dagManager.handleKillFlowRequest(flowGroup, flowName, Long.parseLong(flowExecutionId)); - this.killsInvoked.mark(); - } else if (dagActionType.equals(DagActionStore.FlowActionType.LAUNCH)) { - // If multi-active scheduler is NOT turned on we should not receive these type of events - if (!this.isMultiActiveSchedulerEnabled) { + switch (operation) { + case INSERT: + log.info("DagAction change ({}) received for flow: {}", dagActionType, dagAction); + if (dagActionType.equals(DagActionStore.FlowActionType.RESUME)) { + dagManager.handleResumeFlowRequest(flowGroup, flowName, flowExecutionId); + //TODO: add a flag for if condition only if multi-active is enabled + this.resumesInvoked.mark(); + } else if (dagActionType.equals(DagActionStore.FlowActionType.KILL)) { + dagManager.handleKillFlowRequest(flowGroup, flowName, flowExecutionId); + if (isMultiLeaderDagManagerEnabled) { + DagManager.DagId dagId = DagManagerUtils.generateDagId(flowGroup, flowName, flowExecutionId); + dagTaskStream.addDagTask(new KillDagTask(dagId)); + } + this.killsInvoked.mark(); + } else if (dagActionType.equals(DagActionStore.FlowActionType.LAUNCH)) { + // If multi-active scheduler is NOT turned on we should not receive these type of events + if (!this.isMultiActiveSchedulerEnabled) { + this.unexpectedErrors.mark(); + throw new RuntimeException(String.format( + "Received %s dagAction while not in multi-active scheduler mode for flowAction: %s", + DagActionStore.FlowActionType.LAUNCH, dagAction)); + } + submitFlowToDagManagerHelper(flowGroup, flowName, flowExecutionId); + //TODO: add a flag for if condition only if multi-active is enabled + } else { + log.warn(String.format("Received unsupported dagAction %s. Expected to be a %s, %s, %s", dagActionType, + DagActionStore.FlowActionType.KILL, DagActionStore.FlowActionType.RESUME, + DagActionStore.FlowActionType.LAUNCH)); this.unexpectedErrors.mark(); - throw new RuntimeException(String.format("Received LAUNCH dagAction while not in multi-active scheduler " - + "mode for flowAction: %s", dagAction)); + return; } submitFlowToDagManagerHelper(flowGroup, flowName, flowExecutionId); - } else { - log.warn("Received unsupported dagAction {}. Expected to be a KILL, RESUME, or LAUNCH", dagActionType); + break; + + case UPDATE: + log.warn(String.format("Received an %s action to the DagActionStore when values in this store are never supposed" + + " to be updated. Flow group: %s name %s executionId %s were updated to action %s", OperationType.UPDATE, + flowGroup, flowName, flowExecutionId, dagActionType)); + this.unexpectedErrors.mark(); + break; + + case DELETE: + log.debug("Deleted flow group: {} name: {} executionId {} from DagActionStore", flowGroup, flowName, flowExecutionId); + break; + + default: + log.warn(String.format( + "Received unsupported change type of operation %s. Expected values to be in [%s, %s, %s]", + operation, OperationType.INSERT, OperationType.UPDATE, OperationType.DELETE)); this.unexpectedErrors.mark(); return; - } - } else if (operation.equals("UPDATE")) { - log.warn("Received an UPDATE action to the DagActionStore when values in this store are never supposed to be " - + "updated. Flow group: {} name {} executionId {} were updated to action {}", flowGroup, flowName, - flowExecutionId, dagActionType); - this.unexpectedErrors.mark(); - } else if (operation.equals("DELETE")) { - log.debug("Deleted flow group: {} name: {} executionId {} from DagActionStore", flowGroup, flowName, flowExecutionId); - } else { - log.warn("Received unsupported change type of operation {}. Expected values to be in [INSERT, UPDATE, DELETE]", - operation); - this.unexpectedErrors.mark(); - return; } } catch (Exception e) { - log.warn("Ran into unexpected error processing DagActionStore changes: {}", e); + log.warn("Ran into unexpected error processing DagActionStore changes: ", e); this.unexpectedErrors.mark(); } dagActionsSeenCache.put(changeIdentifier, changeIdentifier); } - protected void submitFlowToDagManagerHelper(String flowGroup, String flowName, String flowExecutionId) { + protected void submitFlowToDagManagerHelper(String flowGroup, String flowName, long flowExecutionId) { // 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; + FlowSpec spec; try { URI flowUri = FlowSpec.Utils.createFlowSpecUri(flowId); spec = (FlowSpec) flowCatalog.getSpecs(flowUri); // Adds flowExecutionId to config to ensure they are consistent across hosts - FlowSpec updatedSpec = FlowSpec.Utils.createFlowSpecWithProperty(spec, ConfigurationKeys.FLOW_EXECUTION_ID_KEY, flowExecutionId); + FlowSpec updatedSpec = FlowSpec.Utils.createFlowSpecWithProperty(spec, ConfigurationKeys.FLOW_EXECUTION_ID_KEY, String.valueOf(flowExecutionId)); this.orchestrator.submitFlowToDagManager(updatedSpec); } catch (URISyntaxException e) { log.warn("Could not create URI object for flowId {}. Exception {}", flowId, e.getMessage()); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaAvroJobStatusMonitor.java b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaAvroJobStatusMonitor.java index c152970cb1a..f00ec3ef781 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaAvroJobStatusMonitor.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaAvroJobStatusMonitor.java @@ -46,6 +46,7 @@ import org.apache.gobblin.metrics.reporter.util.SchemaVersionWriter; import org.apache.gobblin.runtime.troubleshooter.JobIssueEventHandler; import org.apache.gobblin.service.ExecutionStatus; +import org.apache.gobblin.service.modules.orchestration.DagProcessingEngine; import org.apache.gobblin.util.ConfigUtils; @@ -64,10 +65,10 @@ public class KafkaAvroJobStatusMonitor extends KafkaJobStatusMonitor { @Getter private Meter messageParseFailures; - public KafkaAvroJobStatusMonitor(String topic, Config config, int numThreads, - JobIssueEventHandler jobIssueEventHandler, GaaSObservabilityEventProducer observabilityEventProducer) + public KafkaAvroJobStatusMonitor(String topic, Config config, int numThreads, JobIssueEventHandler jobIssueEventHandler, + GaaSObservabilityEventProducer observabilityEventProducer, DagProcessingEngine dagProcessingEngine) throws IOException, ReflectiveOperationException { - super(topic, config, numThreads, jobIssueEventHandler, observabilityEventProducer); + super(topic, config, numThreads, jobIssueEventHandler, observabilityEventProducer, dagProcessingEngine); if (ConfigUtils.getBoolean(config, ConfigurationKeys.METRICS_REPORTING_KAFKA_USE_SCHEMA_REGISTRY, false)) { KafkaAvroSchemaRegistry schemaRegistry = (KafkaAvroSchemaRegistry) new KafkaAvroSchemaRegistryFactory(). diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java index 87e1c3b5a14..df4272217c9 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java @@ -37,10 +37,13 @@ import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.eventbus.EventBus; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; import com.google.common.annotations.VisibleForTesting; + +import lombok.AccessLevel; import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -60,6 +63,8 @@ import org.apache.gobblin.runtime.troubleshooter.JobIssueEventHandler; import org.apache.gobblin.service.ExecutionStatus; import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.modules.orchestration.DagProcessingEngine; +import org.apache.gobblin.service.monitoring.event.JobStatusEvent; import org.apache.gobblin.source.workunit.WorkUnit; import org.apache.gobblin.util.ConfigUtils; import org.apache.gobblin.util.retry.RetryerFactory; @@ -94,6 +99,8 @@ public abstract class KafkaJobStatusMonitor extends HighLevelConsumer stateStore; private final ScheduledExecutorService scheduledExecutorService; + @Getter(AccessLevel.PUBLIC) + protected static final EventBus eventBus = new EventBus(KafkaJobStatusMonitor.class.getSimpleName()); private static final Config RETRYER_FALLBACK_CONFIG = ConfigFactory.parseMap(ImmutableMap.of( RETRY_TIME_OUT_MS, TimeUnit.HOURS.toMillis(24L), // after a day, presume non-transient and give up RETRY_INTERVAL_MS, TimeUnit.MINUTES.toMillis(1L), // back-off to once/minute @@ -110,9 +117,10 @@ public abstract class KafkaJobStatusMonitor extends HighLevelConsumer persistJobStatusRetryer; private final GaaSObservabilityEventProducer eventProducer; + DagProcessingEngine dagProcessingEngine; public KafkaJobStatusMonitor(String topic, Config config, int numThreads, JobIssueEventHandler jobIssueEventHandler, - GaaSObservabilityEventProducer observabilityEventProducer) + GaaSObservabilityEventProducer observabilityEventProducer, DagProcessingEngine dagProcessingEngine) throws ReflectiveOperationException { super(topic, config.withFallback(DEFAULTS), numThreads); String stateStoreFactoryClass = ConfigUtils.getString(config, ConfigurationKeys.STATE_STORE_FACTORY_CLASS_KEY, FileContextBasedFsStateStoreFactory.class.getName()); @@ -138,7 +146,8 @@ public void onRetry(Attempt attempt) { } } })); - this.eventProducer = observabilityEventProducer; + this.eventProducer = observabilityEventProducer; + this.dagProcessingEngine = dagProcessingEngine; } @Override @@ -191,6 +200,7 @@ protected void processMessage(DecodeableKafkaRecord message) { if (jobStatus != null) { try (Timer.Context context = getMetricContext().timer(GET_AND_SET_JOB_STATUS).time()) { addJobStatusToStateStore(jobStatus, this.stateStore, this.eventProducer); + //TODO: Add Advance and CleanUp DagTask to DagTaskStream } } return null; @@ -276,6 +286,7 @@ static void addJobStatusToStateStore(org.apache.gobblin.configuration.State jobS stateStore.put(storeName, tableName, jobStatus); if (isNewStateTransitionToFinal(jobStatus, states)) { eventProducer.emitObservabilityEvent(jobStatus); + eventBus.post(new JobStatusEvent(jobStatus)); } } catch (Exception e) { log.warn("Meet exception when adding jobStatus to state store at " diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitorFactory.java b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitorFactory.java index 0b80b4f2eaa..142051623da 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitorFactory.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitorFactory.java @@ -33,6 +33,7 @@ import org.apache.gobblin.runtime.api.GobblinInstanceEnvironment; import org.apache.gobblin.runtime.troubleshooter.JobIssueEventHandler; import org.apache.gobblin.runtime.troubleshooter.MultiContextIssueRepository; +import org.apache.gobblin.service.modules.orchestration.DagProcessingEngine; import org.apache.gobblin.util.ConfigUtils; import org.apache.gobblin.util.reflection.GobblinConstructorUtils; @@ -49,19 +50,21 @@ public class KafkaJobStatusMonitorFactory implements Provider timeOpAndWrapIOException(SupplierThrowingIO> sta } private List getLatestExecutionIds(List jobStatusStates, int count) { - // `distinct()`, to avoid each flow execution ID replicating as many times as it has child jobs - Iterator flowExecutionIds = jobStatusStates.stream().map(this::getFlowExecutionId).distinct().iterator(); + // `distinct()`, to avoid each flow execution ID replicating as many times as it + // has child jobs + Iterator flowExecutionIds = jobStatusStates.stream().map(JobStatusRetriever::getFlowExecutionId).distinct().iterator(); return Ordering.natural().greatestOf(flowExecutionIds, count); } } diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitor.java b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitor.java index 8b197a352ef..7db003a6740 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitor.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/SpecStoreChangeMonitor.java @@ -110,7 +110,7 @@ protected void processMessage(DecodeableKafkaRecord message) { String tid = value.getTxId(); Long produceTimestamp = value.getProduceTimestampMillis(); - String operation = value.getOperationType().name(); + OperationType operation = value.getOperationType(); produceToConsumeDelayValue = calcMillisSince(produceTimestamp); log.debug("Processing message where specUri is {} tid: {} operation: {} delay: {}", key, tid, operation, @@ -123,7 +123,7 @@ protected void processMessage(DecodeableKafkaRecord message) { } Spec spec; - URI specAsUri = null; + URI specAsUri; try { specAsUri = new URI(key); @@ -133,18 +133,17 @@ protected void processMessage(DecodeableKafkaRecord message) { return; } - spec = (!operation.equals("DELETE")) ? this.flowCatalog.getSpecWrapper(specAsUri) : null; + spec = operation != OperationType.DELETE ? this.flowCatalog.getSpecWrapper(specAsUri) : null; // The monitor should continue to process messages regardless of failures with individual messages, instead we use // metrics to keep track of failure to process certain SpecStoreChange events try { // Call respective action for the type of change received - AddSpecResponse response; - if (operation.equals("INSERT") || operation.equals("UPDATE")) { + AddSpecResponse response; + if (operation == OperationType.INSERT || operation == OperationType.UPDATE) { response = scheduler.onAddSpec(spec); - // Null response means the dag failed to compile - if (response != null && FlowCatalog.isCompileSuccessful((String) response.getValue())) { + if (response != null && FlowCatalog.isCompileSuccessful(response.getValue())) { log.info("Successfully added spec {} to scheduler response {}", spec, StringEscapeUtils.escapeJson(response.getValue().toString())); this.successfullyAddedSpecs.mark(); @@ -153,19 +152,19 @@ protected void processMessage(DecodeableKafkaRecord message) { + "invalidate the earlier compilation. Examine changes to locate error. Response is {}", spec, response); this.failedAddedSpecs.mark(); } - } else if (operation.equals("DELETE")) { + } else if (operation == OperationType.DELETE) { log.info("Deleting spec {} after receiving spec store change event", specAsUri); scheduler.onDeleteSpec(specAsUri, FlowSpec.Builder.DEFAULT_VERSION); this.deletedSpecs.mark(); } else { - log.warn("Received unsupported change type of operation {}. Expected values to be in " - + "[INSERT, UPDATE, DELETE, HEARTBEAT]. Look for issue with kafka event consumer or emitter", - operation); + log.warn(String.format("Received unsupported change type of operation %s. Expected values to be in " + + "[%s, %s, %s, %s]. Look for issue with kafka event consumer or emitter", + operation, OperationType.INSERT, OperationType.UPDATE, OperationType.DELETE, OperationType.HEARTBEAT)); this.unexpectedErrors.mark(); return; } } catch (Exception e) { - log.warn("Ran into unexpected error processing SpecStore changes. Reexamine scheduler. Error: {}", e); + log.warn("Ran into unexpected error processing SpecStore changes. Reexamine scheduler. Error: ", e); this.unexpectedErrors.mark(); return; } diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/event/JobStatusEvent.java b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/event/JobStatusEvent.java new file mode 100644 index 00000000000..4d1dd6534c2 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/event/JobStatusEvent.java @@ -0,0 +1,49 @@ +/* + * 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.monitoring.event; + +import lombok.Getter; + +import org.apache.gobblin.configuration.State; +import org.apache.gobblin.metrics.event.TimingEvent; +import org.apache.gobblin.service.ExecutionStatus; +import org.apache.gobblin.service.monitoring.JobStatus; +import org.apache.gobblin.service.monitoring.JobStatusRetriever; + + +@Getter +public class JobStatusEvent { + State jobStatusState; + String flowGroup; + String flowName; + long flowExecutionId; + String jobGroup; + String jobName; + ExecutionStatus status; + JobStatus jobStatus; + public JobStatusEvent(State jobStatusState) { + this.jobStatusState = jobStatusState; + this.status = ExecutionStatus.valueOf(jobStatusState.getProp(JobStatusRetriever.EVENT_NAME_FIELD)); + this.flowName = jobStatusState.getProp(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD); + this.flowGroup = jobStatusState.getProp(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD); + this.flowExecutionId = jobStatusState.getPropAsLong(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD); + this.jobName = jobStatusState.getProp(TimingEvent.FlowEventConstants.JOB_NAME_FIELD); + this.jobGroup = jobStatusState.getProp(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD); + this.jobStatus = JobStatusRetriever.createJobStatusBuilderFromState(jobStatusState).build(); + } +} diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagManagerFlowTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagManagerFlowTest.java index 5445a209670..4e05de90f3b 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagManagerFlowTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagManagerFlowTest.java @@ -69,8 +69,8 @@ public class DagManagerFlowTest { private static final String TABLE = "dag_action_store"; private static final String flowGroup = "testFlowGroup"; private static final String flowName = "testFlowName"; - private static final String flowExecutionId = "12345677"; - private static final String flowExecutionId_2 = "12345678"; + private static final long flowExecutionId = 12345677L; + private static final long flowExecutionId_2 = 12345678L; private DagActionStore dagActionStore; @BeforeClass diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandlerTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandlerTest.java index 67289267332..70ad36f6b7e 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandlerTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/FlowTriggerHandlerTest.java @@ -34,7 +34,7 @@ public class FlowTriggerHandlerTest { String cronExpressionSuffix = truncateFirstTwoFieldsOfCronExpression(cronExpression); int schedulerBackOffMillis = 10; DagActionStore.DagAction flowAction = new DagActionStore.DagAction("flowName", "flowGroup", - String.valueOf(eventToRevisit), DagActionStore.FlowActionType.LAUNCH); + eventToRevisit, DagActionStore.FlowActionType.LAUNCH); MultiActiveLeaseArbiter.LeasedToAnotherStatus leasedToAnotherStatus = new MultiActiveLeaseArbiter.LeasedToAnotherStatus(flowAction, minimumLingerDurationMillis); diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/OrchestratorTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/OrchestratorTest.java index 444f2dc8cd6..7252057ba4c 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/OrchestratorTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/OrchestratorTest.java @@ -107,8 +107,8 @@ public void setup() throws Exception { this._mockFlowTriggerHandler = mock(FlowTriggerHandler.class); this.orchestrator = new Orchestrator(ConfigUtils.propertiesToConfig(orchestratorProperties), - this.mockStatusGenerator, Optional.of(this.topologyCatalog), Optional.absent(), Optional.of(logger), - Optional.of(this._mockFlowTriggerHandler), new SharedFlowMetricsSingleton( + this.mockStatusGenerator, Optional.of(this.topologyCatalog), Optional.absent(), Optional.absent(), + Optional.of(logger), Optional.of(this._mockFlowTriggerHandler), new SharedFlowMetricsSingleton( ConfigUtils.propertiesToConfig(orchestratorProperties))); this.topologyCatalog.addListener(orchestrator); this.flowCatalog.addListener(orchestrator); diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java index 9949c97d567..cec5477597b 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java @@ -53,7 +53,7 @@ import org.apache.gobblin.runtime.spec_catalog.TopologyCatalog; import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor; import org.apache.gobblin.scheduler.SchedulerService; -import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; import org.apache.gobblin.service.modules.flow.MockedSpecCompiler; import org.apache.gobblin.service.modules.flow.SpecCompiler; import org.apache.gobblin.service.modules.flowgraph.Dag; @@ -114,7 +114,7 @@ public void testJobSchedulerInit() throws Throwable { properties.setProperty(FLOWSPEC_STORE_DIR_KEY, specDir.getAbsolutePath()); FlowCatalog flowCatalog = new FlowCatalog(ConfigUtils.propertiesToConfig(properties)); SpecCatalogListener mockListener = Mockito.mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); + when(mockListener.getName()).thenReturn(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse("")); flowCatalog.addListener(mockListener); ServiceBasedAppLauncher serviceLauncher = new ServiceBasedAppLauncher(properties, "GaaSJobSchedulerTest"); @@ -198,7 +198,7 @@ public void testJobSchedulerInitWithFailedSpec() throws Throwable { // Assume that the catalog can store corrupted flows SpecCatalogListener mockListener = Mockito.mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); + when(mockListener.getName()).thenReturn(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse("")); flowCatalog.addListener(mockListener); @@ -262,7 +262,7 @@ public void testJobSchedulerUnschedule() throws Throwable { // Assume that the catalog can store corrupted flows SpecCatalogListener mockListener = Mockito.mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); + when(mockListener.getName()).thenReturn(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse("")); flowCatalog.addListener(mockListener); diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/monitoring/GitConfigMonitorTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/monitoring/GitConfigMonitorTest.java index dad94b05411..2b04cd740d6 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/monitoring/GitConfigMonitorTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/monitoring/GitConfigMonitorTest.java @@ -56,7 +56,7 @@ import org.apache.gobblin.runtime.api.SpecCatalogListener; import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse; import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; -import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.ServiceConstants; import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; @@ -112,7 +112,7 @@ public void setup() throws Exception { this.flowCatalog = new FlowCatalog(config); this.mockListener = mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); + when(mockListener.getName()).thenReturn(ServiceConstants.GOBBLIN_SERVICE_JOB_SCHEDULER_LISTENER_CLASS); when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse("")); this.flowCatalog.addListener(mockListener);