From 295b529eb59cbac77728895ee6b49810494132bb Mon Sep 17 00:00:00 2001 From: Arjun Singh Bora Date: Thu, 15 Aug 2024 16:20:06 -0700 Subject: [PATCH 1/4] remove flowconfigs resource handler for single active code paths --- .../gobblin/service/ServiceConfigKeys.java | 22 -- ....gobblin.service.flowconfigs.restspec.json | 31 -- ...gobblin.service.flowstatuses.restspec.json | 5 +- ....gobblin.service.flowconfigs.snapshot.json | 124 ------- ...gobblin.service.flowstatuses.snapshot.json | 5 +- .../gobblin/service/FlowConfigClient.java | 198 ---------- .../gobblin/service/FlowConfigTest.java | 337 ------------------ ...nfigV2Test.java => FlowConfigsV2Test.java} | 291 ++++++++++++--- .../FlowConfigV2ResourceLocalHandler.java | 127 ------- .../gobblin/service/FlowConfigsResource.java | 181 ---------- .../service/FlowConfigsResourceHandler.java | 69 ---- .../service/FlowConfigsV2Resource.java | 41 +-- .../service/FlowConfigsV2ResourceHandler.java | 21 -- .../service/FlowExecutionResource.java | 171 ++++++++- ...lowExecutionResourceHandlerInterface.java} | 14 +- .../FlowExecutionResourceLocalHandler.java | 273 -------------- .../gobblin/service/FlowStatusResource.java | 10 +- .../restli/FlowConfigsV2ResourceHandler.java} | 211 ++++++----- .../FlowConfigResourceLocalHandlerTest.java | 9 +- ...FlowExecutionResourceLocalHandlerTest.java | 17 +- .../gobblin/runtime/util/InjectionNames.java | 2 - ...ollerUserDefinedMessageHandlerFactory.java | 206 ----------- .../core/GobblinServiceConfiguration.java | 13 - .../core/GobblinServiceGuiceModule.java | 72 +--- .../modules/core/GobblinServiceManager.java | 27 +- .../DagManagementStateStore.java | 1 - .../MultiActiveLeaseArbiterFactory.java | 2 +- .../restli/FlowExecutionResourceHandler.java | 143 ++++++++ ...bblinServiceFlowConfigResourceHandler.java | 286 --------------- ...linServiceFlowConfigV2ResourceHandler.java | 43 --- ...onfigV2ResourceHandlerWithWarmStandby.java | 127 ------- ...inServiceFlowExecutionResourceHandler.java | 109 ------ ...ecutionResourceHandlerWithWarmStandby.java | 91 ----- .../service/modules/utils/HelixUtils.java | 186 ---------- .../DagActionStoreChangeMonitor.java | 6 +- .../gobblin/service/FlowExecutionTest.java | 109 +++--- .../service/GobblinServiceManagerTest.java | 7 +- .../modules/orchestration/DagFlowTest.java | 58 +++ .../orchestration/OrchestratorTest.java | 5 +- .../modules/restli/FlowConfigUtilsTest.java | 5 +- 40 files changed, 852 insertions(+), 2803 deletions(-) delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowconfigs.restspec.json delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowconfigs.snapshot.json delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/main/java/org/apache/gobblin/service/FlowConfigClient.java delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigTest.java rename gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/{FlowConfigV2Test.java => FlowConfigsV2Test.java} (65%) delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigV2ResourceLocalHandler.java delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResource.java delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandler.java delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2ResourceHandler.java rename gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/{FlowExecutionResourceHandler.java => FlowExecutionResourceHandlerInterface.java} (70%) delete mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandler.java rename gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/{FlowConfigResourceLocalHandler.java => modules/restli/FlowConfigsV2ResourceHandler.java} (67%) delete mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/ControllerUserDefinedMessageHandlerFactory.java create mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java delete mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigResourceHandler.java delete mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandler.java delete mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby.java delete mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandler.java delete mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java delete mode 100644 gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/HelixUtils.java rename gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowStatusTest.java => gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionTest.java (79%) create mode 100644 gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.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 c3632b0c79a..8b6c1df037e 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 @@ -34,32 +34,15 @@ public class ServiceConfigKeys { public static final String GOBBLIN_SERVICE_TOPOLOGY_SPEC_FACTORY_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "topologySpecFactory.enabled"; public static final String GOBBLIN_SERVICE_GIT_CONFIG_MONITOR_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "gitConfigMonitor.enabled"; public static final String GOBBLIN_SERVICE_JOB_STATUS_MONITOR_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "jobStatusMonitor.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"; - - // Helix / ServiceScheduler Keys - public static final String HELIX_CLUSTER_NAME_KEY = GOBBLIN_SERVICE_PREFIX + "helix.cluster.name"; - public static final String ZK_CONNECTION_STRING_KEY = GOBBLIN_SERVICE_PREFIX + "zk.connection.string"; - public static final String HELIX_INSTANCE_NAME_OPTION_NAME = "helix_instance_name"; - public static final String HELIX_INSTANCE_NAME_KEY = GOBBLIN_SERVICE_PREFIX + "helixInstanceName"; public static final String GOBBLIN_SERVICE_FLOWSPEC = GOBBLIN_SERVICE_PREFIX + "flowSpec"; 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"; public static final String COMPILATION_RESPONSE = "compilation.response"; - // Flow Catalog Keys - public static final String GOBBLIN_SERVICE_FLOW_CATALOG_LOCAL_COMMIT = GOBBLIN_SERVICE_PREFIX + "flowCatalog.localCommit"; - public static final boolean DEFAULT_GOBBLIN_SERVICE_FLOW_CATALOG_LOCAL_COMMIT = true; - // Job Level Keys public static final String WORK_UNIT_SIZE = GOBBLIN_SERVICE_PREFIX + "work.unit.size"; public static final String TOTAL_WORK_UNIT_SIZE = GOBBLIN_SERVICE_PREFIX + "total.work.unit.size"; @@ -103,8 +86,6 @@ public class ServiceConfigKeys { public static final String DEFAULT_NODE_SECURITY = "true"; public static final String SERVICE_PORT = GOBBLIN_SERVICE_PREFIX + "port"; - public static final String SERVICE_NAME = GOBBLIN_SERVICE_PREFIX + "serviceName"; - public static final String SERVICE_URL_PREFIX = GOBBLIN_SERVICE_PREFIX + "serviceUrlPrefix"; // Prefix for config to ServiceBasedAppLauncher that will only be used by GaaS and not orchestrated jobs public static final String GOBBLIN_SERVICE_APP_LAUNCHER_PREFIX = "gobblinServiceAppLauncher"; @@ -115,9 +96,6 @@ public class ServiceConfigKeys { public static final String LEADER_URL = "leaderUrl"; - public static final String FORCE_LEADER = GOBBLIN_SERVICE_PREFIX + "forceLeader"; - public static final boolean DEFAULT_FORCE_LEADER = false; - public static final String QUOTA_MANAGER_CLASS = GOBBLIN_SERVICE_PREFIX + "quotaManager.class"; public static final String DEFAULT_QUOTA_MANAGER = "org.apache.gobblin.service.modules.orchestration.InMemoryUserQuotaManager"; diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowconfigs.restspec.json b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowconfigs.restspec.json deleted file mode 100644 index 68c990c04ce..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowconfigs.restspec.json +++ /dev/null @@ -1,31 +0,0 @@ -{ - "name" : "flowconfigs", - "namespace" : "org.apache.gobblin.service", - "path" : "/flowconfigs", - "schema" : "org.apache.gobblin.service.FlowConfig", - "doc" : "Resource for handling flow configuration requests\n\ngenerated from: org.apache.gobblin.service.FlowConfigsResource", - "collection" : { - "identifier" : { - "name" : "id", - "type" : "org.apache.gobblin.service.FlowId", - "params" : "com.linkedin.restli.common.EmptyRecord" - }, - "supports" : [ "create", "delete", "get", "update" ], - "methods" : [ { - "method" : "create", - "doc" : "Create a flow configuration that the service will forward to execution instances for execution" - }, { - "method" : "get", - "doc" : "Retrieve the flow configuration with the given key" - }, { - "method" : "update", - "doc" : "Update the flow configuration with the specified key. Running flows are not affected.\n An error is raised if the flow configuration does not exist." - }, { - "method" : "delete", - "doc" : "Delete a configured flow. Running flows are not affected. The schedule will be removed for scheduled flows." - } ], - "entity" : { - "path" : "/flowconfigs/{id}" - } - } -} \ No newline at end of file diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json index 6e6b67ea24a..b6a0ee0eef8 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json @@ -1,9 +1,12 @@ { + "annotations" : { + "deprecated" : { } + }, "name" : "flowstatuses", "namespace" : "org.apache.gobblin.service", "path" : "/flowstatuses", "schema" : "org.apache.gobblin.service.FlowStatus", - "doc" : "Resource for handling flow status requests\n\ngenerated from: org.apache.gobblin.service.FlowStatusResource", + "doc" : "Resource for handling flow status requests. Deprecated, use {@link FlowExecutionResource}\n\ngenerated from: org.apache.gobblin.service.FlowStatusResource", "collection" : { "identifier" : { "name" : "id", diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowconfigs.snapshot.json b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowconfigs.snapshot.json deleted file mode 100644 index 9845e8dca26..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowconfigs.snapshot.json +++ /dev/null @@ -1,124 +0,0 @@ -{ - "models" : [ { - "type" : "record", - "name" : "EmptyRecord", - "namespace" : "com.linkedin.restli.common", - "doc" : "An literally empty record. Intended as a marker to indicate the absence of content where a record type is required. If used the underlying DataMap *must* be empty, EmptyRecordValidator is provided to help enforce this. For example, CreateRequest extends Request to indicate it has no response body. Also, a ComplexKeyResource implementation that has no ParamKey should have a signature like XyzResource implements ComplexKeyResource.", - "fields" : [ ], - "validate" : { - "com.linkedin.restli.common.EmptyRecordValidator" : { } - } - }, { - "type" : "record", - "name" : "FlowConfig", - "namespace" : "org.apache.gobblin.service", - "doc" : "Defines a flow configuration that can be compiled into Gobblin jobs", - "fields" : [ { - "name" : "id", - "type" : { - "type" : "record", - "name" : "FlowId", - "doc" : "Identifier for a Gobblin as a Service flow", - "fields" : [ { - "name" : "flowName", - "type" : "string", - "doc" : "Name of the flow", - "validate" : { - "strlen" : { - "max" : 128, - "min" : 1 - } - } - }, { - "name" : "flowGroup", - "type" : "string", - "doc" : "Group of the flow. This defines the namespace for the flow.", - "validate" : { - "strlen" : { - "max" : 128, - "min" : 1 - } - } - } ] - }, - "doc" : "Identifier for the flow" - }, { - "name" : "schedule", - "type" : { - "type" : "record", - "name" : "Schedule", - "doc" : "Attributes for defining a job schedule", - "fields" : [ { - "name" : "cronSchedule", - "type" : "string", - "doc" : "Schedule for flow in cron format", - "validate" : { - "org.apache.gobblin.service.validator.CronValidator" : { } - } - }, { - "name" : "runImmediately", - "type" : "boolean", - "doc" : "Set to true to request that a job with a schedule be run immediately in addition to being scheduled", - "default" : false - } ] - }, - "doc" : "Optional schedule for when to execution the flow. If a schedule is not specified then the flow is executed immediately.", - "optional" : true - }, { - "name" : "templateUris", - "type" : "string", - "doc" : "Comma separated list of URIs for templates used in the flow. The template location is defined by the multiproduct that packages the template.", - "validate" : { - "org.apache.gobblin.service.validator.TemplateUriValidator" : { } - } - }, { - "name" : "explain", - "type" : "boolean", - "doc" : "Return the compiled flow as a string. If enabled, the flow is not added.", - "default" : false - }, { - "name" : "owningGroup", - "type" : "string", - "doc" : "Optional string name of group that the requester belongs to for group ownership of flows.", - "optional" : true - }, { - "name" : "properties", - "type" : { - "type" : "map", - "values" : "string" - }, - "doc" : "Properties for the flow. These properties are passed to the compiled Gobblin jobs." - } ] - }, "org.apache.gobblin.service.FlowId", "org.apache.gobblin.service.Schedule" ], - "schema" : { - "name" : "flowconfigs", - "namespace" : "org.apache.gobblin.service", - "path" : "/flowconfigs", - "schema" : "org.apache.gobblin.service.FlowConfig", - "doc" : "Resource for handling flow configuration requests\n\ngenerated from: org.apache.gobblin.service.FlowConfigsResource", - "collection" : { - "identifier" : { - "name" : "id", - "type" : "org.apache.gobblin.service.FlowId", - "params" : "com.linkedin.restli.common.EmptyRecord" - }, - "supports" : [ "create", "delete", "get", "update" ], - "methods" : [ { - "method" : "create", - "doc" : "Create a flow configuration that the service will forward to execution instances for execution" - }, { - "method" : "get", - "doc" : "Retrieve the flow configuration with the given key" - }, { - "method" : "update", - "doc" : "Update the flow configuration with the specified key. Running flows are not affected.\n An error is raised if the flow configuration does not exist." - }, { - "method" : "delete", - "doc" : "Delete a configured flow. Running flows are not affected. The schedule will be removed for scheduled flows." - } ], - "entity" : { - "path" : "/flowconfigs/{id}" - } - } - } -} \ No newline at end of file diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json index e6a52bc8355..e85d93df793 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json @@ -264,11 +264,14 @@ "deprecated" : "Use FlowExecution instead" }, "org.apache.gobblin.service.FlowStatusId", "org.apache.gobblin.service.Issue", "org.apache.gobblin.service.IssueSeverity", "org.apache.gobblin.service.JobId", "org.apache.gobblin.service.JobState", "org.apache.gobblin.service.JobStatistics", "org.apache.gobblin.service.JobStatus", "org.apache.gobblin.service.Timestamp" ], "schema" : { + "annotations" : { + "deprecated" : { } + }, "name" : "flowstatuses", "namespace" : "org.apache.gobblin.service", "path" : "/flowstatuses", "schema" : "org.apache.gobblin.service.FlowStatus", - "doc" : "Resource for handling flow status requests\n\ngenerated from: org.apache.gobblin.service.FlowStatusResource", + "doc" : "Resource for handling flow status requests. Deprecated, use {@link FlowExecutionResource}\n\ngenerated from: org.apache.gobblin.service.FlowStatusResource", "collection" : { "identifier" : { "name" : "id", diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/main/java/org/apache/gobblin/service/FlowConfigClient.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/main/java/org/apache/gobblin/service/FlowConfigClient.java deleted file mode 100644 index ae74f262bed..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/main/java/org/apache/gobblin/service/FlowConfigClient.java +++ /dev/null @@ -1,198 +0,0 @@ -/* - * 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; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Collections; -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Optional; -import com.linkedin.common.callback.FutureCallback; -import com.linkedin.common.util.None; -import com.linkedin.r2.RemoteInvocationException; -import com.linkedin.r2.transport.common.Client; -import com.linkedin.r2.transport.common.bridge.client.TransportClientAdapter; -import com.linkedin.r2.transport.http.client.HttpClientFactory; -import com.linkedin.restli.client.CreateIdRequest; -import com.linkedin.restli.client.DeleteRequest; -import com.linkedin.restli.client.GetRequest; -import com.linkedin.restli.client.Response; -import com.linkedin.restli.client.ResponseFuture; -import com.linkedin.restli.client.RestClient; -import com.linkedin.restli.client.UpdateRequest; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; -import com.linkedin.restli.common.IdResponse; - - -/** - * Flow Configuration client for REST flow configuration server - */ -public class FlowConfigClient implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(FlowConfigClient.class); - - private Optional _httpClientFactory; - private Optional _restClient; - private final FlowconfigsRequestBuilders _flowconfigsRequestBuilders; - public static final String DELETE_STATE_STORE_KEY = "delete.state.store"; - - /** - * Construct a {@link FlowConfigClient} to communicate with http flow config server at URI serverUri - * @param serverUri address and port of the REST server - */ - public FlowConfigClient(String serverUri) { - this(serverUri, Collections.emptyMap()); - } - - public FlowConfigClient(String serverUri, Map properties) { - LOG.debug("FlowConfigClient with serverUri " + serverUri); - - _httpClientFactory = Optional.of(new HttpClientFactory()); - Client r2Client = new TransportClientAdapter(_httpClientFactory.get().getClient(properties)); - _restClient = Optional.of(new RestClient(r2Client, serverUri)); - - _flowconfigsRequestBuilders = createRequestBuilders(); - } - - /** - * Construct a {@link FlowConfigClient} to communicate with http flow config server at URI serverUri - * @param restClient restClient to send restli request - */ - public FlowConfigClient(RestClient restClient) { - LOG.debug("FlowConfigClient with restClient " + restClient); - - _httpClientFactory = Optional.absent(); - _restClient = Optional.of(restClient); - - _flowconfigsRequestBuilders = createRequestBuilders(); - } - - // Clients using different service name can override this method - // RequestBuilders decide the name of the service requests go to. - protected FlowconfigsRequestBuilders createRequestBuilders() { - return new FlowconfigsRequestBuilders(); - } - - /** - * Create a flow configuration - * @param flowConfig flow configuration attributes - * @throws RemoteInvocationException - */ - public void createFlowConfig(FlowConfig flowConfig) - throws RemoteInvocationException { - LOG.debug("createFlowConfig with groupName " + flowConfig.getId().getFlowGroup() + " flowName " + - flowConfig.getId().getFlowName()); - - CreateIdRequest, FlowConfig> request = - _flowconfigsRequestBuilders.create().input(flowConfig).build(); - ResponseFuture>> flowConfigResponseFuture = - _restClient.get().sendRequest(request); - - flowConfigResponseFuture.getResponse(); - } - - /** - * Update a flow configuration - * @param flowConfig flow configuration attributes - * @throws RemoteInvocationException - */ - public void updateFlowConfig(FlowConfig flowConfig) - throws RemoteInvocationException { - LOG.debug("updateFlowConfig with groupName " + flowConfig.getId().getFlowGroup() + " flowName " + - flowConfig.getId().getFlowName()); - - FlowId flowId = new FlowId().setFlowGroup(flowConfig.getId().getFlowGroup()) - .setFlowName(flowConfig.getId().getFlowName()); - - UpdateRequest updateRequest = - _flowconfigsRequestBuilders.update().id(new ComplexResourceKey<>(flowId, new EmptyRecord())) - .input(flowConfig).build(); - - ResponseFuture response = _restClient.get().sendRequest(updateRequest); - - response.getResponse(); - } - - /** - * Get a flow configuration - * @param flowId identifier of flow configuration to get - * @return a {@link FlowConfig} with the flow configuration - * @throws RemoteInvocationException - */ - public FlowConfig getFlowConfig(FlowId flowId) - throws RemoteInvocationException { - LOG.debug("getFlowConfig with groupName " + flowId.getFlowGroup() + " flowName " + flowId.getFlowName()); - - GetRequest getRequest = _flowconfigsRequestBuilders.get() - .id(new ComplexResourceKey<>(flowId, new EmptyRecord())).build(); - - Response response = - _restClient.get().sendRequest(getRequest).getResponse(); - return response.getEntity(); - } - - /** - * Delete a flow configuration - * @param flowId identifier of flow configuration to delete - * @throws RemoteInvocationException - */ - public void deleteFlowConfig(FlowId flowId) - throws RemoteInvocationException { - LOG.debug("deleteFlowConfig with groupName " + flowId.getFlowGroup() + " flowName " + - flowId.getFlowName()); - - DeleteRequest deleteRequest = _flowconfigsRequestBuilders.delete() - .id(new ComplexResourceKey<>(flowId, new EmptyRecord())).build(); - ResponseFuture response = _restClient.get().sendRequest(deleteRequest); - - response.getResponse(); - } - - /** - * Delete a flow configuration - * @param flowId identifier of flow configuration to delete - * @throws RemoteInvocationException - */ - public void deleteFlowConfigWithStateStore(FlowId flowId) - throws RemoteInvocationException { - LOG.debug("deleteFlowConfig and state store with groupName " + flowId.getFlowGroup() + " flowName " + - flowId.getFlowName()); - - DeleteRequest deleteRequest = _flowconfigsRequestBuilders.delete() - .id(new ComplexResourceKey<>(flowId, new EmptyRecord())).setHeader(DELETE_STATE_STORE_KEY, Boolean.TRUE.toString()).build(); - ResponseFuture response = _restClient.get().sendRequest(deleteRequest); - - response.getResponse(); - } - - @Override - public void close() - throws IOException { - if (_restClient.isPresent()) { - _restClient.get().shutdown(new FutureCallback()); - } - - if (_httpClientFactory.isPresent()) { - _httpClientFactory.get().shutdown(new FutureCallback()); - } - } -} \ No newline at end of file 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 deleted file mode 100644 index 992e98ebe2a..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigTest.java +++ /dev/null @@ -1,337 +0,0 @@ -/* - * 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; - -import java.io.File; -import java.util.ArrayList; -import java.util.Map; - -import org.apache.commons.io.FileUtils; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.io.Files; -import com.google.inject.Binder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.name.Names; -import com.linkedin.data.template.StringMap; -import com.linkedin.r2.transport.http.client.HttpClientFactory; -import com.linkedin.restli.client.RestLiResponseException; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.server.resources.BaseResource; -import com.typesafe.config.Config; - -import org.apache.gobblin.config.ConfigBuilder; -import org.apache.gobblin.configuration.ConfigurationKeys; -import org.apache.gobblin.restli.EmbeddedRestliServer; -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.runtime.spec_store.FSSpecStore; - -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - - -@Test(groups = { "gobblin.service" }, singleThreaded = true) -public class FlowConfigTest { - private FlowConfigClient _client; - private EmbeddedRestliServer _server; - private File _testDirectory; - - private static final String TEST_SPEC_STORE_DIR = "/tmp/flowConfigTest/"; - private static final String TEST_GROUP_NAME = "testGroup1"; - private static final String TEST_FLOW_NAME = "testFlow1"; - private static final String TEST_SCHEDULE = "0 1/0 * ? * *"; - private static final String TEST_TEMPLATE_URI = "FS:///templates/test.template"; - private static final String TEST_DUMMY_GROUP_NAME = "dummyGroup"; - private static final String TEST_DUMMY_FLOW_NAME = "dummyFlow"; - - @BeforeClass - public void setUp() throws Exception { - ConfigBuilder configBuilder = ConfigBuilder.create(); - - _testDirectory = Files.createTempDir(); - - configBuilder - .addPrimitive(ConfigurationKeys.JOB_CONFIG_FILE_DIR_KEY, _testDirectory.getAbsolutePath()) - .addPrimitive(FSSpecStore.SPECSTORE_FS_DIR_KEY, TEST_SPEC_STORE_DIR); - cleanUpDir(TEST_SPEC_STORE_DIR); - - Config config = configBuilder.build(); - final FlowCatalog flowCatalog = new FlowCatalog(config); - final SpecCatalogListener mockListener = mock(SpecCatalogListener.class); - when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_ORCHESTRATOR_LISTENER_CLASS); - when(mockListener.onAddSpec(any())).thenReturn(new AddSpecResponse("")); - flowCatalog.addListener(mockListener); - flowCatalog.startAsync(); - flowCatalog.awaitRunning(); - - Injector injector = Guice.createInjector(new Module() { - @Override - public void configure(Binder binder) { - binder.bind(FlowConfigsResourceHandler.class) - .toInstance(new FlowConfigResourceLocalHandler(flowCatalog)); - - // indicate that we are in unit testing since the resource is being blocked until flow catalog changes have - // been made - binder.bindConstant().annotatedWith(Names.named(FlowConfigsResource.INJECT_READY_TO_USE)).to(Boolean.TRUE); - binder.bind(RequesterService.class).toInstance(new NoopRequesterService(config)); - } - }); - - _server = EmbeddedRestliServer.builder().resources( - Lists.>newArrayList(FlowConfigsResource.class)).injector(injector).build(); - - _server.startAsync(); - _server.awaitRunning(); - - Map transportClientProperties = Maps.newHashMap(); - transportClientProperties.put(HttpClientFactory.HTTP_REQUEST_TIMEOUT, "10000"); - _client = - new FlowConfigClient(String.format("http://localhost:%s/", _server.getPort()), transportClientProperties); - } - - private void cleanUpDir(String dir) throws Exception { - File specStoreDir = new File(dir); - if (specStoreDir.exists()) { - FileUtils.deleteDirectory(specStoreDir); - } - } - - @Test - public void testCreateBadSchedule() throws Exception { - Map flowProperties = Maps.newHashMap(); - flowProperties.put("param1", "value1"); - - FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME)) - .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule("bad schedule"). - setRunImmediately(true)) - .setProperties(new StringMap(flowProperties)); - - try { - _client.createFlowConfig(flowConfig); - } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.S_422_UNPROCESSABLE_ENTITY.getCode()); - return; - } - - Assert.fail("Get should have gotten a 422 error"); - } - - @Test - public void testCreateBadTemplateUri() throws Exception { - Map flowProperties = Maps.newHashMap(); - flowProperties.put("param1", "value1"); - - FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME)) - .setTemplateUris("FILE://bad/uri").setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE). - setRunImmediately(true)) - .setProperties(new StringMap(flowProperties)); - - try { - _client.createFlowConfig(flowConfig); - } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.S_422_UNPROCESSABLE_ENTITY.getCode()); - return; - } - - Assert.fail("Get should have gotten a 422 error"); - } - - @Test - public void testCreate() throws Exception { - Map flowProperties = Maps.newHashMap(); - flowProperties.put("param1", "value1"); - - FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME)) - .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE). - setRunImmediately(true)) - .setProperties(new StringMap(flowProperties)); - - _client.createFlowConfig(flowConfig); - } - - @Test (dependsOnMethods = "testCreate") - public void testCreateAgain() throws Exception { - Map flowProperties = Maps.newHashMap(); - flowProperties.put("param1", "value1"); - - FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME)) - .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE)) - .setProperties(new StringMap(flowProperties)); - - try { - _client.createFlowConfig(flowConfig); - } catch (RestLiResponseException e) { - Assert.fail("Create Again should pass without complaining that the spec already exists."); - } - } - - @Test (dependsOnMethods = "testCreateAgain") - public void testGet() throws Exception { - FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME); - FlowConfig flowConfig = _client.getFlowConfig(flowId); - - Assert.assertEquals(flowConfig.getId().getFlowGroup(), TEST_GROUP_NAME); - Assert.assertEquals(flowConfig.getId().getFlowName(), TEST_FLOW_NAME); - Assert.assertEquals(flowConfig.getSchedule().getCronSchedule(), TEST_SCHEDULE ); - Assert.assertEquals(flowConfig.getTemplateUris(), TEST_TEMPLATE_URI); - Assert.assertFalse(flowConfig.getSchedule().isRunImmediately()); - // Add this assert back when getFlowSpec() is changed to return the raw flow spec - //Assert.assertEquals(flowConfig.getProperties().size(), 1); - Assert.assertEquals(flowConfig.getProperties().get("param1"), "value1"); - } - - @Test (dependsOnMethods = "testGet") - public void testUpdate() throws Exception { - FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME); - - Map flowProperties = Maps.newHashMap(); - flowProperties.put("param1", "value1b"); - flowProperties.put("param2", "value2b"); - - FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME)) - .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE)) - .setProperties(new StringMap(flowProperties)); - - _client.updateFlowConfig(flowConfig); - - FlowConfig retrievedFlowConfig = _client.getFlowConfig(flowId); - - Assert.assertEquals(retrievedFlowConfig.getId().getFlowGroup(), TEST_GROUP_NAME); - Assert.assertEquals(retrievedFlowConfig.getId().getFlowName(), TEST_FLOW_NAME); - Assert.assertEquals(retrievedFlowConfig.getSchedule().getCronSchedule(), TEST_SCHEDULE ); - Assert.assertEquals(retrievedFlowConfig.getTemplateUris(), TEST_TEMPLATE_URI); - // Add this asssert when getFlowSpec() is changed to return the raw flow spec - //Assert.assertEquals(flowConfig.getProperties().size(), 2); - Assert.assertEquals(retrievedFlowConfig.getProperties().get("param1"), "value1b"); - Assert.assertEquals(retrievedFlowConfig.getProperties().get("param2"), "value2b"); - Assert.assertEquals(retrievedFlowConfig.getProperties().get(RequesterService.REQUESTER_LIST), RequesterService.serialize(new ArrayList<>())); - } - - @Test (dependsOnMethods = "testUpdate") - public void testUnschedule() throws Exception { - FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME); - Map flowProperties = Maps.newHashMap(); - flowProperties.put("param1", "value1"); - flowProperties.put(ConfigurationKeys.FLOW_UNSCHEDULE_KEY, "true"); - - FlowConfig flowConfig = new FlowConfig().setId(flowId) - .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE). - setRunImmediately(true)) - .setProperties(new StringMap(flowProperties)); - - _client.updateFlowConfig(flowConfig); - - FlowConfig persistedFlowConfig = _client.getFlowConfig(flowId); - - Assert.assertFalse(persistedFlowConfig.getProperties().containsKey(ConfigurationKeys.FLOW_UNSCHEDULE_KEY)); - Assert.assertEquals(persistedFlowConfig.getSchedule().getCronSchedule(), FlowConfigResourceLocalHandler.NEVER_RUN_CRON_SCHEDULE.getCronSchedule()); - } - - @Test (dependsOnMethods = "testUnschedule") - public void testDelete() throws Exception { - FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME); - - // make sure flow config exists - FlowConfig flowConfig = _client.getFlowConfig(flowId); - Assert.assertEquals(flowConfig.getId().getFlowGroup(), TEST_GROUP_NAME); - Assert.assertEquals(flowConfig.getId().getFlowName(), TEST_FLOW_NAME); - - _client.deleteFlowConfig(flowId); - - try { - _client.getFlowConfig(flowId); - } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); - return; - } - - Assert.fail("Get should have gotten a 404 error"); - } - - @Test - public void testBadGet() throws Exception { - FlowId flowId = new FlowId().setFlowGroup(TEST_DUMMY_GROUP_NAME).setFlowName(TEST_DUMMY_FLOW_NAME); - - try { - _client.getFlowConfig(flowId); - } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); - return; - } - - Assert.fail("Get should have raised a 404 error"); - } - - @Test - public void testBadDelete() throws Exception { - FlowId flowId = new FlowId().setFlowGroup(TEST_DUMMY_GROUP_NAME).setFlowName(TEST_DUMMY_FLOW_NAME); - - try { - _client.getFlowConfig(flowId); - } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); - return; - } - - Assert.fail("Get should have raised a 404 error"); - } - - @Test - public void testBadUpdate() throws Exception { - Map flowProperties = Maps.newHashMap(); - flowProperties.put("param1", "value1b"); - flowProperties.put("param2", "value2b"); - - FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_DUMMY_GROUP_NAME) - .setFlowName(TEST_DUMMY_FLOW_NAME)) - .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE)) - .setProperties(new StringMap(flowProperties)); - - try { - _client.updateFlowConfig(flowConfig); - } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); - return; - } - - Assert.fail("Update should have raised a 404 error"); - } - - @AfterClass(alwaysRun = true) - public void tearDown() throws Exception { - if (_client != null) { - _client.close(); - } - if (_server != null) { - _server.stopAsync(); - _server.awaitTerminated(); - } - _testDirectory.delete(); - cleanUpDir(TEST_SPEC_STORE_DIR); - } -} 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/FlowConfigsV2Test.java similarity index 65% rename from gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java rename to gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigsV2Test.java index 57a390db235..01e9d77f8f7 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/FlowConfigsV2Test.java @@ -20,12 +20,11 @@ import java.io.File; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.mortbay.jetty.HttpStatus; -import org.mockito.ArgumentMatchers; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -35,15 +34,13 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.io.Files; -import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.name.Names; import com.linkedin.data.DataMap; import com.linkedin.data.template.StringMap; import com.linkedin.r2.transport.http.client.HttpClientFactory; import com.linkedin.restli.client.RestLiResponseException; +import com.linkedin.restli.common.HttpStatus; import com.linkedin.restli.common.PatchRequest; import com.linkedin.restli.internal.server.util.DataMapUtils; import com.linkedin.restli.server.resources.BaseResource; @@ -60,23 +57,25 @@ import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse; import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; import org.apache.gobblin.runtime.spec_store.FSSpecStore; +import org.apache.gobblin.service.modules.restli.FlowConfigsV2ResourceHandler; -import static org.mockito.Mockito.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @Test(groups = { "gobblin.service" }, singleThreaded = true) -public class FlowConfigV2Test { +public class FlowConfigsV2Test { private FlowConfigV2Client _client; private EmbeddedRestliServer _server; private File _testDirectory; private TestRequesterService _requesterService; private GroupOwnershipService groupOwnershipService; - private File groupConfigFile; - private Set _compilationFailureFlowPaths = Sets.newHashSet(); + private final Set _compilationFailureFlowPaths = Sets.newHashSet(); private static final String TEST_SPEC_STORE_DIR = "/tmp/flowConfigV2Test/"; + private static final String TEST_DUMMY_GROUP_NAME = "dummyGroup"; + private static final String TEST_DUMMY_FLOW_NAME = "dummyFlow"; private static final String TEST_GROUP_NAME = "testGroup1"; private static final String TEST_FLOW_NAME = "testFlow1"; private static final String TEST_FLOW_NAME_2 = "testFlow2"; @@ -89,6 +88,7 @@ public class FlowConfigV2Test { private static final String TEST_FLOW_NAME_9 = "testFlow9"; private static final String TEST_FLOW_NAME_10 = "testFlow10"; private static final String TEST_FLOW_NAME_11 = "testFlow11"; + private static final String TEST_FLOW_NAME_12 = "testFlow12"; private static final String TEST_SCHEDULE = "0 1/0 * ? * *"; private static final String TEST_TEMPLATE_URI = "FS:///templates/test.template"; @@ -97,53 +97,43 @@ public class FlowConfigV2Test { @BeforeClass public void setUp() throws Exception { - ConfigBuilder configBuilder = ConfigBuilder.create(); - _testDirectory = Files.createTempDir(); + cleanUpDir(TEST_SPEC_STORE_DIR); - configBuilder + Config config = ConfigBuilder.create() .addPrimitive(ConfigurationKeys.JOB_CONFIG_FILE_DIR_KEY, _testDirectory.getAbsolutePath()) - .addPrimitive(FSSpecStore.SPECSTORE_FS_DIR_KEY, TEST_SPEC_STORE_DIR); - cleanUpDir(TEST_SPEC_STORE_DIR); + .addPrimitive(FSSpecStore.SPECSTORE_FS_DIR_KEY, TEST_SPEC_STORE_DIR).build(); - Config config = configBuilder.build(); final FlowCatalog flowCatalog = new FlowCatalog(config); final SpecCatalogListener mockListener = mock(SpecCatalogListener.class); when(mockListener.getName()).thenReturn(ServiceConfigKeys.GOBBLIN_ORCHESTRATOR_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) -> { - return !_compilationFailureFlowPaths.contains(flowSpec.getUri().getPath()); - }))).thenReturn(new AddSpecResponse("")); + when(mockListener.onAddSpec(any())).thenAnswer(invocation -> new AddSpecResponse(_compilationFailureFlowPaths + .contains(invocation.getArgument(0).getUri().getPath()) ? null : "") + ); + flowCatalog.addListener(mockListener); flowCatalog.startAsync(); flowCatalog.awaitRunning(); _requesterService = new TestRequesterService(ConfigFactory.empty()); - this.groupConfigFile = new File(_testDirectory + "/TestGroups.json"); + File groupConfigFile = new File(_testDirectory + "/TestGroups.json"); String groups ="{\"testGroup\": \"testName,testName2\"}"; - Files.write(groups.getBytes(), this.groupConfigFile); + Files.write(groups.getBytes(), groupConfigFile); Config groupServiceConfig = ConfigBuilder.create() - .addPrimitive(LocalGroupOwnershipService.GROUP_MEMBER_LIST, this.groupConfigFile.getAbsolutePath()) + .addPrimitive(LocalGroupOwnershipService.GROUP_MEMBER_LIST, groupConfigFile.getAbsolutePath()) .build(); groupOwnershipService = new LocalGroupOwnershipService(groupServiceConfig); - Injector injector = Guice.createInjector(new Module() { - @Override - public void configure(Binder binder) { - binder.bind(FlowConfigsV2ResourceHandler.class).toInstance(new FlowConfigV2ResourceLocalHandler(flowCatalog)); - // indicate that we are in unit testing since the resource is being blocked until flow catalog changes have - // been made - binder.bindConstant().annotatedWith(Names.named(FlowConfigsV2Resource.INJECT_READY_TO_USE)).to(Boolean.TRUE); - binder.bind(RequesterService.class).toInstance(_requesterService); - binder.bind(GroupOwnershipService.class).toInstance(groupOwnershipService); - } + Injector injector = Guice.createInjector(binder -> { + binder.bind(FlowConfigsV2ResourceHandler.class).toInstance(new FlowConfigsV2ResourceHandler("service_name", flowCatalog)); + binder.bind(RequesterService.class).toInstance(_requesterService); + binder.bind(GroupOwnershipService.class).toInstance(groupOwnershipService); }); _server = EmbeddedRestliServer.builder().resources( - Lists.>newArrayList(FlowConfigsV2Resource.class)).injector(injector).build(); + Lists.newArrayList(FlowConfigsV2Resource.class)).injector(injector).build(); _server.startAsync(); _server.awaitRunning(); @@ -161,6 +151,194 @@ protected void cleanUpDir(String dir) throws Exception { } } + @Test + public void testCreateBadSchedule() throws Exception { + Map flowProperties = Maps.newHashMap(); + flowProperties.put("param1", "value1"); + + FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME)) + .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule("bad schedule"). + setRunImmediately(true)) + .setProperties(new StringMap(flowProperties)); + + try { + _client.createFlowConfig(flowConfig); + } catch (RestLiResponseException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_422_UNPROCESSABLE_ENTITY.getCode()); + return; + } + + Assert.fail("Get should have gotten a 422 error"); + } + + @Test + public void testCreateBadTemplateUri() throws Exception { + Map flowProperties = Maps.newHashMap(); + flowProperties.put("param1", "value1"); + + FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME)) + .setTemplateUris("FILE://bad/uri").setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE). + setRunImmediately(true)) + .setProperties(new StringMap(flowProperties)); + + try { + _client.createFlowConfig(flowConfig); + } catch (RestLiResponseException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_422_UNPROCESSABLE_ENTITY.getCode()); + return; + } + + Assert.fail("Get should have gotten a 422 error"); + } + + @Test + public void testCreate() throws Exception { + Map flowProperties = Maps.newHashMap(); + flowProperties.put("param1", "value1"); + _requesterService.setRequester(TEST_REQUESTER); + + FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_12)) + .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE). + setRunImmediately(true)) + .setProperties(new StringMap(flowProperties)); + + _client.createFlowConfig(flowConfig); + } + + @Test (dependsOnMethods = "testCreate") + public void testGet() throws Exception { + FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_12); + FlowConfig flowConfig = _client.getFlowConfig(flowId); + + Assert.assertEquals(flowConfig.getId().getFlowGroup(), TEST_GROUP_NAME); + Assert.assertEquals(flowConfig.getId().getFlowName(), TEST_FLOW_NAME_12); + Assert.assertEquals(flowConfig.getSchedule().getCronSchedule(), TEST_SCHEDULE ); + Assert.assertEquals(flowConfig.getTemplateUris(), TEST_TEMPLATE_URI); + Assert.assertTrue(flowConfig.getSchedule().isRunImmediately()); + // Add this assert back when getFlowSpec() is changed to return the raw flow spec + //Assert.assertEquals(flowConfig.getProperties().size(), 1); + Assert.assertEquals(flowConfig.getProperties().get("param1"), "value1"); + } + + @Test (dependsOnMethods = "testGet") + public void testUpdate() throws Exception { + FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_12); + _requesterService.setRequester(TEST_REQUESTER); + + Map flowProperties = Maps.newHashMap(); + flowProperties.put("param1", "value1b"); + flowProperties.put("param2", "value2b"); + + FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_12)) + .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE)) + .setProperties(new StringMap(flowProperties)); + + _client.updateFlowConfig(flowConfig); + + FlowConfig retrievedFlowConfig = _client.getFlowConfig(flowId); + + Assert.assertEquals(retrievedFlowConfig.getId().getFlowGroup(), TEST_GROUP_NAME); + Assert.assertEquals(retrievedFlowConfig.getId().getFlowName(), TEST_FLOW_NAME_12); + Assert.assertEquals(Objects.requireNonNull(retrievedFlowConfig.getSchedule()).getCronSchedule(), TEST_SCHEDULE); + Assert.assertEquals(retrievedFlowConfig.getTemplateUris(), TEST_TEMPLATE_URI); + // Add this asssert when getFlowSpec() is changed to return the raw flow spec + //Assert.assertEquals(flowConfig.getProperties().size(), 2); + Assert.assertEquals(retrievedFlowConfig.getProperties().get("param1"), "value1b"); + Assert.assertEquals(retrievedFlowConfig.getProperties().get("param2"), "value2b"); + Assert.assertEquals(RequesterService.deserialize(retrievedFlowConfig.getProperties().get(RequesterService.REQUESTER_LIST)), + Lists.newArrayList(TEST_REQUESTER)); + } + + @Test (dependsOnMethods = "testUpdate") + public void testUnschedule() throws Exception { + FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_12); + Map flowProperties = Maps.newHashMap(); + flowProperties.put("param1", "value1"); + flowProperties.put(ConfigurationKeys.FLOW_UNSCHEDULE_KEY, "true"); + + FlowConfig flowConfig = new FlowConfig().setId(flowId) + .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE). + setRunImmediately(true)) + .setProperties(new StringMap(flowProperties)); + + _client.updateFlowConfig(flowConfig); + + FlowConfig persistedFlowConfig = _client.getFlowConfig(flowId); + + Assert.assertFalse(persistedFlowConfig.getProperties().containsKey(ConfigurationKeys.FLOW_UNSCHEDULE_KEY)); + Assert.assertEquals(Objects.requireNonNull(persistedFlowConfig.getSchedule()).getCronSchedule(), FlowConfigsV2ResourceHandler.NEVER_RUN_CRON_SCHEDULE.getCronSchedule()); + } + + @Test (dependsOnMethods = "testUnschedule") + public void testDelete() throws Exception { + FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_12); + + // make sure flow config exists + FlowConfig flowConfig = _client.getFlowConfig(flowId); + Assert.assertEquals(flowConfig.getId().getFlowGroup(), TEST_GROUP_NAME); + Assert.assertEquals(flowConfig.getId().getFlowName(), TEST_FLOW_NAME_12); + + _client.deleteFlowConfig(flowId); + + try { + _client.getFlowConfig(flowId); + } catch (RestLiResponseException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); + return; + } + + Assert.fail("Get should have gotten a 404 error"); + } + + @Test + public void testBadGet() throws Exception { + FlowId flowId = new FlowId().setFlowGroup(TEST_DUMMY_GROUP_NAME).setFlowName(TEST_DUMMY_FLOW_NAME); + + try { + _client.getFlowConfig(flowId); + } catch (RestLiResponseException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); + return; + } + + Assert.fail("Get should have raised a 404 error"); + } + + @Test + public void testBadDelete() throws Exception { + FlowId flowId = new FlowId().setFlowGroup(TEST_DUMMY_GROUP_NAME).setFlowName(TEST_DUMMY_FLOW_NAME); + + try { + _client.getFlowConfig(flowId); + } catch (RestLiResponseException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); + return; + } + + Assert.fail("Get should have raised a 404 error"); + } + + @Test + public void testBadUpdate() throws Exception { + Map flowProperties = Maps.newHashMap(); + flowProperties.put("param1", "value1b"); + flowProperties.put("param2", "value2b"); + + FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_DUMMY_GROUP_NAME) + .setFlowName(TEST_DUMMY_FLOW_NAME)) + .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE)) + .setProperties(new StringMap(flowProperties)); + + try { + _client.updateFlowConfig(flowConfig); + } catch (RestLiResponseException e) { + Assert.assertEquals(e.getStatus(), HttpStatus.S_404_NOT_FOUND.getCode()); + return; + } + + Assert.fail("Update should have raised a 404 error"); + } + @Test public void testCheckFlowExecutionId() throws Exception { Map flowProperties = Maps.newHashMap(); @@ -179,7 +357,7 @@ public void testCheckFlowExecutionId() throws Exception { Assert.assertEquals(_client.createFlowConfig(flowConfig).getFlowExecutionId().longValue(), -1L); } - @Test + @Test (dependsOnMethods = "testDelete") public void testCreateRejectedWhenFailsCompilation() throws Exception { FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_10); _requesterService.setRequester(TEST_REQUESTER); @@ -199,12 +377,12 @@ public void testCreateRejectedWhenFailsCompilation() throws Exception { _client.createFlowConfig(flowConfig); Assert.fail("create seemingly accepted (despite anticipated flow compilation failure)"); } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.ORDINAL_400_Bad_Request); + Assert.assertEquals(e.getStatus(), HttpStatus.S_400_BAD_REQUEST.getCode()); Assert.assertTrue(e.getMessage().contains("Flow was not compiled successfully.")); } } - @Test + @Test (dependsOnMethods = "testDelete") public void testPartialUpdate() throws Exception { FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_3); _requesterService.setRequester(TEST_REQUESTER); @@ -231,15 +409,16 @@ public void testPartialUpdate() throws Exception { FlowConfig retrievedFlowConfig = _client.getFlowConfig(flowId); - Assert.assertTrue(retrievedFlowConfig.getSchedule().isRunImmediately()); + Assert.assertTrue(Objects.requireNonNull(retrievedFlowConfig.getSchedule()).isRunImmediately()); Assert.assertEquals(retrievedFlowConfig.getProperties().get("param1"), "value1"); Assert.assertEquals(retrievedFlowConfig.getProperties().get("param2"), "value4"); Assert.assertFalse(retrievedFlowConfig.getProperties().containsKey("param3")); } - @Test (expectedExceptions = RestLiResponseException.class) + @Test (expectedExceptions = RestLiResponseException.class, dependsOnMethods = "testDelete") public void testPartialUpdateNotPossibleWithoutCreateFirst() throws Exception { - FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME); + String flowName = TEST_FLOW_NAME + 3; + FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(flowName); String patchJson = "{\"schedule\":{\"$set\":{\"runImmediately\":true}}," + "\"properties\":{\"$set\":{\"param2\":\"value4\"},\"$delete\":[\"param3\"]}}"; @@ -250,7 +429,7 @@ public void testPartialUpdateNotPossibleWithoutCreateFirst() throws Exception { _client.partialUpdateFlowConfig(flowId, flowConfigPatch); } - @Test + @Test (dependsOnMethods = "testDelete") public void testPartialUpdateRejectedWhenFailsCompilation() throws Exception { FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_11); _requesterService.setRequester(TEST_REQUESTER); @@ -279,21 +458,21 @@ public void testPartialUpdateRejectedWhenFailsCompilation() throws Exception { _client.partialUpdateFlowConfig(flowId, flowConfigPatch); Assert.fail("update seemingly accepted (despite anticipated flow compilation failure)"); } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.ORDINAL_400_Bad_Request); + Assert.assertEquals(e.getStatus(), HttpStatus.S_400_BAD_REQUEST.getCode()); Assert.assertTrue(e.getMessage().contains("Flow was not compiled successfully.")); } // verify that prior state of flow config still retained: that updates had no effect FlowConfig retrievedFlowConfig = _client.getFlowConfig(flowId); - Assert.assertTrue(!retrievedFlowConfig.getSchedule().isRunImmediately()); + Assert.assertFalse(retrievedFlowConfig.getSchedule().isRunImmediately()); Assert.assertEquals(retrievedFlowConfig.getProperties().get("param1"), "value1"); Assert.assertEquals(retrievedFlowConfig.getProperties().get("param2"), "value2"); Assert.assertEquals(retrievedFlowConfig.getProperties().get("param3"), "value3"); Assert.assertFalse(retrievedFlowConfig.getProperties().containsKey("param5")); } - @Test + @Test (dependsOnMethods = "testDelete") public void testDisallowedRequester() throws Exception { try { ServiceRequester testRequester = new ServiceRequester("testName", "testType", "testFrom"); @@ -310,11 +489,11 @@ public void testDisallowedRequester() throws Exception { testRequester.setName("testName2"); _client.deleteFlowConfig(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_4)); } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.ORDINAL_401_Unauthorized); + Assert.assertEquals(e.getStatus(), HttpStatus.S_401_UNAUTHORIZED.getCode()); } } - @Test + @Test (dependsOnMethods = "testDelete") public void testGroupRequesterAllowed() throws Exception { ServiceRequester testRequester = new ServiceRequester("testName", "USER_PRINCIPAL", "testFrom"); _requesterService.setRequester(testRequester); @@ -331,7 +510,7 @@ public void testGroupRequesterAllowed() throws Exception { _client.deleteFlowConfig(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_5)); } - @Test + @Test (dependsOnMethods = "testDelete") public void testGroupRequesterRejected() throws Exception { try { ServiceRequester testRequester = new ServiceRequester("testName", "USER_PRINCIPAL", "testFrom"); @@ -348,11 +527,11 @@ public void testGroupRequesterRejected() throws Exception { testRequester.setName("testName3"); _client.deleteFlowConfig(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_6)); } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.ORDINAL_401_Unauthorized); + Assert.assertEquals(e.getStatus(), HttpStatus.S_401_UNAUTHORIZED.getCode()); } } - @Test + @Test (dependsOnMethods = "testDelete") public void testGroupUpdateRejected() throws Exception { _requesterService.setRequester(TEST_REQUESTER); Map flowProperties = Maps.newHashMap(); @@ -370,11 +549,11 @@ public void testGroupUpdateRejected() throws Exception { _client.updateFlowConfig(flowConfig); Assert.fail("Expected update to be rejected"); } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.ORDINAL_401_Unauthorized); + Assert.assertEquals(e.getStatus(), HttpStatus.S_401_UNAUTHORIZED.getCode()); } } - @Test + @Test (dependsOnMethods = "testDelete") public void testRequesterUpdate() throws Exception { _requesterService.setRequester(TEST_REQUESTER); Map flowProperties = Maps.newHashMap(); @@ -399,7 +578,7 @@ public void testRequesterUpdate() throws Exception { Lists.newArrayList(TEST_REQUESTER2)); } - @Test + @Test (dependsOnMethods = "testDelete") public void testRequesterUpdateRejected() throws Exception { _requesterService.setRequester(TEST_REQUESTER); Map flowProperties = Maps.newHashMap(); @@ -417,11 +596,11 @@ public void testRequesterUpdateRejected() throws Exception { _client.updateFlowConfig(flowConfig); Assert.fail("Expected update to be rejected"); } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.ORDINAL_401_Unauthorized); + Assert.assertEquals(e.getStatus(), HttpStatus.S_401_UNAUTHORIZED.getCode()); } } - @Test + @Test (dependsOnMethods = "testDelete") public void testInvalidFlowId() throws Exception { Map flowProperties = Maps.newHashMap(); flowProperties.put("param1", "value1"); @@ -443,7 +622,7 @@ public void testInvalidFlowId() throws Exception { try { _client.createFlowConfig(flowConfig); } catch (RestLiResponseException e) { - Assert.assertEquals(e.getStatus(), HttpStatus.ORDINAL_422_Unprocessable_Entity); + Assert.assertEquals(e.getStatus(), HttpStatus.S_422_UNPROCESSABLE_ENTITY.getCode()); Assert.assertTrue(e.getMessage().contains("is out of range")); return; } @@ -451,7 +630,7 @@ public void testInvalidFlowId() throws Exception { Assert.fail(); } - @Test + @Test (dependsOnMethods = "testDelete") public void testRunFlow() throws Exception { String flowName = "testRunFlow"; FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(flowName); diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigV2ResourceLocalHandler.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigV2ResourceLocalHandler.java deleted file mode 100644 index cd1447ad534..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigV2ResourceLocalHandler.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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; - -import java.util.Map; - -import org.apache.commons.lang3.StringEscapeUtils; - -import com.linkedin.data.template.StringMap; -import com.linkedin.data.transform.DataProcessingException; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.common.PatchRequest; -import com.linkedin.restli.server.CreateKVResponse; -import com.linkedin.restli.server.RestLiServiceException; -import com.linkedin.restli.server.UpdateResponse; -import com.linkedin.restli.server.util.PatchApplier; - -import javax.inject.Inject; -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.configuration.ConfigurationKeys; -import org.apache.gobblin.exception.QuotaExceededException; -import org.apache.gobblin.runtime.api.FlowSpec; -import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse; -import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; - - -@Slf4j -public class FlowConfigV2ResourceLocalHandler extends FlowConfigResourceLocalHandler implements FlowConfigsV2ResourceHandler { - - @Inject - public FlowConfigV2ResourceLocalHandler(FlowCatalog flowCatalog) { - super(flowCatalog); - } - - @Override - /** - * Add flowConfig locally and trigger all listeners iff @param triggerListener is set to true - */ - public CreateKVResponse createFlowConfig(FlowConfig flowConfig, boolean triggerListener) throws FlowConfigLoggedException { - String createLog = "[GAAS-REST] Create called with flowGroup " + flowConfig.getId().getFlowGroup() + " flowName " + flowConfig.getId().getFlowName(); - this.createFlow.mark(); - - if (flowConfig.hasExplain()) { - createLog += " explain " + flowConfig.isExplain(); - } - - log.info(createLog); - FlowSpec flowSpec = createFlowSpecForConfig(flowConfig); - FlowStatusId flowStatusId = - new FlowStatusId().setFlowName(flowSpec.getConfigAsProperties().getProperty(ConfigurationKeys.FLOW_NAME_KEY)) - .setFlowGroup(flowSpec.getConfigAsProperties().getProperty(ConfigurationKeys.FLOW_GROUP_KEY)); - if (flowSpec.getConfigAsProperties().containsKey(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)) { - flowStatusId.setFlowExecutionId(Long.valueOf(flowSpec.getConfigAsProperties().getProperty(ConfigurationKeys.FLOW_EXECUTION_ID_KEY))); - } else { - flowStatusId.setFlowExecutionId(-1L); - } - - // Return conflict and take no action if flowSpec has already been created - if (this.flowCatalog.exists(flowSpec.getUri())) { - log.warn("FlowSpec with URI {} already exists, no action will be taken", flowSpec.getUri()); - return new CreateKVResponse<>(new RestLiServiceException(HttpStatus.S_409_CONFLICT, - "FlowSpec with URI " + flowSpec.getUri() + " already exists, no action will be taken")); - } - - Map responseMap; - try { - responseMap = this.flowCatalog.put(flowSpec, triggerListener); - } catch (QuotaExceededException e) { - throw new RestLiServiceException(HttpStatus.S_503_SERVICE_UNAVAILABLE, e.getMessage()); - } catch (Throwable e) { - // TODO: Compilation errors should fall under throwable exceptions as well instead of checking for strings - log.warn(String.format("Failed to add flow configuration %s.%s to catalog due to", flowConfig.getId().getFlowGroup(), flowConfig.getId().getFlowName()), e); - throw new RestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, e.getMessage()); - } - HttpStatus httpStatus; - - if (flowConfig.hasExplain() && flowConfig.isExplain()) { - //This is an Explain request. So no resource is actually created. - //Enrich original FlowConfig entity by adding the compiledFlow to the properties map. - StringMap props = flowConfig.getProperties(); - AddSpecResponse addSpecResponse = responseMap.getOrDefault(ServiceConfigKeys.COMPILATION_RESPONSE, null); - props.put("gobblin.flow.compiled", - addSpecResponse != null && addSpecResponse.getValue() != null ? StringEscapeUtils.escapeJson(addSpecResponse.getValue()) : ""); - flowConfig.setProperties(props); - httpStatus = HttpStatus.S_200_OK; - } else if (Boolean.parseBoolean(responseMap.getOrDefault(ServiceConfigKeys.COMPILATION_SUCCESSFUL, new AddSpecResponse<>("false")).getValue().toString())) { - httpStatus = HttpStatus.S_201_CREATED; - } else { - throw new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, getErrorMessage(flowSpec)); - } - - return new CreateKVResponse<>(new ComplexResourceKey<>(flowConfig.getId(), flowStatusId), flowConfig, httpStatus); - } - - /** - * Note: this method is only implemented for testing, normally partial update would be called in - * GobblinServiceFlowConfigResourceHandler.partialUpdateFlowConfig - */ - @Override - public UpdateResponse partialUpdateFlowConfig(FlowId flowId, PatchRequest flowConfigPatch) throws FlowConfigLoggedException { - FlowConfig flowConfig = getFlowConfig(flowId); - - try { - PatchApplier.applyPatch(flowConfig, flowConfigPatch); - } catch (DataProcessingException e) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, "Failed to apply partial update", e); - } - - return updateFlowConfig(flowId, flowConfig); - } -} 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 deleted file mode 100644 index 95c0d37a9e9..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResource.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * 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; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.collect.ImmutableSet; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.server.CreateResponse; -import com.linkedin.restli.server.UpdateResponse; -import com.linkedin.restli.server.annotations.RestLiCollection; -import com.linkedin.restli.server.resources.ComplexKeyResourceTemplate; - -import javax.inject.Inject; -import javax.inject.Named; - -/** - * Resource for handling flow configuration requests - */ -@RestLiCollection(name = "flowconfigs", namespace = "org.apache.gobblin.service", keyName = "id") -public class FlowConfigsResource extends ComplexKeyResourceTemplate { - private static final Logger LOG = LoggerFactory.getLogger(FlowConfigsResource.class); - - public static final String INJECT_READY_TO_USE = "readToUse"; - - private static final Set ALLOWED_METADATA = ImmutableSet.of("delete.state.store"); - - @Inject - private FlowConfigsResourceHandler flowConfigsResourceHandler; - - // For getting who sends the request - @Inject - private RequesterService requesterService; - - // For blocking use of this resource until it is ready - @Inject - @Named(INJECT_READY_TO_USE) - private Boolean readyToUse; - - public FlowConfigsResource() { - } - - /** - * Retrieve the flow configuration with the given key - * @param key flow config id key containing group name and flow name - * @return {@link FlowConfig} with flow configuration - */ - @Override - public FlowConfig get(ComplexResourceKey key) { - String flowGroup = key.getKey().getFlowGroup(); - String flowName = key.getKey().getFlowName(); - FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName); - return this.flowConfigsResourceHandler.getFlowConfig(flowId); - } - - /** - * Create a flow configuration that the service will forward to execution instances for execution - * @param flowConfig flow configuration - * @return {@link CreateResponse} - */ - @Override - public CreateResponse create(FlowConfig flowConfig) { - if (flowConfig.hasOwningGroup()) { - throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "Owning group property may " - + "not be set through flowconfigs API, use flowconfigsV2"); - } - - List requesterList = this.requesterService.findRequesters(this); - - try { - String serialized = RequesterService.serialize(requesterList); - flowConfig.getProperties().put(RequesterService.REQUESTER_LIST, serialized); - LOG.info("Rest requester list is " + serialized); - } catch (IOException e) { - throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "cannot get who is the requester", e); - } - return this.flowConfigsResourceHandler.createFlowConfig(flowConfig); - } - - /** - * Update the flow configuration with the specified key. Running flows are not affected. - * An error is raised if the flow configuration does not exist. - * @param key composite key containing group name and flow name that identifies the flow to update - * @param flowConfig new flow configuration - * @return {@link UpdateResponse} - */ - @Override - public UpdateResponse update(ComplexResourceKey key, FlowConfig flowConfig) { - if (flowConfig.hasOwningGroup()) { - throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "Owning group property may " - + "not be set through flowconfigs API, use flowconfigsV2"); - } - if (flowConfig.getProperties().containsKey(RequesterService.REQUESTER_LIST)) { - throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, RequesterService.REQUESTER_LIST + " property may " - + "not be set through flowconfigs API, use flowconfigsV2"); - } - checkRequester(this.requesterService, get(key), this.requesterService.findRequesters(this)); - String flowGroup = key.getKey().getFlowGroup(); - String flowName = key.getKey().getFlowName(); - FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName); - return this.flowConfigsResourceHandler.updateFlowConfig(flowId, flowConfig); - } - - /** - * Delete a configured flow. Running flows are not affected. The schedule will be removed for scheduled flows. - * @param key composite key containing flow group and flow name that identifies the flow to remove from the flow catalog - * @return {@link UpdateResponse} - */ - @Override - public UpdateResponse delete(ComplexResourceKey key) { - checkRequester(this.requesterService, get(key), this.requesterService.findRequesters(this)); - String flowGroup = key.getKey().getFlowGroup(); - String flowName = key.getKey().getFlowName(); - FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName); - return this.flowConfigsResourceHandler.deleteFlowConfig(flowId, getHeaders()); - } - - /** - * Check that all {@link ServiceRequester}s in this request are contained within the original service requester list - * when the flow was submitted. If they are not, throw a {@link FlowConfigLoggedException} with {@link HttpStatus#S_401_UNAUTHORIZED}. - * If there is a failure when deserializing the original requester list, throw a {@link FlowConfigLoggedException} with - * {@link HttpStatus#S_400_BAD_REQUEST}. - * - * @param requesterService the {@link RequesterService} used to verify the requester - * @param originalFlowConfig original flow config to find original requester - * @param requesterList list of requesters for this request - */ - public static void checkRequester( - RequesterService requesterService, FlowConfig originalFlowConfig, List requesterList) { - if (requesterService.isRequesterWhitelisted(requesterList)) { - return; - } - - try { - String serializedOriginalRequesterList = originalFlowConfig.getProperties().get(RequesterService.REQUESTER_LIST); - if (serializedOriginalRequesterList != null) { - List originalRequesterList = RequesterService.deserialize(serializedOriginalRequesterList); - if (!requesterService.isRequesterAllowed(originalRequesterList, requesterList)) { - throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "Requester not allowed to make this request"); - } - } - } catch (IOException e) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, "Failed to get original requester list", e); - } - } - - private Properties getHeaders() { - Properties headerProperties = new Properties(); - for (Map.Entry entry : getContext().getRequestHeaders().entrySet()) { - if (ALLOWED_METADATA.contains(entry.getKey())) { - headerProperties.put(entry.getKey(), entry.getValue()); - } - } - return headerProperties; - } -} - diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandler.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandler.java deleted file mode 100644 index 6f6ace9c99c..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandler.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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; - -import java.util.Collection; -import java.util.Properties; - -import com.linkedin.restli.common.PatchRequest; -import com.linkedin.restli.server.CreateResponse; -import com.linkedin.restli.server.UpdateResponse; - -import org.apache.gobblin.runtime.api.FlowSpecSearchObject; - - -public interface FlowConfigsResourceHandler { - /** - * Get {@link FlowConfig} - */ - FlowConfig getFlowConfig(FlowId flowId) throws FlowConfigLoggedException; - /** - * Get {@link FlowConfig} - * @return - */ - Collection getFlowConfig(FlowSpecSearchObject flowSpecSearchObject) throws FlowConfigLoggedException; - /** - * Get all {@link FlowConfig} - */ - Collection getAllFlowConfigs(); - /** - * Get all {@link FlowConfig} with pagination - */ - Collection getAllFlowConfigs(int start, int count); - - /** - * Add {@link FlowConfig} - */ - CreateResponse createFlowConfig(FlowConfig flowConfig) throws FlowConfigLoggedException; - - /** - * Update {@link FlowConfig} - */ - UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig) throws FlowConfigLoggedException; - - /** - * Partial update a {@link FlowConfig} - */ - UpdateResponse partialUpdateFlowConfig(FlowId flowId, PatchRequest flowConfig) throws FlowConfigLoggedException; - - /** - * Delete {@link FlowConfig} - */ - UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) throws FlowConfigLoggedException; - -} 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..0c08694474b 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 @@ -53,10 +53,10 @@ import com.linkedin.restli.server.util.PatchApplier; import javax.inject.Inject; -import javax.inject.Named; import lombok.extern.slf4j.Slf4j; import org.apache.gobblin.runtime.api.FlowSpecSearchObject; +import org.apache.gobblin.service.modules.restli.FlowConfigsV2ResourceHandler; /** @@ -66,13 +66,8 @@ @RestLiCollection(name = "flowconfigsV2", namespace = "org.apache.gobblin.service", 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"; private static final Set ALLOWED_METADATA = ImmutableSet.of("delete.state.store"); - - @edu.umd.cs.findbugs.annotations.SuppressWarnings("MS_SHOULD_BE_FINAL") - public static FlowConfigsResourceHandler global_flowConfigsResourceHandler = null; - @Inject private FlowConfigsV2ResourceHandler flowConfigsResourceHandler; @@ -80,11 +75,6 @@ public class FlowConfigsV2Resource extends ComplexKeyResourceTemplate key) { - return this.getFlowConfigResourceHandler().getFlowConfig(key.getKey()); + return this.flowConfigsResourceHandler.getFlowConfig(key.getKey()); } /** @@ -108,9 +98,9 @@ public FlowConfig get(ComplexResourceKey key) { public List getAll(@Context PagingContext pagingContext) { // Check to see if the count and start parameters are user defined or default from the framework if (!pagingContext.hasCount() && !pagingContext.hasStart()) - return (List) this.getFlowConfigResourceHandler().getAllFlowConfigs(); + return (List) this.flowConfigsResourceHandler.getAllFlowConfigs(); else { - return (List) this.getFlowConfigResourceHandler().getAllFlowConfigs(pagingContext.getStart(), pagingContext.getCount()); + return (List) this.flowConfigsResourceHandler.getAllFlowConfigs(pagingContext.getStart(), pagingContext.getCount()); } } @@ -145,7 +135,7 @@ public List getFilteredFlows(@Context PagingContext context, isRunImmediately, owningGroup, propertyFilter, context.getStart(), context.getCount()); } - return (List) this.getFlowConfigResourceHandler().getFlowConfig(flowSpecSearchObject); + return (List) this.flowConfigsResourceHandler.getFlowConfig(flowSpecSearchObject); } /** @@ -155,7 +145,7 @@ public List getFilteredFlows(@Context PagingContext context, */ @ReturnEntity @Override - public CreateKVResponse create(FlowConfig flowConfig) { + public CreateKVResponse, FlowConfig> create(FlowConfig flowConfig) { List requesterList = this.requesterService.findRequesters(this); try { String serialized = RequesterService.serialize(requesterList); @@ -167,7 +157,7 @@ public CreateKVResponse create(FlowConfig flowConfig) { } catch (IOException e) { throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "cannot get who is the requester", e); } - return (CreateKVResponse) this.getFlowConfigResourceHandler().createFlowConfig(flowConfig); + return this.flowConfigsResourceHandler.createFlowConfig(flowConfig); } /** @@ -183,7 +173,7 @@ public UpdateResponse update(ComplexResourceKey key, FlowC String flowGroup = key.getKey().getFlowGroup(); String flowName = key.getKey().getFlowName(); FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName); - return this.getFlowConfigResourceHandler().updateFlowConfig(flowId, flowConfig); + return this.flowConfigsResourceHandler.updateFlowConfig(flowId, flowConfig); } /** @@ -205,7 +195,7 @@ public UpdateResponse update(ComplexResourceKey key, Patch String flowGroup = key.getKey().getFlowGroup(); String flowName = key.getKey().getFlowName(); FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName); - return this.getFlowConfigResourceHandler().partialUpdateFlowConfig(flowId, flowConfigPatch); + return this.flowConfigsResourceHandler.partialUpdateFlowConfig(flowId, flowConfigPatch); } /** @@ -219,7 +209,7 @@ public UpdateResponse delete(ComplexResourceKey key) { String flowGroup = key.getKey().getFlowGroup(); String flowName = key.getKey().getFlowName(); FlowId flowId = new FlowId().setFlowGroup(flowGroup).setFlowName(flowName); - return this.getFlowConfigResourceHandler().deleteFlowConfig(flowId, getHeaders()); + return this.flowConfigsResourceHandler.deleteFlowConfig(flowId, getHeaders()); } /** @@ -236,13 +226,6 @@ public String runImmediately(@PathKeysParam PathKeys pathKeys) { return "Successfully triggered flow " + id.getKey().toString(); } - private FlowConfigsResourceHandler getFlowConfigResourceHandler() { - if (global_flowConfigsResourceHandler != null) { - return global_flowConfigsResourceHandler; - } - return flowConfigsResourceHandler; - } - private Properties getHeaders() { Properties headerProperties = new Properties(); for (Map.Entry entry : getContext().getRequestHeaders().entrySet()) { @@ -277,8 +260,8 @@ public void checkPropertyUpdatesAllowed(List requesterList, Fl // Check that requester is part of owning group if owning group is being updated if (updatedFlowConfig.hasOwningGroup() && !this.groupOwnershipService.isMemberOfGroup(requesterList, updatedFlowConfig.getOwningGroup())) { - throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "Requester not part of owning group specified. Requester " + requesterList - + " should join group " + updatedFlowConfig.getOwningGroup() + " and retry."); + throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "Requester not part of owning group specified. Requester " + + requesterList + " should join group " + updatedFlowConfig.getOwningGroup() + " and retry."); } if (updatedFlowConfig.hasProperties() && updatedFlowConfig.getProperties().containsKey(RequesterService.REQUESTER_LIST)) { diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2ResourceHandler.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2ResourceHandler.java deleted file mode 100644 index 7528cc87ad2..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2ResourceHandler.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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 interface FlowConfigsV2ResourceHandler extends FlowConfigsResourceHandler { -} 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..d630069f0b5 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 @@ -17,9 +17,19 @@ package org.apache.gobblin.service; +import java.time.Duration; +import java.time.Instant; +import java.util.Comparator; +import java.util.Iterator; import java.util.List; +import java.util.stream.Collectors; +import org.apache.commons.lang3.ObjectUtils; + +import com.google.common.base.Strings; import com.google.inject.Inject; +import com.linkedin.data.template.SetMode; +import com.linkedin.data.template.StringMap; import com.linkedin.restli.common.ComplexResourceKey; import com.linkedin.restli.common.EmptyRecord; import com.linkedin.restli.server.PagingContext; @@ -35,15 +45,22 @@ import com.linkedin.restli.server.annotations.RestLiCollection; import com.linkedin.restli.server.resources.ComplexKeyResourceTemplate; +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.service.monitoring.FlowStatus; +import org.apache.gobblin.service.monitoring.FlowStatusGenerator; +import org.apache.gobblin.service.monitoring.JobStatusRetriever; + /** * Resource for handling flow execution requests */ +@Slf4j @RestLiCollection(name = "flowexecutions", namespace = "org.apache.gobblin.service", keyName = "id") public class FlowExecutionResource extends ComplexKeyResourceTemplate { @Inject - FlowExecutionResourceHandler flowExecutionResourceHandler; + FlowExecutionResourceHandlerInterface flowExecutionResourceHandler; public FlowExecutionResource() {} @@ -99,5 +116,157 @@ public void resume(@PathKeysParam PathKeys pathKeys) { public UpdateResponse delete(ComplexResourceKey key) { return this.flowExecutionResourceHandler.delete(key); } + + public static List getLatestFlowStatusesFromGenerator(FlowId flowId, + Integer count, String tag, String executionStatus, FlowStatusGenerator flowStatusGenerator) { + if (count == null) { + count = 1; + } + log.info("get latest called with flowGroup " + flowId.getFlowGroup() + " flowName " + flowId.getFlowName() + " count " + count); + + return flowStatusGenerator.getLatestFlowStatus(flowId.getFlowName(), flowId.getFlowGroup(), count, tag, executionStatus); + } + + public static FlowStatus getFlowStatusFromGenerator(ComplexResourceKey key, + FlowStatusGenerator flowStatusGenerator) { + String flowGroup = key.getKey().getFlowGroup(); + String flowName = key.getKey().getFlowName(); + long flowExecutionId = key.getKey().getFlowExecutionId(); + + log.info("Get called with flowGroup " + flowGroup + " flowName " + flowName + " flowExecutionId " + flowExecutionId); + + return flowStatusGenerator.getFlowStatus(flowName, flowGroup, flowExecutionId, null); + } + + /** + * Forms a {@link FlowExecution} from a {@link FlowStatus} + * @param monitoringFlowStatus + * @return a {@link FlowExecution} converted from a {@link FlowStatus} + */ + public static FlowExecution convertFlowStatus(FlowStatus monitoringFlowStatus, + boolean includeIssues) { + if (monitoringFlowStatus == null) { + return null; + } + + Iterator jobStatusIter = monitoringFlowStatus.getJobStatusIterator(); + JobStatusArray jobStatusArray = new JobStatusArray(); + FlowId flowId = new FlowId().setFlowName(monitoringFlowStatus.getFlowName()) + .setFlowGroup(monitoringFlowStatus.getFlowGroup()); + + long flowEndTime = 0L; + long maxJobEndTime = Long.MIN_VALUE; + String flowMessage = ""; + + while (jobStatusIter.hasNext()) { + org.apache.gobblin.service.monitoring.JobStatus queriedJobStatus = jobStatusIter.next(); + + // Check if this is the flow status instead of a single job status + if (JobStatusRetriever.isFlowStatus(queriedJobStatus)) { + flowEndTime = queriedJobStatus.getEndTime(); + if (queriedJobStatus.getMessage() != null) { + flowMessage = queriedJobStatus.getMessage(); + } + continue; + } + + maxJobEndTime = Math.max(maxJobEndTime, queriedJobStatus.getEndTime()); + + org.apache.gobblin.service.JobStatus jobStatus = new org.apache.gobblin.service.JobStatus(); + + Long timeLeft = estimateCopyTimeLeft(queriedJobStatus.getLastProgressEventTime(), queriedJobStatus.getStartTime(), + queriedJobStatus.getProgressPercentage()); + + jobStatus.setFlowId(flowId) + .setJobId(new JobId() + .setJobName(queriedJobStatus.getJobName()) + .setJobGroup(queriedJobStatus.getJobGroup())) + .setJobTag(queriedJobStatus.getJobTag(), SetMode.IGNORE_NULL) + .setExecutionStatistics(new JobStatistics() + .setExecutionStartTime(queriedJobStatus.getStartTime()) + .setExecutionEndTime(queriedJobStatus.getEndTime()) + .setProcessedCount(queriedJobStatus.getProcessedCount()) + .setJobProgress(queriedJobStatus.getProgressPercentage()) + .setEstimatedSecondsToCompletion(timeLeft)) + .setExecutionStatus(ExecutionStatus.valueOf(queriedJobStatus.getEventName())) + .setMessage(queriedJobStatus.getMessage()) + .setJobState(new JobState() + .setLowWatermark(queriedJobStatus.getLowWatermark()). + setHighWatermark(queriedJobStatus.getHighWatermark())); + + if (includeIssues) { + jobStatus.setIssues(new IssueArray(queriedJobStatus.getIssues().get().stream() + .map(FlowExecutionResource::convertIssueToRestApiObject) + .collect(Collectors.toList()))); + } else { + jobStatus.setIssues(new IssueArray()); + } + + if (!Strings.isNullOrEmpty(queriedJobStatus.getMetrics())) { + jobStatus.setMetrics(queriedJobStatus.getMetrics()); + } + + jobStatusArray.add(jobStatus); + } + + flowEndTime = flowEndTime == 0L ? maxJobEndTime : flowEndTime; + + jobStatusArray.sort(Comparator.comparing((org.apache.gobblin.service.JobStatus js) -> js.getExecutionStatistics().getExecutionStartTime())); + + return new FlowExecution() + .setId(new FlowStatusId().setFlowGroup(flowId.getFlowGroup()).setFlowName(flowId.getFlowName()) + .setFlowExecutionId(monitoringFlowStatus.getFlowExecutionId())) + .setExecutionStatistics(new FlowStatistics().setExecutionStartTime(getFlowStartTime(monitoringFlowStatus)) + .setExecutionEndTime(flowEndTime)) + .setMessage(flowMessage) + .setExecutionStatus(monitoringFlowStatus.getFlowExecutionStatus()) + .setJobStatuses(jobStatusArray); + } + + private static org.apache.gobblin.service.Issue convertIssueToRestApiObject( + org.apache.gobblin.runtime.troubleshooter.Issue issues) { + org.apache.gobblin.service.Issue converted = new org.apache.gobblin.service.Issue(); + + converted.setCode(issues.getCode()) + .setSummary(ObjectUtils.firstNonNull(issues.getSummary(), "")) + .setDetails(ObjectUtils.firstNonNull(issues.getDetails(), "")) + .setSeverity(IssueSeverity.valueOf(issues.getSeverity().name())) + .setTime(issues.getTime().toInstant().toEpochMilli()); + + if (issues.getProperties() != null) { + converted.setProperties(new StringMap(issues.getProperties())); + } else { + converted.setProperties(new StringMap()); + } + + return converted; + } + + /** + * Return the flow start time given a {@link FlowStatus}. Flow execution ID is + * assumed to be the flow start time. + */ + private static long getFlowStartTime(FlowStatus flowStatus) { + return flowStatus.getFlowExecutionId(); + } + + /** + * Estimate the time left to complete the copy based on the following formula - + * timeLeft = (100/completionPercentage - 1) * timeElapsed + * @param currentTime as an epoch + * @param startTime as an epoch + * @param completionPercentage of the job + * @return time left in seconds + */ + public static long estimateCopyTimeLeft(Long currentTime, Long startTime, int completionPercentage) { + if (completionPercentage == 0) { + return 0; + } + + Instant current = Instant.ofEpochMilli(currentTime); + Instant start = Instant.ofEpochMilli(startTime); + long timeElapsed = Duration.between(start, current).getSeconds(); + return (long) (timeElapsed * (100.0 / (double) completionPercentage - 1)); + } } diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandler.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java similarity index 70% rename from gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandler.java rename to gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java index 8b2935f0d52..48e3ac149d7 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandler.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java @@ -25,16 +25,18 @@ import com.linkedin.restli.server.UpdateResponse; -public interface FlowExecutionResourceHandler { +// Unlike FlowConfigsV2ResourceHandler, this is an interface rather than a class because it's implementation needs +// classes from gobblin-service module, and adding gobblin-service as a dependency will cause circular dependency, +public interface FlowExecutionResourceHandlerInterface { /** * Get {@link FlowExecution} */ - public FlowExecution get(ComplexResourceKey key); + FlowExecution get(ComplexResourceKey key); /** * Get latest {@link FlowExecution} */ - public List getLatestFlowExecution(PagingContext context, FlowId flowId, Integer count, String tag, + List getLatestFlowExecution(PagingContext context, FlowId flowId, Integer count, String tag, String executionStatus, Boolean includeIssues); /** @@ -43,16 +45,16 @@ public List getLatestFlowExecution(PagingContext context, FlowId * NOTE: `executionStatus` param not provided yet, without justifying use case, due to complexity of interaction with `countPerFlow` * and resulting efficiency concern of performing across many flows sharing the single named group. */ - public List getLatestFlowGroupExecutions(PagingContext context, String flowGroup, Integer countPerFLow, + List getLatestFlowGroupExecutions(PagingContext context, String flowGroup, Integer countPerFLow, String tag, Boolean includeIssues); /** * Resume a failed {@link FlowExecution} from the point before failure */ - public void resume(ComplexResourceKey key); + void resume(ComplexResourceKey key); /** * Kill a running {@link FlowExecution} */ - public UpdateResponse delete(ComplexResourceKey key); + UpdateResponse delete(ComplexResourceKey key); } diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandler.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandler.java deleted file mode 100644 index 3a03d71e4d9..00000000000 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandler.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * 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; - -import java.time.Duration; -import java.time.Instant; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.stream.Collectors; - -import org.apache.commons.lang3.ObjectUtils; - -import com.google.common.base.Strings; -import com.linkedin.data.template.SetMode; -import com.linkedin.data.template.StringMap; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.server.PagingContext; -import com.linkedin.restli.server.RestLiServiceException; -import com.linkedin.restli.server.UpdateResponse; - -import javax.inject.Inject; -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.runtime.troubleshooter.Issue; -import org.apache.gobblin.service.monitoring.FlowStatus; -import org.apache.gobblin.service.monitoring.FlowStatusGenerator; -import org.apache.gobblin.service.monitoring.JobStatusRetriever; - - -@Slf4j -public class FlowExecutionResourceLocalHandler implements FlowExecutionResourceHandler { - - private final FlowStatusGenerator flowStatusGenerator; - - @Inject - public FlowExecutionResourceLocalHandler(FlowStatusGenerator flowStatusGenerator) { - this.flowStatusGenerator = flowStatusGenerator; - } - - @Override - public FlowExecution get(ComplexResourceKey key) { - FlowExecution flowExecution = convertFlowStatus(getFlowStatusFromGenerator(key, this.flowStatusGenerator), true); - if (flowExecution == null) { - throw new RestLiServiceException(HttpStatus.S_404_NOT_FOUND, "No flow execution found for flowStatusId " + key.getKey() - + ". The flowStatusId may be incorrect, or the flow execution may have been cleaned up."); - } - return flowExecution; - } - - @Override - public List getLatestFlowExecution(PagingContext context, FlowId flowId, Integer count, String tag, - String executionStatus, Boolean includeIssues) { - List flowStatuses = getLatestFlowStatusesFromGenerator(flowId, count, tag, executionStatus, this.flowStatusGenerator); - - if (flowStatuses != null) { - return flowStatuses.stream() - .map((FlowStatus monitoringFlowStatus) -> convertFlowStatus(monitoringFlowStatus, includeIssues)) - .collect(Collectors.toList()); - } - - throw new RestLiServiceException(HttpStatus.S_404_NOT_FOUND, "No flow execution found for flowId " + flowId - + ". The flowId may be incorrect, the flow execution may have been cleaned up, or not matching tag (" + tag - + ") and/or execution status (" + executionStatus + ")."); - } - - @Override - public List getLatestFlowGroupExecutions(PagingContext context, String flowGroup, Integer countPerFlow, - String tag, Boolean includeIssues) { - List flowStatuses = - getLatestFlowGroupStatusesFromGenerator(flowGroup, countPerFlow, tag, this.flowStatusGenerator); - - if (flowStatuses != null) { - // todo: flow end time will be incorrect when dag manager is not used - // and FLOW_SUCCEEDED/FLOW_CANCELLED/FlowFailed events are not sent - return flowStatuses.stream() - .map((FlowStatus monitoringFlowStatus) -> convertFlowStatus(monitoringFlowStatus, includeIssues)) - .collect(Collectors.toList()); - } - - throw new RestLiServiceException(HttpStatus.S_404_NOT_FOUND, "No flow executions found for flowGroup " + flowGroup - + ". The group name may be incorrect, the flow execution may have been cleaned up, or not matching tag (" + tag - + ")."); - } - - @Override - public void resume(ComplexResourceKey key) { - throw new UnsupportedOperationException("Resume should be handled in GobblinServiceFlowConfigResourceHandler"); - } - - @Override - public UpdateResponse delete(ComplexResourceKey key) { - throw new UnsupportedOperationException("Delete should be handled in GobblinServiceFlowConfigResourceHandler"); - } - - public static org.apache.gobblin.service.monitoring.FlowStatus getFlowStatusFromGenerator(ComplexResourceKey key, - FlowStatusGenerator flowStatusGenerator) { - String flowGroup = key.getKey().getFlowGroup(); - String flowName = key.getKey().getFlowName(); - long flowExecutionId = key.getKey().getFlowExecutionId(); - - log.info("Get called with flowGroup " + flowGroup + " flowName " + flowName + " flowExecutionId " + flowExecutionId); - - return flowStatusGenerator.getFlowStatus(flowName, flowGroup, flowExecutionId, null); - } - - public static List getLatestFlowStatusesFromGenerator(FlowId flowId, - Integer count, String tag, String executionStatus, FlowStatusGenerator flowStatusGenerator) { - if (count == null) { - count = 1; - } - log.info("get latest called with flowGroup " + flowId.getFlowGroup() + " flowName " + flowId.getFlowName() + " count " + count); - - return flowStatusGenerator.getLatestFlowStatus(flowId.getFlowName(), flowId.getFlowGroup(), count, tag, executionStatus); - } - - public static List getLatestFlowGroupStatusesFromGenerator(String flowGroup, - Integer countPerFlowName, String tag, FlowStatusGenerator flowStatusGenerator) { - if (countPerFlowName == null) { - countPerFlowName = 1; - } - log.info("get latest (for group) called with flowGroup " + flowGroup + " count " + countPerFlowName); - - return flowStatusGenerator.getFlowStatusesAcrossGroup(flowGroup, countPerFlowName, tag); - } - - /** - * Forms a {@link FlowExecution} from a {@link org.apache.gobblin.service.monitoring.FlowStatus} - * @param monitoringFlowStatus - * @return a {@link FlowExecution} converted from a {@link org.apache.gobblin.service.monitoring.FlowStatus} - */ - public static FlowExecution convertFlowStatus(org.apache.gobblin.service.monitoring.FlowStatus monitoringFlowStatus, - boolean includeIssues) { - if (monitoringFlowStatus == null) { - return null; - } - - Iterator jobStatusIter = monitoringFlowStatus.getJobStatusIterator(); - JobStatusArray jobStatusArray = new JobStatusArray(); - FlowId flowId = new FlowId().setFlowName(monitoringFlowStatus.getFlowName()) - .setFlowGroup(monitoringFlowStatus.getFlowGroup()); - - long flowEndTime = 0L; - long maxJobEndTime = Long.MIN_VALUE; - String flowMessage = ""; - - while (jobStatusIter.hasNext()) { - org.apache.gobblin.service.monitoring.JobStatus queriedJobStatus = jobStatusIter.next(); - - // Check if this is the flow status instead of a single job status - if (JobStatusRetriever.isFlowStatus(queriedJobStatus)) { - flowEndTime = queriedJobStatus.getEndTime(); - if (queriedJobStatus.getMessage() != null) { - flowMessage = queriedJobStatus.getMessage(); - } - continue; - } - - maxJobEndTime = Math.max(maxJobEndTime, queriedJobStatus.getEndTime()); - - JobStatus jobStatus = new JobStatus(); - - Long timeLeft = estimateCopyTimeLeft(queriedJobStatus.getLastProgressEventTime(), queriedJobStatus.getStartTime(), - queriedJobStatus.getProgressPercentage()); - - jobStatus.setFlowId(flowId) - .setJobId(new JobId() - .setJobName(queriedJobStatus.getJobName()) - .setJobGroup(queriedJobStatus.getJobGroup())) - .setJobTag(queriedJobStatus.getJobTag(), SetMode.IGNORE_NULL) - .setExecutionStatistics(new JobStatistics() - .setExecutionStartTime(queriedJobStatus.getStartTime()) - .setExecutionEndTime(queriedJobStatus.getEndTime()) - .setProcessedCount(queriedJobStatus.getProcessedCount()) - .setJobProgress(queriedJobStatus.getProgressPercentage()) - .setEstimatedSecondsToCompletion(timeLeft)) - .setExecutionStatus(ExecutionStatus.valueOf(queriedJobStatus.getEventName())) - .setMessage(queriedJobStatus.getMessage()) - .setJobState(new JobState() - .setLowWatermark(queriedJobStatus.getLowWatermark()). - setHighWatermark(queriedJobStatus.getHighWatermark())); - - if (includeIssues) { - jobStatus.setIssues(new IssueArray(queriedJobStatus.getIssues().get().stream() - .map(FlowExecutionResourceLocalHandler::convertIssueToRestApiObject) - .collect(Collectors.toList()))); - } else { - jobStatus.setIssues(new IssueArray()); - } - - if (!Strings.isNullOrEmpty(queriedJobStatus.getMetrics())) { - jobStatus.setMetrics(queriedJobStatus.getMetrics()); - } - - jobStatusArray.add(jobStatus); - } - - // If DagManager is not enabled, we have to determine flow end time by individual job's end times. - flowEndTime = flowEndTime == 0L ? maxJobEndTime : flowEndTime; - - jobStatusArray.sort(Comparator.comparing((JobStatus js) -> js.getExecutionStatistics().getExecutionStartTime())); - - return new FlowExecution() - .setId(new FlowStatusId().setFlowGroup(flowId.getFlowGroup()).setFlowName(flowId.getFlowName()) - .setFlowExecutionId(monitoringFlowStatus.getFlowExecutionId())) - .setExecutionStatistics(new FlowStatistics().setExecutionStartTime(getFlowStartTime(monitoringFlowStatus)) - .setExecutionEndTime(flowEndTime)) - .setMessage(flowMessage) - .setExecutionStatus(monitoringFlowStatus.getFlowExecutionStatus()) - .setJobStatuses(jobStatusArray); - } - - private static org.apache.gobblin.service.Issue convertIssueToRestApiObject(Issue issues) { - org.apache.gobblin.service.Issue converted = new org.apache.gobblin.service.Issue(); - - converted.setCode(issues.getCode()) - .setSummary(ObjectUtils.firstNonNull(issues.getSummary(), "")) - .setDetails(ObjectUtils.firstNonNull(issues.getDetails(), "")) - .setSeverity(IssueSeverity.valueOf(issues.getSeverity().name())) - .setTime(issues.getTime().toInstant().toEpochMilli()); - - if (issues.getProperties() != null) { - converted.setProperties(new StringMap(issues.getProperties())); - } else { - converted.setProperties(new StringMap()); - } - - return converted; - } - - /** - * Return the flow start time given a {@link org.apache.gobblin.service.monitoring.FlowStatus}. Flow execution ID is - * assumed to be the flow start time. - */ - private static long getFlowStartTime(org.apache.gobblin.service.monitoring.FlowStatus flowStatus) { - return flowStatus.getFlowExecutionId(); - } - - /** - * Estimate the time left to complete the copy based on the following formula - - * timeLeft = (100/completionPercentage - 1) * timeElapsed - * @param currentTime as an epoch - * @param startTime as an epoch - * @param completionPercentage of the job - * @return time left in seconds - */ - public static long estimateCopyTimeLeft(Long currentTime, Long startTime, int completionPercentage) { - if (completionPercentage == 0) { - return 0; - } - - Instant current = Instant.ofEpochMilli(currentTime); - Instant start = Instant.ofEpochMilli(startTime); - long timeElapsed = Duration.between(start, current).getSeconds(); - return (long) (timeElapsed * (100.0 / (double) completionPercentage - 1)); - } -} 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..07874ce8df7 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 @@ -36,11 +36,11 @@ /** - * Resource for handling flow status requests + * Resource for handling flow status requests. Deprecated, use {@link FlowExecutionResource} */ +@Deprecated @RestLiCollection(name = "flowstatuses", namespace = "org.apache.gobblin.service", keyName = "id") public class FlowStatusResource extends ComplexKeyResourceTemplate { - public static final String MESSAGE_SEPARATOR = ", "; @Inject FlowStatusGenerator _flowStatusGenerator; @@ -55,13 +55,13 @@ public FlowStatusResource() {} @Override public FlowStatus get(ComplexResourceKey key) { // this returns null to raise a 404 error if flowStatus is null - return convertFlowStatus(FlowExecutionResourceLocalHandler.getFlowStatusFromGenerator(key, this._flowStatusGenerator)); + return convertFlowStatus(FlowExecutionResource.getFlowStatusFromGenerator(key, this._flowStatusGenerator)); } @Finder("latestFlowStatus") public List getLatestFlowStatus(@Context PagingContext context, @QueryParam("flowId") FlowId flowId, @Optional @QueryParam("count") Integer count, @Optional @QueryParam("tag") String tag) { - List flowStatuses = FlowExecutionResourceLocalHandler + List flowStatuses = FlowExecutionResource .getLatestFlowStatusesFromGenerator(flowId, count, tag, null, this._flowStatusGenerator); if (flowStatuses != null) { @@ -82,7 +82,7 @@ private FlowStatus convertFlowStatus(org.apache.gobblin.service.monitoring.FlowS if (monitoringFlowStatus == null) { return null; } - FlowExecution flowExecution = FlowExecutionResourceLocalHandler.convertFlowStatus(monitoringFlowStatus, false); + FlowExecution flowExecution = FlowExecutionResource.convertFlowStatus(monitoringFlowStatus, false); return new FlowStatus() .setId(flowExecution.getId()) .setExecutionStatistics(flowExecution.getExecutionStatistics()) diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigResourceLocalHandler.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/modules/restli/FlowConfigsV2ResourceHandler.java similarity index 67% rename from gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigResourceLocalHandler.java rename to gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/modules/restli/FlowConfigsV2ResourceHandler.java index fa0b2f46ca9..3b3869f4357 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigResourceLocalHandler.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/modules/restli/FlowConfigsV2ResourceHandler.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.gobblin.service; +package org.apache.gobblin.service.modules.restli; import java.net.URI; import java.net.URISyntaxException; @@ -29,22 +29,25 @@ import java.util.Properties; import java.util.stream.Collectors; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringEscapeUtils; import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.data.template.StringMap; +import com.linkedin.data.transform.DataProcessingException; import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; import com.linkedin.restli.common.HttpStatus; import com.linkedin.restli.common.PatchRequest; -import com.linkedin.restli.server.CreateResponse; +import com.linkedin.restli.server.CreateKVResponse; import com.linkedin.restli.server.RestLiServiceException; import com.linkedin.restli.server.UpdateResponse; +import com.linkedin.restli.server.util.PatchApplier; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; import javax.inject.Inject; +import javax.inject.Named; import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -60,14 +63,22 @@ import org.apache.gobblin.runtime.api.SpecNotFoundException; import org.apache.gobblin.runtime.spec_catalog.AddSpecResponse; import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; +import org.apache.gobblin.runtime.util.InjectionNames; +import org.apache.gobblin.service.FlowConfig; +import org.apache.gobblin.service.FlowConfigLoggedException; +import org.apache.gobblin.service.FlowId; +import org.apache.gobblin.service.FlowStatusId; +import org.apache.gobblin.service.RequesterService; +import org.apache.gobblin.service.Schedule; +import org.apache.gobblin.service.ServiceConfigKeys; import org.apache.gobblin.util.ConfigUtils; -/** - * A {@link FlowConfigsResourceHandler} that handles Restli locally. - */ @Slf4j -public class FlowConfigResourceLocalHandler implements FlowConfigsResourceHandler { +public class FlowConfigsV2ResourceHandler { + + @Getter + private String serviceName; public static final Schedule NEVER_RUN_CRON_SCHEDULE = new Schedule().setCronSchedule("0 0 0 ? 1 1 2050"); @Getter protected FlowCatalog flowCatalog; @@ -76,9 +87,11 @@ public class FlowConfigResourceLocalHandler implements FlowConfigsResourceHandle protected final ContextAwareMeter runImmediatelyFlow; @Inject - public FlowConfigResourceLocalHandler(FlowCatalog flowCatalog) { + public FlowConfigsV2ResourceHandler(@Named(InjectionNames.SERVICE_NAME) String serviceName, FlowCatalog flowCatalog) { + this.serviceName = serviceName; this.flowCatalog = flowCatalog; - MetricContext metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()), getClass()); + MetricContext + metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(ConfigFactory.empty()), getClass()); this.createFlow = metricContext.contextAwareMeter( MetricRegistry.name(ServiceMetricNames.GOBBLIN_SERVICE_PREFIX, ServiceMetricNames.CREATE_FLOW_METER)); this.deleteFlow = metricContext.contextAwareMeter( @@ -87,10 +100,8 @@ public FlowConfigResourceLocalHandler(FlowCatalog flowCatalog) { MetricRegistry.name(ServiceMetricNames.GOBBLIN_SERVICE_PREFIX, ServiceMetricNames.RUN_IMMEDIATELY_FLOW_METER)); } - /** - * Get flow config given a {@link FlowId} - */ - public FlowConfig getFlowConfig(FlowId flowId) throws FlowConfigLoggedException { + public FlowConfig getFlowConfig(FlowId flowId) + throws FlowConfigLoggedException { log.info("[GAAS-REST] Get called with flowGroup {} flowName {}", flowId.getFlowGroup(), flowId.getFlowName()); try { @@ -104,10 +115,6 @@ public FlowConfig getFlowConfig(FlowId flowId) throws FlowConfigLoggedException } } - /** - * Get flow config given a {@link FlowSpecSearchObject} - * @return all the {@link FlowConfig}s that satisfy the {@link FlowSpecSearchObject} - */ public Collection getFlowConfig(FlowSpecSearchObject flowSpecSearchObject) throws FlowConfigLoggedException { log.info("[GAAS-REST] Get called with flowSpecSearchObject {}", flowSpecSearchObject); return flowCatalog.getSpecs(flowSpecSearchObject).stream().map(FlowSpec.Utils::toFlowConfig).collect(Collectors.toList()); @@ -128,56 +135,53 @@ public Collection getAllFlowConfigs(int start, int count) { return flowCatalog.getSpecsPaginated(start, count).stream().map(FlowSpec.Utils::toFlowConfig).collect(Collectors.toList()); } - /** - * Add flowConfig locally and trigger all listeners iff @param triggerListener is set to true - */ - public CreateResponse createFlowConfig(FlowConfig flowConfig, boolean triggerListener) throws FlowConfigLoggedException { - log.info("[GAAS-REST] Create called with flowGroup " + flowConfig.getId().getFlowGroup() + " flowName " + flowConfig.getId().getFlowName()); - this.createFlow.mark(); - if (!flowConfig.hasSchedule() || StringUtils.isEmpty(flowConfig.getSchedule().getCronSchedule())) { - this.runImmediatelyFlow.mark(); - } + public UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) + throws FlowConfigLoggedException { + log.info("[GAAS-REST] Delete called with flowGroup {} flowName {}", flowId.getFlowGroup(), flowId.getFlowName()); + this.deleteFlow.mark(); + URI flowUri = null; - if (flowConfig.hasExplain()) { - //Return Error if FlowConfig has explain set. Explain request is only valid for v2 FlowConfig. - return new CreateResponse(new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, "FlowConfig with explain not supported.")); + try { + flowUri = FlowSpec.Utils.createFlowSpecUri(flowId); + this.flowCatalog.remove(flowUri, header, true); + return new UpdateResponse(HttpStatus.S_200_OK); + } catch (URISyntaxException e) { + throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, "bad URI " + flowUri, e); } + } - FlowSpec flowSpec = createFlowSpecForConfig(flowConfig); - // Existence of a flow spec in the flow catalog implies that the flow is currently running. - // If the new flow spec has a schedule we should allow submission of the new flow to accept the new schedule. - // However, if the new flow spec does not have a schedule, we should allow submission only if it is not running. - if (!flowConfig.hasSchedule() && this.flowCatalog.exists(flowSpec.getUri())) { - return new CreateResponse(new ComplexResourceKey<>(flowConfig.getId(), new EmptyRecord()), HttpStatus.S_409_CONFLICT); - } else { - try { - this.flowCatalog.put(flowSpec, triggerListener); - } catch (QuotaExceededException e) { - throw new RestLiServiceException(HttpStatus.S_503_SERVICE_UNAVAILABLE, e.getMessage()); - } catch (Throwable e) { - // TODO: Compilation errors should fall under throwable exceptions as well instead of checking for strings - log.warn(String.format("Failed to add flow configuration %s.%s to catalog due to", flowConfig.getId().getFlowGroup(), flowConfig.getId().getFlowName()), e); - throw new RestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, e.getMessage()); - } - return new CreateResponse(new ComplexResourceKey<>(flowConfig.getId(), new EmptyRecord()), HttpStatus.S_201_CREATED); + public UpdateResponse partialUpdateFlowConfig(FlowId flowId, + PatchRequest flowConfigPatch) throws FlowConfigLoggedException { + long modifiedWatermark = System.currentTimeMillis() / 1000; + FlowConfig flowConfig = getFlowConfig(flowId); + + try { + PatchApplier.applyPatch(flowConfig, flowConfigPatch); + } catch (DataProcessingException e) { + throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, "Failed to apply partial update", e); } - } - /** - * Add flowConfig locally and trigger all listeners - */ - public CreateResponse createFlowConfig(FlowConfig flowConfig) throws FlowConfigLoggedException { - return this.createFlowConfig(flowConfig, true); + return updateFlowConfig(flowId, flowConfig, modifiedWatermark); } - public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig, boolean triggerListener) { - // Set the max version to be the largest value so that we blindly update the flow spec in this case - return updateFlowConfig(flowId, flowConfig, triggerListener, Long.MAX_VALUE); + public UpdateResponse updateFlowConfig(FlowId flowId, + FlowConfig flowConfig) throws FlowConfigLoggedException { + // We have modifiedWatermark here to avoid update config happens at the same time on different hosts overwrite each other + // timestamp here will be treated as largest modifiedWatermark that we can update + long version = System.currentTimeMillis() / 1000; + return updateFlowConfig(flowId, flowConfig, version); } - /** - * Update flowConfig locally and trigger all listeners iff @param triggerListener is set to true - */ - public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig, boolean triggerListener, long modifiedWatermark) { + + public UpdateResponse updateFlowConfig(FlowId flowId, + FlowConfig flowConfig, long modifiedWatermark) throws FlowConfigLoggedException { + String flowName = flowId.getFlowName(); + String flowGroup = flowId.getFlowGroup(); + + if (!flowGroup.equals(flowConfig.getId().getFlowGroup()) || !flowName.equals(flowConfig.getId().getFlowName())) { + throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, + "flowName and flowGroup cannot be changed in update", null); + } + log.info("[GAAS-REST] Update called with flowGroup {} flowName {}", flowId.getFlowGroup(), flowId.getFlowName()); if (!flowId.getFlowGroup().equals(flowConfig.getId().getFlowGroup()) || !flowId.getFlowName().equals(flowConfig.getId().getFlowName())) { @@ -201,7 +205,7 @@ public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig, boo FlowSpec flowSpec = createFlowSpecForConfig(flowConfig); Map responseMap; try { - responseMap = this.flowCatalog.update(flowSpec, triggerListener, modifiedWatermark); + responseMap = this.flowCatalog.update(flowSpec, true, modifiedWatermark); } catch (QuotaExceededException e) { throw new RestLiServiceException(HttpStatus.S_503_SERVICE_UNAVAILABLE, e.getMessage()); } catch (Throwable e) { @@ -217,45 +221,70 @@ public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig, boo } } - private boolean isUnscheduleRequest(FlowConfig flowConfig) { - return Boolean.parseBoolean(flowConfig.getProperties().getOrDefault(ConfigurationKeys.FLOW_UNSCHEDULE_KEY, "false")); - } + public CreateKVResponse, FlowConfig> createFlowConfig(FlowConfig flowConfig) throws FlowConfigLoggedException { + if (flowConfig.getProperties().containsKey(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)) { + throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, + String.format("%s cannot be set by the user", ConfigurationKeys.FLOW_EXECUTION_ID_KEY), null); + } - /** - * Update flowConfig locally and trigger all listeners - */ - public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig) throws FlowConfigLoggedException { - return updateFlowConfig(flowId, flowConfig, true); - } + String createLog = "[GAAS-REST] Create called with flowGroup " + flowConfig.getId().getFlowGroup() + " flowName " + flowConfig.getId().getFlowName(); + this.createFlow.mark(); - @Override - public UpdateResponse partialUpdateFlowConfig(FlowId flowId, PatchRequest flowConfigPatch) throws FlowConfigLoggedException { - throw new UnsupportedOperationException("Partial update only supported by GobblinServiceFlowConfigResourceHandler"); - } + if (flowConfig.hasExplain()) { + createLog += " explain " + flowConfig.isExplain(); + } - /** - * Delete flowConfig locally and trigger all listeners iff @param triggerListener is set to true - */ - public UpdateResponse deleteFlowConfig(FlowId flowId, Properties header, boolean triggerListener) throws FlowConfigLoggedException { + log.info(createLog); + FlowSpec flowSpec = createFlowSpecForConfig(flowConfig); + FlowStatusId flowStatusId = + new FlowStatusId().setFlowName(flowSpec.getConfigAsProperties().getProperty(ConfigurationKeys.FLOW_NAME_KEY)) + .setFlowGroup(flowSpec.getConfigAsProperties().getProperty(ConfigurationKeys.FLOW_GROUP_KEY)); + if (flowSpec.getConfigAsProperties().containsKey(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)) { + flowStatusId.setFlowExecutionId(Long.valueOf(flowSpec.getConfigAsProperties().getProperty(ConfigurationKeys.FLOW_EXECUTION_ID_KEY))); + } else { + flowStatusId.setFlowExecutionId(-1L); + } - log.info("[GAAS-REST] Delete called with flowGroup {} flowName {}", flowId.getFlowGroup(), flowId.getFlowName()); - this.deleteFlow.mark(); - URI flowUri = null; + // Return conflict and take no action if flowSpec has already been created + if (this.flowCatalog.exists(flowSpec.getUri())) { + log.warn("FlowSpec with URI {} already exists, no action will be taken", flowSpec.getUri()); + return new CreateKVResponse<>(new RestLiServiceException(HttpStatus.S_409_CONFLICT, + "FlowSpec with URI " + flowSpec.getUri() + " already exists, no action will be taken")); + } + Map responseMap; try { - flowUri = FlowSpec.Utils.createFlowSpecUri(flowId); - this.flowCatalog.remove(flowUri, header, triggerListener); - return new UpdateResponse(HttpStatus.S_200_OK); - } catch (URISyntaxException e) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, "bad URI " + flowUri, e); + responseMap = this.flowCatalog.put(flowSpec, true); + } catch (QuotaExceededException e) { + throw new RestLiServiceException(HttpStatus.S_503_SERVICE_UNAVAILABLE, e.getMessage()); + } catch (Throwable e) { + // TODO: Compilation errors should fall under throwable exceptions as well instead of checking for strings + log.warn(String.format("Failed to add flow configuration %s.%s to catalog due to", flowConfig.getId().getFlowGroup(), flowConfig.getId().getFlowName()), e); + throw new RestLiServiceException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, e.getMessage()); } + + HttpStatus httpStatus; + + if (flowConfig.hasExplain() && flowConfig.isExplain()) { + //This is an Explain request. So no resource is actually created. + //Enrich original FlowConfig entity by adding the compiledFlow to the properties map. + StringMap props = flowConfig.getProperties(); + AddSpecResponse addSpecResponse = responseMap.getOrDefault(ServiceConfigKeys.COMPILATION_RESPONSE, null); + props.put("gobblin.flow.compiled", + addSpecResponse != null && addSpecResponse.getValue() != null ? StringEscapeUtils.escapeJson(addSpecResponse.getValue()) : ""); + flowConfig.setProperties(props); + httpStatus = HttpStatus.S_200_OK; + } else if (Boolean.parseBoolean(responseMap.getOrDefault(ServiceConfigKeys.COMPILATION_SUCCESSFUL, new AddSpecResponse<>("false")).getValue().toString())) { + httpStatus = HttpStatus.S_201_CREATED; + } else { + throw new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, getErrorMessage(flowSpec)); + } + + return new CreateKVResponse<>(new ComplexResourceKey<>(flowConfig.getId(), flowStatusId), flowConfig, httpStatus); } - /** - * Delete flowConfig locally and trigger all listeners - */ - public UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) throws FlowConfigLoggedException { - return deleteFlowConfig(flowId, header, true); + private boolean isUnscheduleRequest(FlowConfig flowConfig) { + return Boolean.parseBoolean(flowConfig.getProperties().getOrDefault(ConfigurationKeys.FLOW_UNSCHEDULE_KEY, "false")); } /** diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigResourceLocalHandlerTest.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigResourceLocalHandlerTest.java index 6a28686cddf..bfb24a2df6e 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigResourceLocalHandlerTest.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigResourceLocalHandlerTest.java @@ -29,6 +29,7 @@ import org.apache.gobblin.configuration.ConfigurationKeys; import org.apache.gobblin.runtime.api.FlowSpec; +import org.apache.gobblin.service.modules.restli.FlowConfigsV2ResourceHandler; public class FlowConfigResourceLocalHandlerTest { @@ -47,11 +48,11 @@ public void testCreateFlowSpecForConfig() throws URISyntaxException { setRunImmediately(true)) .setProperties(new StringMap(flowProperties)); - FlowSpec flowSpec = FlowConfigResourceLocalHandler.createFlowSpecForConfig(flowConfig); + FlowSpec flowSpec = FlowConfigsV2ResourceHandler.createFlowSpecForConfig(flowConfig); Assert.assertEquals(flowSpec.getConfig().getString(ConfigurationKeys.FLOW_GROUP_KEY), TEST_GROUP_NAME); Assert.assertEquals(flowSpec.getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY), TEST_FLOW_NAME); Assert.assertEquals(flowSpec.getConfig().getString(ConfigurationKeys.JOB_SCHEDULE_KEY), TEST_SCHEDULE); - Assert.assertEquals(flowSpec.getConfig().getBoolean(ConfigurationKeys.FLOW_RUN_IMMEDIATELY), true); + Assert.assertTrue(flowSpec.getConfig().getBoolean(ConfigurationKeys.FLOW_RUN_IMMEDIATELY)); Assert.assertEquals(flowSpec.getConfig().getString("param1"), "a:b:c*.d"); Assert.assertEquals(flowSpec.getTemplateURIs().get().size(), 1); Assert.assertTrue(flowSpec.getTemplateURIs().get().contains(new URI(TEST_TEMPLATE_URI))); @@ -64,12 +65,12 @@ public void testCreateFlowSpecForConfig() throws URISyntaxException { .setTemplateUris(TEST_TEMPLATE_URI).setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE). setRunImmediately(true)) .setProperties(new StringMap(flowProperties)); - flowSpec = FlowConfigResourceLocalHandler.createFlowSpecForConfig(flowConfig); + flowSpec = FlowConfigsV2ResourceHandler.createFlowSpecForConfig(flowConfig); Assert.assertEquals(flowSpec.getConfig().getString(ConfigurationKeys.FLOW_GROUP_KEY), TEST_GROUP_NAME); Assert.assertEquals(flowSpec.getConfig().getString(ConfigurationKeys.FLOW_NAME_KEY), TEST_FLOW_NAME); Assert.assertEquals(flowSpec.getConfig().getString(ConfigurationKeys.JOB_SCHEDULE_KEY), TEST_SCHEDULE); - Assert.assertEquals(flowSpec.getConfig().getBoolean(ConfigurationKeys.FLOW_RUN_IMMEDIATELY), true); + Assert.assertTrue(flowSpec.getConfig().getBoolean(ConfigurationKeys.FLOW_RUN_IMMEDIATELY)); Assert.assertEquals(flowSpec.getConfig().getString("param1"),"value1"); Assert.assertEquals(flowSpec.getConfig().getString("param2"),"value1-123"); Assert.assertEquals(flowSpec.getConfig().getString("param3"), "a:b:c*.d"); diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandlerTest.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandlerTest.java index 40f865b4b62..8215135bf4d 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandlerTest.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandlerTest.java @@ -17,7 +17,6 @@ package org.apache.gobblin.service; - import org.testng.Assert; import org.testng.annotations.Test; @@ -25,42 +24,42 @@ public class FlowExecutionResourceLocalHandlerTest { @Test - public void testEstimateCopyTimeLeftSanityCheck() throws Exception { + public void testEstimateCopyTimeLeftSanityCheck() { long currentTime = 10000; long startTime = 0; int copyPercentage = 50; - long timeLeft = FlowExecutionResourceLocalHandler.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); + long timeLeft = FlowExecutionResource.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); Assert.assertEquals(timeLeft, 10); } @Test - public void testEstimateCopyTimeLeftSimple() throws Exception { + public void testEstimateCopyTimeLeftSimple() { long currentTime = 50000; long startTime = 20000; int copyPercentage = 10; - long timeLeft = FlowExecutionResourceLocalHandler.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); + long timeLeft = FlowExecutionResource.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); Assert.assertEquals(timeLeft, 270); } @Test - public void testEstimateCopyTimeLeftMedium() throws Exception { + public void testEstimateCopyTimeLeftMedium() { long currentTime = 5000000; long startTime = 1500000; int copyPercentage = 25; - long timeLeft = FlowExecutionResourceLocalHandler.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); + long timeLeft = FlowExecutionResource.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); Assert.assertEquals(timeLeft, 10500); } @Test - public void testEstimateCopyTimeRealData() throws Exception { + public void testEstimateCopyTimeRealData() { long currentTime = 1626717751099L; long startTime = 1626716510626L; int copyPercentage = 24; - long timeLeft = FlowExecutionResourceLocalHandler.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); + long timeLeft = FlowExecutionResource.estimateCopyTimeLeft(currentTime, startTime, copyPercentage); Assert.assertEquals(timeLeft, 3926L); } } \ No newline at end of file diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/InjectionNames.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/InjectionNames.java index 9e716be1652..f75604cbf9c 100644 --- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/InjectionNames.java +++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/InjectionNames.java @@ -23,6 +23,4 @@ * */ public final class InjectionNames { public static final String SERVICE_NAME = "serviceName"; - public static final String FORCE_LEADER = "forceLeader"; - public static final String FLOW_CATALOG_LOCAL_COMMIT = "flowCatalogLocalCommit"; } 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 deleted file mode 100644 index 6e9cff7828d..00000000000 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/ControllerUserDefinedMessageHandlerFactory.java +++ /dev/null @@ -1,206 +0,0 @@ -/* - * 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.core; - -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.Collections; -import java.util.List; -import java.util.Properties; - -import org.apache.helix.NotificationContext; -import org.apache.helix.messaging.handling.HelixTaskResult; -import org.apache.helix.messaging.handling.MessageHandler; -import org.apache.helix.messaging.handling.MultiTypeMessageHandlerFactory; -import org.apache.helix.model.Message; - -import lombok.AllArgsConstructor; -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.runtime.api.FlowSpec; -import org.apache.gobblin.runtime.api.Spec; -import org.apache.gobblin.service.FlowConfig; -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.modules.restli.FlowConfigUtils; -import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; - -/** - * A custom {@link MultiTypeMessageHandlerFactory} for {@link org.apache.gobblin.service.modules.core.ControllerUserDefinedMessageHandlerFactory}s that - * handle messages of type {@link org.apache.helix.model.Message.MessageType#USER_DEFINE_MSG}. - */ -@AllArgsConstructor -class ControllerUserDefinedMessageHandlerFactory implements MultiTypeMessageHandlerFactory { - private final boolean flowCatalogLocalCommit; - private final GobblinServiceJobScheduler jobScheduler; - private final FlowConfigsResourceHandler resourceHandler; - private final String serviceName; - - @Override - public MessageHandler createHandler(Message message, NotificationContext context) { - return new ControllerUserDefinedMessageHandler(message, context, serviceName, flowCatalogLocalCommit, jobScheduler, resourceHandler); - } - - @Override - public String getMessageType() { - return Message.MessageType.USER_DEFINE_MSG.toString(); - } - - public List getMessageTypes() { - return Collections.singletonList(getMessageType()); - } - - @Override - public void reset() { - - } - - /** - * A custom {@link MessageHandler} for handling user-defined messages to the controller. - */ - @Slf4j - private static class ControllerUserDefinedMessageHandler extends MessageHandler { - private final boolean flowCatalogLocalCommit; - private final GobblinServiceJobScheduler jobScheduler; - private final FlowConfigsResourceHandler resourceHandler; - private final String serviceName; - - public ControllerUserDefinedMessageHandler(Message message, NotificationContext context, String serviceName, - boolean flowCatalogLocalCommit, GobblinServiceJobScheduler scheduler, - FlowConfigsResourceHandler resourceHandler) { - super(message, context); - this.serviceName = serviceName; - this.flowCatalogLocalCommit = flowCatalogLocalCommit; - this.jobScheduler = scheduler; - this.resourceHandler = resourceHandler; - } - - /** - * Method to handle add flow config message forwarded by Helix (Standby) node. - * In load balance mode, the FlowCatalog I/O was handled on standby when receiving Restli, so only need to handle - * {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onAddSpec(Spec)} part. - * Otherwise, we have to handle both FlowCatalog I/O and {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onAddSpec(Spec)}. - * - * Please refer to {@link FlowConfigResourceLocalHandler#createFlowConfig(FlowConfig)}. It will handle both FlowCatalog I/O and - * {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onAddSpec(Spec)} in non-balance mode. - */ - private void handleAdd(String msg) - throws IOException { - FlowConfig config = FlowConfigUtils.deserializeFlowConfig(msg); - if (this.flowCatalogLocalCommit) { - // in balance mode, flow spec is already added in flow catalog on standby node. - FlowSpec flowSpec = FlowConfigResourceLocalHandler.createFlowSpecForConfig(config); - log.info("Only handle add {} scheduling because flow catalog is committed locally on standby.", flowSpec); - jobScheduler.onAddSpec(flowSpec); - } else { - resourceHandler.createFlowConfig(config); - } - } - - /** - * Method to handle add flow config message forwarded by Helix (Standby) node. - * In load balance mode, the FlowCatalog I/O was handled on standby when receiving Restli, so only need to handle - * {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onUpdateSpec(Spec)} part. - * Otherwise, we have to handle both FlowCatalog I/O and {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onUpdateSpec(Spec)}. - * - * Please refer to {@link FlowConfigResourceLocalHandler#updateFlowConfig(FlowId, FlowConfig)}. It will handle both FlowCatalog I/O and - * {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onUpdateSpec(Spec)} in non-balance mode. - */ - private void handleUpdate(String msg) - throws IOException { - FlowConfig config = FlowConfigUtils.deserializeFlowConfig(msg); - if (flowCatalogLocalCommit) { - // in balance mode, flow spec is already updated in flow catalog on standby node. - FlowSpec flowSpec = FlowConfigResourceLocalHandler.createFlowSpecForConfig(config); - log.info("Only handle update {} scheduling because flow catalog is committed locally on standby.", flowSpec); - jobScheduler.onUpdateSpec(flowSpec); - } else { - resourceHandler.updateFlowConfig(config.getId(), config); - } - } - - /** - * Method to handle add flow config message forwarded by Helix (Standby) node. - * In load balance mode, the FlowCatalog I/O was handled on standby when receiving Restli, so only need to handle - * {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onDeleteSpec(URI, String, Properties)} part. - * Otherwise, we have to handle both FlowCatalog I/O and {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onDeleteSpec(URI, String, Properties)}. - * - * Please refer to {@link FlowConfigResourceLocalHandler#deleteFlowConfig(FlowId, Properties)}. It will handle both FlowCatalog I/O and - * {@link org.apache.gobblin.runtime.api.SpecCatalogListener#onDeleteSpec(URI, String, Properties)} in non-balance mode. - */ - private void handleDelete(String msg) - throws IOException { - try { - FlowId id = FlowConfigUtils.deserializeFlowId(msg); - if (flowCatalogLocalCommit) { - // in balance mode, flow spec is already deleted in flow catalog on standby node. - URI flowUri = FlowSpec.Utils.createFlowSpecUri(id); - log.info("Only handle update {} scheduling because flow catalog is committed locally on standby.", flowUri); - jobScheduler.onDeleteSpec(flowUri, FlowSpec.Builder.DEFAULT_VERSION); - } else { - resourceHandler.deleteFlowConfig(id, new Properties()); - } - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - - @Override - public HelixTaskResult handleMessage() - throws InterruptedException { - if (jobScheduler.isActive()) { - // we want to make sure current node is in active state - 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)) { - handleAdd(msg); - } else if (_message.getMsgSubType().equals(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE)) { - handleDelete(msg); - } else if (_message.getMsgSubType().equals(ServiceConfigKeys.HELIX_FLOWSPEC_UPDATE)) { - handleUpdate(msg); - } - } catch (IOException e) { - log.error("Cannot process Helix message.", e); - HelixTaskResult helixTaskResult = new HelixTaskResult(); - helixTaskResult.setSuccess(false); - return helixTaskResult; - } - } else { - String msg = _message.getAttribute(Message.Attributes.INNER_MESSAGE); - log.error("ControllerUserDefinedMessage received but ignored due to not in active mode: {}, type {}", msg, - _message.getMsgSubType()); - } - HelixTaskResult helixTaskResult = new HelixTaskResult(); - helixTaskResult.setSuccess(true); - - return helixTaskResult; - } - - @Override - public void onError(Exception e, ErrorCode code, ErrorType type) { - log.error( - String.format("Failed to handle message with exception %s, error code %s, error type %s", e, code, type)); - } - } -} - - 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 a7a64034d13..8361b1efc53 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 @@ -43,13 +43,6 @@ public class GobblinServiceConfiguration { private final boolean isTopologySpecFactoryEnabled; private final boolean isGitConfigMonitorEnabled; private final boolean isJobStatusMonitorEnabled; - - @Getter - private final boolean isHelixManagerEnabled; - - @Getter - private final boolean flowCatalogLocalCommit; - private final boolean onlyAnnounceLeader; private final Config innerConfig; @Nullable @@ -66,17 +59,12 @@ public GobblinServiceConfiguration(String serviceName, String serviceId, Config ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_FLOW_CATALOG_ENABLED_KEY, true); if (isFlowCatalogEnabled) { - flowCatalogLocalCommit = - ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_FLOW_CATALOG_LOCAL_COMMIT, - ServiceConfigKeys.DEFAULT_GOBBLIN_SERVICE_FLOW_CATALOG_LOCAL_COMMIT); isGitConfigMonitorEnabled = ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_GIT_CONFIG_MONITOR_ENABLED_KEY, false); } else { - flowCatalogLocalCommit = false; isGitConfigMonitorEnabled = false; } - this.isHelixManagerEnabled = config.hasPath(ServiceConfigKeys.ZK_CONNECTION_STRING_KEY); this.isJobStatusMonitorEnabled = ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_JOB_STATUS_MONITOR_ENABLED_KEY, true); this.isSchedulerEnabled = @@ -85,6 +73,5 @@ public GobblinServiceConfiguration(String serviceName, String serviceId, Config ConfigUtils.getBoolean(config, ServiceConfigKeys.GOBBLIN_SERVICE_RESTLI_SERVER_ENABLED_KEY, true); 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); } } 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 cd2d9301d29..b428962c1b5 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 @@ -19,12 +19,10 @@ import java.util.Objects; -import org.apache.helix.HelixManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.collect.Lists; -import com.google.common.eventbus.EventBus; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Injector; @@ -47,15 +45,9 @@ import org.apache.gobblin.runtime.troubleshooter.MultiContextIssueRepository; import org.apache.gobblin.runtime.util.InjectionNames; import org.apache.gobblin.scheduler.SchedulerService; -import org.apache.gobblin.service.FlowConfigResourceLocalHandler; -import org.apache.gobblin.service.FlowConfigV2ResourceLocalHandler; -import org.apache.gobblin.service.FlowConfigsResource; -import org.apache.gobblin.service.FlowConfigsResourceHandler; import org.apache.gobblin.service.FlowConfigsV2Resource; -import org.apache.gobblin.service.FlowConfigsV2ResourceHandler; import org.apache.gobblin.service.FlowExecutionResource; -import org.apache.gobblin.service.FlowExecutionResourceHandler; -import org.apache.gobblin.service.FlowExecutionResourceLocalHandler; +import org.apache.gobblin.service.FlowExecutionResourceHandlerInterface; import org.apache.gobblin.service.FlowStatusResource; import org.apache.gobblin.service.GroupOwnershipService; import org.apache.gobblin.service.NoopRequesterService; @@ -82,14 +74,12 @@ import org.apache.gobblin.service.modules.orchestration.UserQuotaManager; import org.apache.gobblin.service.modules.orchestration.proc.DagProcUtils; import org.apache.gobblin.service.modules.orchestration.task.DagProcessingEngineMetrics; -import org.apache.gobblin.service.modules.restli.GobblinServiceFlowConfigResourceHandler; -import org.apache.gobblin.service.modules.restli.GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby; -import org.apache.gobblin.service.modules.restli.GobblinServiceFlowExecutionResourceHandlerWithWarmStandby; +import org.apache.gobblin.service.modules.restli.FlowConfigsV2ResourceHandler; +import org.apache.gobblin.service.modules.restli.FlowExecutionResourceHandler; import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; import org.apache.gobblin.service.modules.topology.TopologySpecFactory; import org.apache.gobblin.service.modules.troubleshooter.MySqlMultiContextIssueRepository; import org.apache.gobblin.service.modules.utils.FlowCompilationValidationHelper; -import org.apache.gobblin.service.modules.utils.HelixUtils; import org.apache.gobblin.service.modules.utils.SharedFlowMetricsSingleton; import org.apache.gobblin.service.monitoring.DagManagementDagActionStoreChangeMonitor; import org.apache.gobblin.service.monitoring.DagManagementDagActionStoreChangeMonitorFactory; @@ -144,23 +134,14 @@ public void configure(Binder binder) { binder.bind(GobblinInstanceEnvironment.class).toInstance(gobblinInstanceEnvironment); - binder.bind(EventBus.class) - .annotatedWith(Names.named(GobblinServiceManager.SERVICE_EVENT_BUS_NAME)) - .toInstance(new EventBus(GobblinServiceManager.class.getSimpleName())); - binder.bindConstant().annotatedWith(Names.named(InjectionNames.SERVICE_NAME)).to(serviceConfig.getServiceName()); - binder.bindConstant() - .annotatedWith(Names.named(InjectionNames.FORCE_LEADER)) - .to(ConfigUtils.getBoolean(serviceConfig.getInnerConfig(), ServiceConfigKeys.FORCE_LEADER, - ServiceConfigKeys.DEFAULT_FORCE_LEADER)); - - binder.bindConstant() - .annotatedWith(Names.named(InjectionNames.FLOW_CATALOG_LOCAL_COMMIT)) - .to(serviceConfig.isFlowCatalogLocalCommit()); - binder.bind(FlowConfigsResourceHandler.class).to(GobblinServiceFlowConfigResourceHandler.class); - binder.bind(FlowConfigsV2ResourceHandler.class).to(GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby.class); - binder.bind(FlowExecutionResourceHandler.class).to(GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.class); + binder.bind(FlowConfigsV2Resource.class); + binder.bind(FlowStatusResource.class); + binder.bind(FlowExecutionResource.class); + binder.bind(FlowConfigsV2ResourceHandler.class); + binder.bind(FlowExecutionResourceHandler.class); + binder.bind(FlowExecutionResourceHandlerInterface.class).to(FlowExecutionResourceHandler.class); /* Note that two instances of the same class can only be differentiated with an `annotatedWith` marker provided at binding time (optionally bound classes cannot have names associated with them), so both arbiters need to be @@ -184,21 +165,10 @@ binding time (optionally bound classes cannot have names associated with them), binder.bind(DagProcessingEngine.class); binder.bind(DagProcessingEngineMetrics.class); binder.bind(FlowLaunchHandler.class); + binder.bind(MultiActiveLeaseArbiter.class).toProvider(DagActionProcessingMultiActiveLeaseArbiterFactory.class); binder.bind(SpecStoreChangeMonitor.class).toProvider(SpecStoreChangeMonitorFactory.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); - - binder.bindConstant().annotatedWith(Names.named(FlowConfigsResource.INJECT_READY_TO_USE)).to(Boolean.TRUE); - binder.bindConstant().annotatedWith(Names.named(FlowConfigsV2Resource.INJECT_READY_TO_USE)).to(Boolean.TRUE); - binder.bind(RequesterService.class) - .to(NoopRequesterService.class); + binder.bind(RequesterService.class).to(NoopRequesterService.class); binder.bind(SharedFlowMetricsSingleton.class); binder.bind(FlowCompilationValidationHelper.class); binder.bind(TopologyCatalog.class); @@ -209,15 +179,6 @@ binding time (optionally bound classes cannot have names associated with them), ServiceConfigKeys.TOPOLOGYSPEC_FACTORY_KEY, ServiceConfigKeys.DEFAULT_TOPOLOGY_SPEC_FACTORY)); } - OptionalBinder.newOptionalBinder(binder, HelixManager.class); - if (serviceConfig.isHelixManagerEnabled()) { - binder.bind(HelixManager.class) - .toInstance(buildHelixManager(serviceConfig.getInnerConfig(), - serviceConfig.getInnerConfig().getString(ServiceConfigKeys.ZK_CONNECTION_STRING_KEY))); - } else { - LOGGER.info("No ZooKeeper connection string. Running in single instance mode."); - } - OptionalBinder.newOptionalBinder(binder, FlowCatalog.class); if (serviceConfig.isFlowCatalogEnabled()) { binder.bind(FlowCatalog.class); @@ -276,17 +237,6 @@ binding time (optionally bound classes cannot have names associated with them), LOGGER.info("Bindings configured"); } - protected HelixManager buildHelixManager(Config config, String zkConnectionString) { - String helixClusterName = config.getString(ServiceConfigKeys.HELIX_CLUSTER_NAME_KEY); - String helixInstanceName = HelixUtils.buildHelixInstanceName(config, GobblinServiceManager.class.getSimpleName()); - - LOGGER.info( - "Creating Helix cluster if not already present [overwrite = false]: " + zkConnectionString); - HelixUtils.createGobblinHelixCluster(zkConnectionString, helixClusterName, false); - - return HelixUtils.buildHelixManager(helixInstanceName, helixClusterName, zkConnectionString); - } - protected static Class getClassByNameOrAlias(Class baseClass, Config config, String classPropertyName, String defaultClass) { String className = ConfigUtils.getString(config, classPropertyName, defaultClass); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java index 0c104276bdf..d9141a01344 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java @@ -64,11 +64,7 @@ import org.apache.gobblin.scheduler.SchedulerService; import org.apache.gobblin.service.FlowConfig; import org.apache.gobblin.service.FlowConfigV2Client; -import org.apache.gobblin.service.FlowConfigsResource; -import org.apache.gobblin.service.FlowConfigsResourceHandler; import org.apache.gobblin.service.FlowConfigsV2Resource; -import org.apache.gobblin.service.FlowConfigsV2ResourceHandler; -import org.apache.gobblin.service.FlowExecutionResourceHandler; import org.apache.gobblin.service.FlowId; import org.apache.gobblin.service.GroupOwnershipService; import org.apache.gobblin.service.Schedule; @@ -77,6 +73,8 @@ import org.apache.gobblin.service.modules.orchestration.DagProcessingEngine; import org.apache.gobblin.service.modules.orchestration.Orchestrator; import org.apache.gobblin.service.modules.orchestration.UserQuotaManager; +import org.apache.gobblin.service.modules.restli.FlowConfigsV2ResourceHandler; +import org.apache.gobblin.service.modules.restli.FlowExecutionResourceHandler; import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; import org.apache.gobblin.service.modules.topology.TopologySpecFactory; import org.apache.gobblin.service.monitoring.DagManagementDagActionStoreChangeMonitor; @@ -94,9 +92,6 @@ public class GobblinServiceManager implements ApplicationLauncher { // These two options are required to launch GobblinServiceManager. public static final String SERVICE_NAME_OPTION_NAME = "service_name"; public static final String SERVICE_ID_OPTION_NAME = "service_id"; - - public static final String SERVICE_EVENT_BUS_NAME = "GobblinServiceManagerEventBus"; - private static final Logger LOGGER = LoggerFactory.getLogger(GobblinServiceManager.class); protected final ServiceBasedAppLauncher serviceLauncher; private volatile boolean stopInProgress = false; @@ -119,11 +114,7 @@ public class GobblinServiceManager implements ApplicationLauncher { @Inject @Getter - protected FlowConfigsResourceHandler resourceHandler; - - @Inject - @Getter - protected FlowConfigsV2ResourceHandler v2ResourceHandler; + protected FlowConfigsV2ResourceHandler resourceHandler; @Inject @Getter @@ -289,7 +280,7 @@ private void registerServicesInLauncher(){ private void configureServices(){ if (configuration.isRestLIServerEnabled()) { this.restliServer = EmbeddedRestliServer.builder() - .resources(Lists.newArrayList(FlowConfigsResource.class, FlowConfigsV2Resource.class)) + .resources(Lists.newArrayList(FlowConfigsV2Resource.class, FlowConfigsV2Resource.class)) .injector(injector) .build(); @@ -318,10 +309,8 @@ public void start() throws ApplicationException { this.gitConfigMonitor.setActive(true); } - // Announce to d2 after services are initialized regardless of leadership if configuration is not enabled - if (!this.configuration.isOnlyAnnounceLeader()) { - this.d2Announcer.markUpServer(); - } + // Announce to d2 after services are initialized + this.d2Announcer.markUpServer(); // Populate TopologyCatalog with all Topologies generated by TopologySpecFactory // This has to be done after the topologyCatalog service is launched @@ -359,9 +348,7 @@ public void stop() throws ApplicationException { this.stopInProgress = true; try { // Stop announcing GaaS instances to d2 when services are stopped - if (!configuration.isOnlyAnnounceLeader()) { - this.d2Announcer.markDownServer(); - } + this.d2Announcer.markDownServer(); this.serviceLauncher.stop(); } catch (ApplicationException ae) { LOGGER.error("Error while stopping Gobblin Service Manager", ae); 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 index d50887b365d..d936f0129b4 100644 --- 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 @@ -99,7 +99,6 @@ public interface DagManagementStateStore { * {@link DagManagementStateStore#addDag}. This call is just an additional identifier which may be used * for DagNode level operations. In the future, it may be merged with checkpointDag. * @param dagNode dag node to be added - * @param dagId dag id of the dag this dag node belongs to */ void updateDagNode(Dag.DagNode dagNode) throws IOException; diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MultiActiveLeaseArbiterFactory.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MultiActiveLeaseArbiterFactory.java index 64c353371f8..6eac369004a 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MultiActiveLeaseArbiterFactory.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MultiActiveLeaseArbiterFactory.java @@ -42,7 +42,7 @@ public MultiActiveLeaseArbiterFactory(Config config, String configPrefix) { throw new RuntimeException(String.format("Unable to initialize multiActiveLeaseArbiter due to missing " + "configurations that should be prefixed by %s.", configPrefix)); } - this.leaseArbiterConfig = config.getConfig(configPrefix); + this.leaseArbiterConfig = config.getConfig(configPrefix).withFallback(config); log.info("Lease arbiter will be initialized with config {}", leaseArbiterConfig); } diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java new file mode 100644 index 00000000000..85bb3851e84 --- /dev/null +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java @@ -0,0 +1,143 @@ +/* + * 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.restli; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.commons.lang.StringUtils; + +import com.google.inject.Inject; +import com.linkedin.restli.common.ComplexResourceKey; +import com.linkedin.restli.common.EmptyRecord; +import com.linkedin.restli.common.HttpStatus; +import com.linkedin.restli.server.PagingContext; +import com.linkedin.restli.server.RestLiServiceException; +import com.linkedin.restli.server.UpdateResponse; + +import lombok.extern.slf4j.Slf4j; + +import org.apache.gobblin.service.FlowExecution; +import org.apache.gobblin.service.FlowExecutionResource; +import org.apache.gobblin.service.FlowExecutionResourceHandlerInterface; +import org.apache.gobblin.service.FlowId; +import org.apache.gobblin.service.FlowStatusId; +import org.apache.gobblin.service.modules.orchestration.DagActionStore; +import org.apache.gobblin.service.modules.orchestration.DagManagementStateStore; +import org.apache.gobblin.service.monitoring.FlowStatus; +import org.apache.gobblin.service.monitoring.FlowStatusGenerator; + + +@Slf4j +public class FlowExecutionResourceHandler implements FlowExecutionResourceHandlerInterface { + private final DagManagementStateStore dagManagementStateStore; + private final FlowStatusGenerator flowStatusGenerator; + + @Inject + public FlowExecutionResourceHandler(FlowStatusGenerator flowStatusGenerator, + DagManagementStateStore dagManagementStateStore) { + this.dagManagementStateStore = dagManagementStateStore; + this.flowStatusGenerator = flowStatusGenerator; + } + + public FlowExecution get(ComplexResourceKey key) { + FlowExecution flowExecution = FlowExecutionResource.convertFlowStatus(FlowExecutionResource.getFlowStatusFromGenerator(key, this.flowStatusGenerator), true); + if (flowExecution == null) { + throw new RestLiServiceException(HttpStatus.S_404_NOT_FOUND, "No flow execution found for flowStatusId " + key.getKey() + + ". The flowStatusId may be incorrect, or the flow execution may have been cleaned up."); + } + return flowExecution; } + + public List getLatestFlowExecution(PagingContext context, FlowId flowId, Integer count, String tag, + String executionStatus, Boolean includeIssues) { + List flowStatuses = FlowExecutionResource.getLatestFlowStatusesFromGenerator(flowId, count, tag, executionStatus, this.flowStatusGenerator); + + if (flowStatuses != null) { + return flowStatuses.stream() + .map((FlowStatus monitoringFlowStatus) -> FlowExecutionResource.convertFlowStatus(monitoringFlowStatus, includeIssues)) + .collect(Collectors.toList()); + } + + throw new RestLiServiceException(HttpStatus.S_404_NOT_FOUND, "No flow execution found for flowId " + flowId + + ". The flowId may be incorrect, the flow execution may have been cleaned up, or not matching tag (" + tag + + ") and/or execution status (" + executionStatus + ")."); + } + + public List getLatestFlowGroupExecutions(PagingContext context, String flowGroup, Integer countPerFlow, + String tag, Boolean includeIssues) { + List flowStatuses = + getLatestFlowGroupStatusesFromGenerator(flowGroup, countPerFlow, tag, this.flowStatusGenerator); + + if (flowStatuses != null) { + // todo: flow end time will be incorrect when dag manager is not used + // and FLOW_SUCCEEDED/FLOW_CANCELLED/FlowFailed events are not sent + return flowStatuses.stream() + .map((FlowStatus monitoringFlowStatus) -> FlowExecutionResource.convertFlowStatus(monitoringFlowStatus, includeIssues)) + .collect(Collectors.toList()); + } + + throw new RestLiServiceException(HttpStatus.S_404_NOT_FOUND, "No flow executions found for flowGroup " + flowGroup + + ". The group name may be incorrect, the flow execution may have been cleaned up, or not matching tag (" + tag + + ")."); + } + + public void resume(ComplexResourceKey key) { + FlowStatusId id = this.get(key).getId(); // pre-check to throw `HttpStatus.S_404_NOT_FOUND`, in case FlowExecution doesn't exist + addDagAction(id.getFlowGroup(), id.getFlowName(), id.getFlowExecutionId(), DagActionStore.DagActionType.RESUME); + } + + public UpdateResponse delete(ComplexResourceKey key) { + FlowStatusId id = this.get(key).getId(); // pre-check to throw `HttpStatus.S_404_NOT_FOUND`, in case FlowExecution doesn't exist + addDagAction(id.getFlowGroup(), id.getFlowName(), id.getFlowExecutionId(), DagActionStore.DagActionType.KILL); + return new UpdateResponse(HttpStatus.S_200_OK); + } + + /** NOTE: may throw {@link RestLiServiceException}, see: https://linkedin.github.io/rest.li/user_guide/restli_server#returning-errors */ + protected void addDagAction(String flowGroup, String flowName, Long flowExecutionId, DagActionStore.DagActionType actionType) { + try { + // If an existing resume request is still pending then do not accept this request + if (this.dagManagementStateStore.existsFlowDagAction(flowGroup, flowName, flowExecutionId, actionType)) { + this.throwErrorResponse("There is already a pending " + actionType + " action for this flow. Please wait to resubmit and wait " + + "for action to be completed.", HttpStatus.S_409_CONFLICT); + return; + } + this.dagManagementStateStore.addFlowDagAction(flowGroup, flowName, flowExecutionId, actionType); + } catch (IOException | SQLException e) { + log.warn( + String.format("Failed to add %s action for flow %s %s %s to dag action store due to:", actionType, flowGroup, + flowName, flowExecutionId), e); + this.throwErrorResponse(e.getMessage(), HttpStatus.S_500_INTERNAL_SERVER_ERROR); + } + } + + private void throwErrorResponse(String exceptionMessage, HttpStatus errorType) { + throw StringUtils.isBlank(exceptionMessage) ? new RestLiServiceException(errorType) : new RestLiServiceException(errorType, exceptionMessage); + } + + public static List getLatestFlowGroupStatusesFromGenerator(String flowGroup, + Integer countPerFlowName, String tag, FlowStatusGenerator flowStatusGenerator) { + if (countPerFlowName == null) { + countPerFlowName = 1; + } + log.info("get latest (for group) called with flowGroup " + flowGroup + " count " + countPerFlowName); + + return flowStatusGenerator.getFlowStatusesAcrossGroup(flowGroup, countPerFlowName, tag); + } +} 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 deleted file mode 100644 index f0db8b5826f..00000000000 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigResourceHandler.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * 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.restli; - -import java.io.IOException; -import java.util.Collection; -import java.util.Properties; -import java.util.UUID; - -import org.apache.helix.HelixManager; -import org.apache.helix.InstanceType; - -import com.google.common.base.Optional; -import com.linkedin.data.transform.DataProcessingException; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.common.PatchRequest; -import com.linkedin.restli.server.CreateResponse; -import com.linkedin.restli.server.UpdateResponse; -import com.linkedin.restli.server.util.PatchApplier; - -import javax.inject.Inject; -import javax.inject.Named; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.configuration.ConfigurationKeys; -import org.apache.gobblin.runtime.api.FlowSpecSearchObject; -import org.apache.gobblin.service.FlowConfig; -import org.apache.gobblin.service.FlowConfigLoggedException; -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.modules.scheduler.GobblinServiceJobScheduler; -import org.apache.gobblin.service.modules.utils.HelixUtils; -import org.apache.gobblin.runtime.util.InjectionNames; - - -/** - * An HA (high available) aware {@link FlowConfigsResourceHandler} which consider if current node is Active or Standby. - * When a Standby mode detected, it will forward the rest-li request ({@link FlowConfig}) - * to the Active. Otherwise it will handle it locally. - */ -@Slf4j -public class GobblinServiceFlowConfigResourceHandler implements FlowConfigsResourceHandler { - @Getter - private String serviceName; - private boolean flowCatalogLocalCommit; - protected FlowConfigResourceLocalHandler localHandler; - private Optional helixManager; - private GobblinServiceJobScheduler jobScheduler; - private boolean forceLeader; - - @Inject - public GobblinServiceFlowConfigResourceHandler(@Named(InjectionNames.SERVICE_NAME) String serviceName, - @Named(InjectionNames.FLOW_CATALOG_LOCAL_COMMIT) boolean flowCatalogLocalCommit, - FlowConfigResourceLocalHandler handler, - Optional manager, - GobblinServiceJobScheduler jobScheduler, - @Named(InjectionNames.FORCE_LEADER) boolean forceLeader) { - this.flowCatalogLocalCommit = flowCatalogLocalCommit; - this.serviceName = serviceName; - this.localHandler = handler; - this.helixManager = manager; - this.jobScheduler = jobScheduler; - this.forceLeader = forceLeader; - } - - @Override - public FlowConfig getFlowConfig(FlowId flowId) - throws FlowConfigLoggedException { - return this.localHandler.getFlowConfig(flowId); - } - - @Override - public Collection getFlowConfig(FlowSpecSearchObject flowSpecSearchObject) throws FlowConfigLoggedException { - return this.localHandler.getFlowConfig(flowSpecSearchObject); - } - - @Override - public Collection getAllFlowConfigs() { - return this.localHandler.getAllFlowConfigs(); - } - - @Override - public Collection getAllFlowConfigs(int start, int count) { - return this.localHandler.getAllFlowConfigs(start, 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 - * then call {@link FlowConfigResourceLocalHandler#createFlowConfig(FlowConfig)}. - * - * Please refer to {@link org.apache.gobblin.service.modules.core.ControllerUserDefinedMessageHandlerFactory} for remote handling. - * - * For better I/O load balance, user can enable {@link GobblinServiceFlowConfigResourceHandler#flowCatalogLocalCommit}. - * The {@link FlowConfig} will be then persisted to {@link org.apache.gobblin.runtime.spec_catalog.FlowCatalog} first before it is - * forwarded to active node (if current node is standby) for execution. - */ - @Override - public CreateResponse createFlowConfig(FlowConfig flowConfig) - throws FlowConfigLoggedException { - String flowName = flowConfig.getId().getFlowName(); - String flowGroup = flowConfig.getId().getFlowGroup(); - - if (flowConfig.getProperties().containsKey(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, - String.format("%s cannot be set by the user", ConfigurationKeys.FLOW_EXECUTION_ID_KEY), - null); - } - - checkHelixConnection(ServiceConfigKeys.HELIX_FLOWSPEC_ADD, flowName, flowGroup); - - if (forceLeader) { - HelixUtils.throwErrorIfNotLeader(helixManager); - } - - try { - if (!jobScheduler.isActive() && helixManager.isPresent()) { - CreateResponse response = null; - if (this.flowCatalogLocalCommit) { - // We will handle FS I/O locally for load balance before forwarding to remote node. - response = this.localHandler.createFlowConfig(flowConfig, true); - } - - 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); - } - - // Do actual work on remote node, directly return success - return response == null ? new CreateResponse(new ComplexResourceKey<>(flowConfig.getId(), new EmptyRecord()), - HttpStatus.S_201_CREATED) : response; - } else { - return this.localHandler.createFlowConfig(flowConfig); - } - } catch (IOException e) { - throw new FlowConfigLoggedException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, - "Cannot create flowConfig [flowName=" + flowName + " flowGroup=" + flowGroup + "]", e); - } - } - - /** - * Updating {@link FlowConfig} should check if current node is active (master). - * If current node is active, call {@link FlowConfigResourceLocalHandler#updateFlowConfig(FlowId, FlowConfig)} directly. - * If current node is standby, forward {@link ServiceConfigKeys#HELIX_FLOWSPEC_UPDATE} to active. The remote active will - * then call {@link FlowConfigResourceLocalHandler#updateFlowConfig(FlowId, FlowConfig)}. - * - * Please refer to {@link org.apache.gobblin.service.modules.core.ControllerUserDefinedMessageHandlerFactory} for remote handling. - * - * For better I/O load balance, user can enable {@link GobblinServiceFlowConfigResourceHandler#flowCatalogLocalCommit}. - * The {@link FlowConfig} will be then persisted to {@link org.apache.gobblin.runtime.spec_catalog.FlowCatalog} first before it is - * forwarded to active node (if current node is standby) for execution. - */ - @Override - public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig) - throws FlowConfigLoggedException { - String flowName = flowId.getFlowName(); - String flowGroup = flowId.getFlowGroup(); - - if (!flowGroup.equals(flowConfig.getId().getFlowGroup()) || !flowName.equals(flowConfig.getId().getFlowName())) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, - "flowName and flowGroup cannot be changed in update", null); - } - - checkHelixConnection(ServiceConfigKeys.HELIX_FLOWSPEC_UPDATE, flowName, flowGroup); - - if (forceLeader) { - HelixUtils.throwErrorIfNotLeader(helixManager); - } - - try { - if (!jobScheduler.isActive() && helixManager.isPresent()) { - - if (this.flowCatalogLocalCommit) { - // We will handle FS I/O locally for load balance before forwarding to remote node. - this.localHandler.updateFlowConfig(flowId, flowConfig, false); - } - - forwardMessage(ServiceConfigKeys.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 + "]"); - return new UpdateResponse(HttpStatus.S_200_OK); - } else { - return this.localHandler.updateFlowConfig(flowId, flowConfig); - } - - } catch (IOException e) { - throw new FlowConfigLoggedException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, - "Cannot update flowConfig [flowName=" + flowName + " flowGroup=" + flowGroup + "]", e); - } - } - - @Override - public UpdateResponse partialUpdateFlowConfig(FlowId flowId, PatchRequest flowConfigPatch) { - FlowConfig flowConfig = getFlowConfig(flowId); - - try { - PatchApplier.applyPatch(flowConfig, flowConfigPatch); - } catch (DataProcessingException e) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, "Failed to apply partial update", e); - } - - return updateFlowConfig(flowId, flowConfig); - } - - /** - * Deleting {@link FlowConfig} should check if current node is active (master). - * If current node is active, call {@link FlowConfigResourceLocalHandler#deleteFlowConfig(FlowId, Properties)} directly. - * If current node is standby, forward {@link ServiceConfigKeys#HELIX_FLOWSPEC_REMOVE} to active. The remote active will - * then call {@link FlowConfigResourceLocalHandler#deleteFlowConfig(FlowId, Properties)}. - * - * Please refer to {@link org.apache.gobblin.service.modules.core.ControllerUserDefinedMessageHandlerFactory} for remote handling. - * - * For better I/O load balance, user can enable {@link GobblinServiceFlowConfigResourceHandler#flowCatalogLocalCommit}. - * The {@link FlowConfig} will be then persisted to {@link org.apache.gobblin.runtime.spec_catalog.FlowCatalog} first before it is - * forwarded to active node (if current node is standby) for execution. - */ - @Override - public UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) - throws FlowConfigLoggedException { - String flowName = flowId.getFlowName(); - String flowGroup = flowId.getFlowGroup(); - - checkHelixConnection(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE, flowName, flowGroup); - - if (forceLeader) { - HelixUtils.throwErrorIfNotLeader(helixManager); - } - - try { - if (!jobScheduler.isActive() && helixManager.isPresent()) { - - if (this.flowCatalogLocalCommit) { - // We will handle FS I/O locally for load balance before forwarding to remote node. - this.localHandler.deleteFlowConfig(flowId, header, false); - } - - forwardMessage(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE, FlowConfigUtils.serializeFlowId(flowId), flowName, flowGroup); - - return new UpdateResponse(HttpStatus.S_200_OK); - } else { - return this.localHandler.deleteFlowConfig(flowId, header); - } - } catch (IOException e) { - throw new FlowConfigLoggedException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, - "Cannot delete flowConfig [flowName=" + flowName + " flowGroup=" + flowGroup + "]", e); - } - } - - private void checkHelixConnection(String opr, String flowName, String flowGroup) throws FlowConfigLoggedException { - if (this.helixManager.isPresent() && !this.helixManager.get().isConnected()) { - // Specs in store will be notified when Scheduler is added as listener to FlowCatalog, so ignore - // .. Specs if in cluster mode and Helix is not yet initialized - log.warn("System not yet initialized. Skipping operation " + opr); - throw new FlowConfigLoggedException(HttpStatus.S_500_INTERNAL_SERVER_ERROR, - "System not yet initialized. Skipping " + opr + " flowConfig [flowName=" + flowName + " flowGroup=" + flowGroup + "]"); - } - } - - private void forwardMessage(String msgSubType, String val, String flowName, String flowGroup) { - HelixUtils.sendUserDefinedMessage(msgSubType, val, UUID.randomUUID().toString(), InstanceType.CONTROLLER, - helixManager.get(), log); - log.info("{} Forwarding {} flowConfig [flowName={} flowGroup={}", serviceName, msgSubType, flowName, flowGroup + "]"); - } -} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandler.java deleted file mode 100644 index 6389d4d0e4b..00000000000 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandler.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.restli; - -import org.apache.helix.HelixManager; - -import com.google.common.base.Optional; - -import javax.inject.Inject; -import javax.inject.Named; - -import org.apache.gobblin.service.FlowConfigV2ResourceLocalHandler; -import org.apache.gobblin.service.FlowConfigsV2ResourceHandler; -import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; -import org.apache.gobblin.runtime.util.InjectionNames; - - -public class GobblinServiceFlowConfigV2ResourceHandler extends GobblinServiceFlowConfigResourceHandler - implements FlowConfigsV2ResourceHandler { - - @Inject - public GobblinServiceFlowConfigV2ResourceHandler(@Named(InjectionNames.SERVICE_NAME) String serviceName, - @Named(InjectionNames.FLOW_CATALOG_LOCAL_COMMIT) boolean flowCatalogLocalCommit, - FlowConfigV2ResourceLocalHandler handler, Optional manager, GobblinServiceJobScheduler jobScheduler, - @Named(InjectionNames.FORCE_LEADER) boolean forceLeader) { - super(serviceName, flowCatalogLocalCommit, handler, manager, jobScheduler, forceLeader); - } -} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby.java deleted file mode 100644 index 7bfd93a4c7b..00000000000 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.restli; - -import com.google.common.base.Optional; -import com.linkedin.data.transform.DataProcessingException; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.common.PatchRequest; -import com.linkedin.restli.server.CreateResponse; -import com.linkedin.restli.server.UpdateResponse; -import com.linkedin.restli.server.util.PatchApplier; -import java.util.Properties; -import javax.inject.Inject; -import javax.inject.Named; -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.configuration.ConfigurationKeys; -import org.apache.gobblin.service.FlowConfig; -import org.apache.gobblin.service.FlowConfigLoggedException; -import org.apache.gobblin.service.FlowConfigResourceLocalHandler; -import org.apache.gobblin.service.FlowConfigV2ResourceLocalHandler; -import org.apache.gobblin.service.FlowId; -import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler; -import org.apache.gobblin.runtime.util.InjectionNames; -import org.apache.helix.HelixManager; - -@Slf4j -public class GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby extends GobblinServiceFlowConfigV2ResourceHandler { - @Inject - public GobblinServiceFlowConfigV2ResourceHandlerWithWarmStandby(@Named(InjectionNames.SERVICE_NAME) String serviceName, - @Named(InjectionNames.FLOW_CATALOG_LOCAL_COMMIT) boolean flowCatalogLocalCommit, - FlowConfigV2ResourceLocalHandler handler, Optional manager, GobblinServiceJobScheduler jobScheduler, - @Named(InjectionNames.FORCE_LEADER) boolean forceLeader) { - super(serviceName, flowCatalogLocalCommit, handler, manager, jobScheduler, forceLeader); - } - - - @Override - public UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) - throws FlowConfigLoggedException { - return this.localHandler.deleteFlowConfig(flowId, header); - } - - @Override - public UpdateResponse partialUpdateFlowConfig(FlowId flowId, - PatchRequest flowConfigPatch) throws FlowConfigLoggedException { - long modifiedWatermark = System.currentTimeMillis() / 1000; - FlowConfig flowConfig = getFlowConfig(flowId); - - try { - PatchApplier.applyPatch(flowConfig, flowConfigPatch); - } catch (DataProcessingException e) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, "Failed to apply partial update", e); - } - - return updateFlowConfig(flowId, flowConfig, modifiedWatermark); - } - - @Override - public UpdateResponse updateFlowConfig(FlowId flowId, - FlowConfig flowConfig) throws FlowConfigLoggedException { - // We have modifiedWatermark here to avoid update config happens at the same time on different hosts overwrite each other - // timestamp here will be treated as largest modifiedWatermark that we can update - long version = System.currentTimeMillis() / 1000; - return updateFlowConfig(flowId, flowConfig, version); - } - public UpdateResponse updateFlowConfig(FlowId flowId, - FlowConfig flowConfig, long modifiedWatermark) throws FlowConfigLoggedException { - String flowName = flowId.getFlowName(); - String flowGroup = flowId.getFlowGroup(); - - if (!flowGroup.equals(flowConfig.getId().getFlowGroup()) || !flowName.equals(flowConfig.getId().getFlowName())) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, - "flowName and flowGroup cannot be changed in update", null); - } - - // We directly call localHandler to create flow config and put it in spec store - - //Instead of helix message, forwarding message is done by change stream of spec store - - return this.localHandler.updateFlowConfig(flowId, flowConfig, true, modifiedWatermark); - } - /** - * Adding {@link FlowConfig} call {@link FlowConfigResourceLocalHandler#createFlowConfig(FlowConfig)} directly. - * no matter it's active or standby, rely on the CDC stream for spec store to forward the change to other hosts - * - */ - @Override - public CreateResponse createFlowConfig(FlowConfig flowConfig) - throws FlowConfigLoggedException { - - if (flowConfig.getProperties().containsKey(ConfigurationKeys.FLOW_EXECUTION_ID_KEY)) { - throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, - String.format("%s cannot be set by the user", ConfigurationKeys.FLOW_EXECUTION_ID_KEY), null); - } - - - CreateResponse response = null; - // We directly call localHandler to create flow config and put it in spec store - response = this.localHandler.createFlowConfig(flowConfig, true); - - //Instead of helix message, forwarding message is done by change stream of spec store - - // Do actual work on remote node, directly return success - - return response == null ? new CreateResponse(new ComplexResourceKey<>(flowConfig.getId(), new EmptyRecord()), - HttpStatus.S_201_CREATED) : response; - - } -} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandler.java deleted file mode 100644 index 489eb2b4da1..00000000000 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandler.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.restli; - -import java.util.List; - -import org.apache.helix.HelixManager; - -import com.google.common.base.Optional; -import com.google.common.eventbus.EventBus; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.server.PagingContext; -import com.linkedin.restli.server.UpdateResponse; - -import javax.inject.Inject; -import javax.inject.Named; -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.service.FlowExecution; -import org.apache.gobblin.service.FlowExecutionResourceHandler; -import org.apache.gobblin.service.FlowExecutionResourceLocalHandler; -import org.apache.gobblin.service.FlowId; -import org.apache.gobblin.service.FlowStatusId; -import org.apache.gobblin.service.modules.core.GobblinServiceManager; -import org.apache.gobblin.service.modules.utils.HelixUtils; -import org.apache.gobblin.runtime.util.InjectionNames; -import org.apache.gobblin.service.monitoring.KillFlowEvent; -import org.apache.gobblin.service.monitoring.ResumeFlowEvent; - - -/** - * {@link FlowExecutionResourceHandler} that calls underlying resource handler, but does extra work that requires objects - * like the {@link HelixManager}. For now, that is just checking leadership and sending the kill through the eventBus - * for the delete method. - */ -@Slf4j -public class GobblinServiceFlowExecutionResourceHandler implements FlowExecutionResourceHandler { - private FlowExecutionResourceLocalHandler localHandler; - private EventBus eventBus; - private Optional helixManager; - private boolean forceLeader; - - @Inject - public GobblinServiceFlowExecutionResourceHandler(FlowExecutionResourceLocalHandler handler, - @Named(GobblinServiceManager.SERVICE_EVENT_BUS_NAME) EventBus eventBus, - Optional manager, @Named(InjectionNames.FORCE_LEADER) boolean forceLeader) { - this.localHandler = handler; - this.eventBus = eventBus; - this.helixManager = manager; - this.forceLeader = forceLeader; - } - - @Override - public FlowExecution get(ComplexResourceKey key) { - return this.localHandler.get(key); - } - - @Override - public List getLatestFlowExecution(PagingContext context, FlowId flowId, Integer count, String tag, - String executionStatus, Boolean includeIssues) { - return this.localHandler.getLatestFlowExecution(context, flowId, count, tag, executionStatus, includeIssues); - } - - @Override - public List getLatestFlowGroupExecutions(PagingContext context, String flowGroup, Integer countPerFlow, - String tag, Boolean includeIssues) { - return this.localHandler.getLatestFlowGroupExecutions(context, flowGroup, countPerFlow, tag, includeIssues); - } - - @Override - public void resume(ComplexResourceKey key) { - String flowGroup = key.getKey().getFlowGroup(); - String flowName = key.getKey().getFlowName(); - Long flowExecutionId = key.getKey().getFlowExecutionId(); - if (this.forceLeader) { - HelixUtils.throwErrorIfNotLeader(this.helixManager); - } - this.eventBus.post(new ResumeFlowEvent(flowGroup, flowName, flowExecutionId)); - } - - @Override - public UpdateResponse delete(ComplexResourceKey key) { - String flowGroup = key.getKey().getFlowGroup(); - String flowName = key.getKey().getFlowName(); - Long flowExecutionId = key.getKey().getFlowExecutionId(); - if (this.forceLeader) { - HelixUtils.throwErrorIfNotLeader(this.helixManager); - } - this.eventBus.post(new KillFlowEvent(flowGroup, flowName, flowExecutionId)); - return new UpdateResponse(HttpStatus.S_200_OK); - } -} 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 deleted file mode 100644 index 1156e2826ae..00000000000 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/GobblinServiceFlowExecutionResourceHandlerWithWarmStandby.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.restli; - -import java.io.IOException; -import java.sql.SQLException; - -import org.apache.commons.lang.StringUtils; -import org.apache.helix.HelixManager; - -import com.google.common.base.Optional; -import com.google.common.eventbus.EventBus; -import com.google.inject.Inject; -import com.linkedin.restli.common.ComplexResourceKey; -import com.linkedin.restli.common.EmptyRecord; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.server.RestLiServiceException; -import com.linkedin.restli.server.UpdateResponse; - -import javax.inject.Named; -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.runtime.util.InjectionNames; -import org.apache.gobblin.service.FlowExecutionResourceLocalHandler; -import org.apache.gobblin.service.FlowStatusId; -import org.apache.gobblin.service.modules.core.GobblinServiceManager; -import org.apache.gobblin.service.modules.orchestration.DagActionStore; -import org.apache.gobblin.service.modules.orchestration.DagManagementStateStore; - - -@Slf4j -public class GobblinServiceFlowExecutionResourceHandlerWithWarmStandby extends GobblinServiceFlowExecutionResourceHandler{ - private DagManagementStateStore dagManagementStateStore; - @Inject - public GobblinServiceFlowExecutionResourceHandlerWithWarmStandby(FlowExecutionResourceLocalHandler handler, - @Named(GobblinServiceManager.SERVICE_EVENT_BUS_NAME) EventBus eventBus, - Optional manager, @Named(InjectionNames.FORCE_LEADER) boolean forceLeader, DagManagementStateStore dagManagementStateStore) { - super(handler, eventBus, manager, forceLeader); - this.dagManagementStateStore = dagManagementStateStore; - } - - @Override - public void resume(ComplexResourceKey key) { - FlowStatusId id = this.get(key).getId(); // pre-check to throw `HttpStatus.S_404_NOT_FOUND`, in case FlowExecution doesn't exist - addDagAction(id.getFlowGroup(), id.getFlowName(), id.getFlowExecutionId(), DagActionStore.DagActionType.RESUME); - } - - @Override - public UpdateResponse delete(ComplexResourceKey key) { - FlowStatusId id = this.get(key).getId(); // pre-check to throw `HttpStatus.S_404_NOT_FOUND`, in case FlowExecution doesn't exist - addDagAction(id.getFlowGroup(), id.getFlowName(), id.getFlowExecutionId(), DagActionStore.DagActionType.KILL); - return new UpdateResponse(HttpStatus.S_200_OK); - } - - /** NOTE: may throw {@link RestLiServiceException}, see: https://linkedin.github.io/rest.li/user_guide/restli_server#returning-errors */ - protected void addDagAction(String flowGroup, String flowName, Long flowExecutionId, DagActionStore.DagActionType actionType) { - try { - // If an existing resume request is still pending then do not accept this request - if (this.dagManagementStateStore.existsFlowDagAction(flowGroup, flowName, flowExecutionId, actionType)) { - this.throwErrorResponse("There is already a pending " + actionType + " action for this flow. Please wait to resubmit and wait " - + "for action to be completed.", HttpStatus.S_409_CONFLICT); - return; - } - this.dagManagementStateStore.addFlowDagAction(flowGroup, flowName, flowExecutionId, actionType); - } catch (IOException | SQLException e) { - log.warn( - String.format("Failed to add %s action for flow %s %s %s to dag action store due to:", actionType, flowGroup, - flowName, flowExecutionId), e); - this.throwErrorResponse(e.getMessage(), HttpStatus.S_500_INTERNAL_SERVER_ERROR); - } - } - - private void throwErrorResponse(String exceptionMessage, HttpStatus errorType) { - throw StringUtils.isBlank(exceptionMessage) ? new RestLiServiceException(errorType) : new RestLiServiceException(errorType, exceptionMessage); - } -} diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/HelixUtils.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/HelixUtils.java deleted file mode 100644 index 9c4efbcd7e3..00000000000 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/HelixUtils.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * 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.utils; - -import java.io.UnsupportedEncodingException; -import java.net.InetAddress; -import java.net.URLDecoder; -import java.net.URLEncoder; -import java.net.UnknownHostException; - -import org.apache.helix.Criteria; -import org.apache.helix.HelixManager; -import org.apache.helix.HelixManagerFactory; -import org.apache.helix.InstanceType; -import org.apache.helix.PropertyKey; -import org.apache.helix.manager.zk.ZKHelixManager; -import org.apache.helix.model.HelixConfigScope; -import org.apache.helix.model.LiveInstance; -import org.apache.helix.model.Message; -import org.apache.helix.tools.ClusterSetup; -import org.slf4j.Logger; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableMap; -import com.linkedin.data.DataMap; -import com.linkedin.restli.common.HttpStatus; -import com.linkedin.restli.server.RestLiServiceException; -import com.typesafe.config.Config; - -import lombok.extern.slf4j.Slf4j; - -import org.apache.gobblin.annotation.Alpha; -import org.apache.gobblin.service.ServiceConfigKeys; -import org.apache.gobblin.util.ConfigUtils; - - -@Alpha -@Slf4j -public class HelixUtils { - public static final String HELIX_INSTANCE_NAME_SEPARATOR = "@"; - - /*** - * Build a Helix Manager (Helix Controller instance). - * - * @param helixInstanceName the Helix Instance name. - * @param helixClusterName the Helix Cluster name. - * @param zkConnectionString the ZooKeeper connection string. - * @return HelixManager - */ - public static HelixManager buildHelixManager(String helixInstanceName, String helixClusterName, String zkConnectionString) { - return HelixManagerFactory.getZKHelixManager(helixClusterName, helixInstanceName, - InstanceType.CONTROLLER, zkConnectionString); - } - - /** - * Create a Helix cluster for the Gobblin Cluster application. - * - * @param zkConnectionString the ZooKeeper connection string - * @param clusterName the Helix cluster name - */ - public static void createGobblinHelixCluster(String zkConnectionString, String clusterName) { - createGobblinHelixCluster(zkConnectionString, clusterName, true); - } - - /** - * Create a Helix cluster for the Gobblin Cluster application. - * - * @param zkConnectionString the ZooKeeper connection string - * @param clusterName the Helix cluster name - * @param overwrite true to overwrite exiting cluster, false to reuse existing cluster - */ - public static void createGobblinHelixCluster(String zkConnectionString, String clusterName, boolean overwrite) { - ClusterSetup clusterSetup = new ClusterSetup(zkConnectionString); - // Create the cluster and overwrite if it already exists - clusterSetup.addCluster(clusterName, overwrite); - // Helix 0.6.x requires a configuration property to have the form key=value. - String autoJoinConfig = ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN + "=true"; - clusterSetup.setConfig(HelixConfigScope.ConfigScopeProperty.CLUSTER, clusterName, autoJoinConfig); - } - - /** - * Get a Helix instance name. - * - * @param namePrefix a prefix of Helix instance names - * @param instanceId an integer instance ID - * @return a Helix instance name that is a concatenation of the given prefix and instance ID - */ - public static String getHelixInstanceName(String namePrefix, int instanceId) { - return namePrefix + "_" + instanceId; - } - - @VisibleForTesting - public static void sendUserDefinedMessage(String messageSubType, String messageVal, String messageId, - InstanceType instanceType, HelixManager helixManager, Logger logger) { - Criteria criteria = new Criteria(); - criteria.setInstanceName("%"); - criteria.setResource("%"); - criteria.setPartition("%"); - criteria.setPartitionState("%"); - criteria.setRecipientInstanceType(instanceType); - criteria.setSessionSpecific(true); - - Message message = new Message(Message.MessageType.USER_DEFINE_MSG.toString(), messageId); - message.setMsgSubType(messageSubType); - message.setAttribute(Message.Attributes.INNER_MESSAGE, messageVal); - message.setMsgState(Message.MessageState.NEW); - message.setTgtSessionId("*"); - - int messagesSent = helixManager.getMessagingService().send(criteria, message); - if (messagesSent == 0) { - logger.error(String.format("Failed to send the %s message to the participants", message)); - } - } - - private static String getUrlFromHelixInstanceName(String helixInstanceName) { - if (!helixInstanceName.contains(HELIX_INSTANCE_NAME_SEPARATOR)) { - return null; - } else { - String url = helixInstanceName.substring(helixInstanceName.indexOf(HELIX_INSTANCE_NAME_SEPARATOR) + 1); - try { - return URLDecoder.decode(url, "UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("Failed to decode URL from helix instance name", e); - } - } - } - - private static String getLeaderUrl(HelixManager helixManager) { - PropertyKey key = helixManager.getHelixDataAccessor().keyBuilder().controllerLeader(); - LiveInstance leader = helixManager.getHelixDataAccessor().getProperty(key); - return getUrlFromHelixInstanceName(leader.getInstanceName()); - } - - /** - * If this host is not the leader, throw a {@link RestLiServiceException}, and include the URL of the leader host in - * the message and in the errorDetails under the key {@link ServiceConfigKeys#LEADER_URL}. - */ - public static void throwErrorIfNotLeader(Optional helixManager) { - if (helixManager.isPresent() && !helixManager.get().isLeader()) { - String leaderUrl = getLeaderUrl(helixManager.get()); - if (leaderUrl == null) { - throw new RuntimeException("Request sent to slave node but could not get leader node URL"); - } - RestLiServiceException exception = new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, "Request must be sent to leader node at URL " + leaderUrl); - exception.setErrorDetails(new DataMap(ImmutableMap.of(ServiceConfigKeys.LEADER_URL, leaderUrl))); - throw exception; - } - } - - /** - * Build helix instance name by getting {@link org.apache.gobblin.service.ServiceConfigKeys#HELIX_INSTANCE_NAME_KEY} - * and appending the host, port, and service name with a separator - */ - public static String buildHelixInstanceName(Config config, String defaultInstanceName) { - String helixInstanceName = ConfigUtils - .getString(config, ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, defaultInstanceName); - - String url = ""; - try { - url = ConfigUtils.getString(config, ServiceConfigKeys.SERVICE_URL_PREFIX, "https://") - + InetAddress.getLocalHost().getHostName() + ":" + ConfigUtils.getString(config, ServiceConfigKeys.SERVICE_PORT, "") - + "/" + ConfigUtils.getString(config, ServiceConfigKeys.SERVICE_NAME, ""); - url = HELIX_INSTANCE_NAME_SEPARATOR + URLEncoder.encode(url, "UTF-8"); - } catch (UnknownHostException | UnsupportedEncodingException e) { - log.warn("Failed to construct helix instance name", e); - } - - return helixInstanceName + url; - } -} 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 e6ec349946c..44005afa752 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 @@ -114,7 +114,6 @@ Metrics need to be created before initializeMonitor() below is called (or more s protected void assignTopicPartitions() { // Expects underlying consumer to handle initializing partitions and offset for the topic - // subscribe to all partitions from latest offset - return; } /** @@ -161,7 +160,7 @@ protected void initializeMonitor() { protected void startUp() {} /* - This method should be called once by the {@link GobblinServiceManager} only after the DagManager, FlowGraph and + This method should be called once by the {@link GobblinServiceManager} only after the FlowGraph and SpecCompiler are initialized and running. */ public synchronized void setActive() { @@ -233,8 +232,7 @@ protected abstract void handleDagAction(String operation, DagActionStore.DagActi String flowName, long flowExecutionId, DagActionStore.DagActionType dagActionType); /** - * For a given dagAction, calls the appropriate method in the DagManager to carry out the desired action. - * @param isStartup true if called for dagAction loaded directly from store upon startup, false otherwise + * This implementation passes on the {@link DagActionStore.DagAction} to {@link DagManagement}. */ protected abstract void handleDagAction(DagActionStore.DagAction dagAction, boolean isStartup); diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowStatusTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionTest.java similarity index 79% rename from gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowStatusTest.java rename to gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionTest.java index d4950fcf366..15380be091c 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowStatusTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionTest.java @@ -30,15 +30,15 @@ import com.google.common.base.Suppliers; import com.google.common.collect.Lists; -import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; -import com.google.inject.Module; import org.apache.gobblin.configuration.State; import org.apache.gobblin.metastore.StateStore; import org.apache.gobblin.restli.EmbeddedRestliServer; import org.apache.gobblin.runtime.troubleshooter.MultiContextIssueRepository; +import org.apache.gobblin.service.modules.orchestration.DagManagementStateStore; +import org.apache.gobblin.service.modules.restli.FlowExecutionResourceHandler; import org.apache.gobblin.service.monitoring.FlowStatusGenerator; import org.apache.gobblin.service.monitoring.JobStatusRetriever; @@ -46,13 +46,12 @@ @Test(groups = { "gobblin.service" }, singleThreaded = true) -public class FlowStatusTest { - private FlowStatusClient _client; +public class FlowExecutionTest { + private FlowExecutionClient client; private EmbeddedRestliServer _server; private List> _listOfJobStatusLists; class TestJobStatusRetriever extends JobStatusRetriever { - protected TestJobStatusRetriever(MultiContextIssueRepository issueRepository) { super(issueRepository); } @@ -104,22 +103,19 @@ public void setUp() throws Exception { JobStatusRetriever jobStatusRetriever = new TestJobStatusRetriever(mock(MultiContextIssueRepository.class)); final FlowStatusGenerator flowStatusGenerator = new FlowStatusGenerator(jobStatusRetriever); - Injector injector = Guice.createInjector(new Module() { - @Override - public void configure(Binder binder) { - binder.bind(FlowStatusGenerator.class) - .toInstance(flowStatusGenerator); - } + Injector injector = Guice.createInjector(binder -> { + binder.bind(FlowStatusGenerator.class).toInstance(flowStatusGenerator); + binder.bind(FlowExecutionResourceHandlerInterface.class) + .toInstance(new FlowExecutionResourceHandler(flowStatusGenerator, mock(DagManagementStateStore.class))); }); _server = EmbeddedRestliServer.builder().resources( - Lists.newArrayList(FlowStatusResource.class)).injector(injector).build(); + Lists.newArrayList(FlowExecutionResource.class)).injector(injector).build(); _server.startAsync(); _server.awaitRunning(); - _client = - new FlowStatusClient(String.format("http://localhost:%s/", _server.getPort())); + client = new FlowExecutionClient(String.format("http://localhost:%s/", _server.getPort())); } /** @@ -165,16 +161,16 @@ public void testFindLatest() throws Exception { _listOfJobStatusLists.add(jobStatusList2); FlowId flowId = new FlowId().setFlowGroup("fgroup1").setFlowName("flow1"); - FlowStatus flowStatus = _client.getLatestFlowStatus(flowId); + FlowExecution flowExecution = client.getLatestFlowExecution(flowId); - Assert.assertEquals(flowStatus.getId().getFlowGroup(), "fgroup1"); - Assert.assertEquals(flowStatus.getId().getFlowName(), "flow1"); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionStartTime().longValue(), 1L); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionEndTime().longValue(), 7000L); - Assert.assertEquals(flowStatus.getMessage(), fs2.getMessage()); - Assert.assertEquals(flowStatus.getExecutionStatus(), ExecutionStatus.COMPLETE); + Assert.assertEquals(flowExecution.getId().getFlowGroup(), "fgroup1"); + Assert.assertEquals(flowExecution.getId().getFlowName(), "flow1"); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionStartTime().longValue(), 1L); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionEndTime().longValue(), 7000L); + Assert.assertEquals(flowExecution.getMessage(), fs2.getMessage()); + Assert.assertEquals(flowExecution.getExecutionStatus(), ExecutionStatus.COMPLETE); - JobStatusArray jobStatuses = flowStatus.getJobStatuses(); + JobStatusArray jobStatuses = flowExecution.getJobStatuses(); Assert.assertEquals(jobStatusList2.size(), jobStatuses.size() + 1); @@ -185,15 +181,15 @@ public void testFindLatest() throws Exception { compareJobStatus(js, mjs); } - List flowStatusList = _client.getLatestFlowStatus(flowId, 2, null); - Assert.assertEquals(flowStatusList.size(), 2); - Assert.assertEquals(flowStatusList.get(0).getId().getFlowExecutionId(), (Long) 1L); - Assert.assertEquals(flowStatusList.get(1).getId().getFlowExecutionId(), (Long) 0L); - Assert.assertEquals(flowStatusList.get(0).getJobStatuses().size(), 2); + List flowExecutionsList = client.getLatestFlowExecution(flowId, 2, null); + Assert.assertEquals(flowExecutionsList.size(), 2); + Assert.assertEquals(flowExecutionsList.get(0).getId().getFlowExecutionId(), (Long) 1L); + Assert.assertEquals(flowExecutionsList.get(1).getId().getFlowExecutionId(), (Long) 0L); + Assert.assertEquals(flowExecutionsList.get(0).getJobStatuses().size(), 2); - List flowStatusList2 = _client.getLatestFlowStatus(flowId, 1, "dataset1"); - Assert.assertEquals(flowStatusList2.get(0).getJobStatuses().size(), 1); - Assert.assertEquals(flowStatusList2.get(0).getJobStatuses().get(0).getJobTag(), "dataset1"); + List flowExecutionsList2 = client.getLatestFlowExecution(flowId, 1, "dataset1"); + Assert.assertEquals(flowExecutionsList2.get(0).getJobStatuses().size(), 1); + Assert.assertEquals(flowExecutionsList2.get(0).getJobStatuses().get(0).getJobTag(), "dataset1"); } /** @@ -226,16 +222,16 @@ public void testGetCompleted() throws Exception { _listOfJobStatusLists.add(jobStatusList); FlowStatusId flowId = new FlowStatusId().setFlowGroup("fgroup1").setFlowName("flow1").setFlowExecutionId(0); - FlowStatus flowStatus = _client.getFlowStatus(flowId); + FlowExecution flowExecution = client.getFlowExecution(flowId); - Assert.assertEquals(flowStatus.getId().getFlowGroup(), "fgroup1"); - Assert.assertEquals(flowStatus.getId().getFlowName(), "flow1"); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionStartTime().longValue(), 0L); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionEndTime().longValue(), 7000L); - Assert.assertEquals(flowStatus.getMessage(), fs1.getMessage()); - Assert.assertEquals(flowStatus.getExecutionStatus(), ExecutionStatus.COMPLETE); + Assert.assertEquals(flowExecution.getId().getFlowGroup(), "fgroup1"); + Assert.assertEquals(flowExecution.getId().getFlowName(), "flow1"); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionStartTime().longValue(), 0L); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionEndTime().longValue(), 7000L); + Assert.assertEquals(flowExecution.getMessage(), fs1.getMessage()); + Assert.assertEquals(flowExecution.getExecutionStatus(), ExecutionStatus.COMPLETE); - JobStatusArray jobStatuses = flowStatus.getJobStatuses(); + JobStatusArray jobStatuses = flowExecution.getJobStatuses(); Assert.assertEquals(jobStatusList.size(), jobStatuses.size() + 1); @@ -277,23 +273,22 @@ public void testGetRunning() throws Exception { _listOfJobStatusLists.add(jobStatusList); FlowStatusId flowId = new FlowStatusId().setFlowGroup("fgroup1").setFlowName("flow1").setFlowExecutionId(0); - FlowStatus flowStatus = _client.getFlowStatus(flowId); + FlowExecution flowExecution = client.getFlowExecution(flowId); - Assert.assertEquals(flowStatus.getId().getFlowGroup(), "fgroup1"); - Assert.assertEquals(flowStatus.getId().getFlowName(), "flow1"); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionStartTime().longValue(), 0L); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionEndTime().longValue(), 6000L); - Assert.assertEquals(flowStatus.getMessage(), fs1.getMessage()); - Assert.assertEquals(flowStatus.getExecutionStatus(), ExecutionStatus.RUNNING); + Assert.assertEquals(flowExecution.getId().getFlowGroup(), "fgroup1"); + Assert.assertEquals(flowExecution.getId().getFlowName(), "flow1"); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionStartTime().longValue(), 0L); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionEndTime().longValue(), 6000L); + Assert.assertEquals(flowExecution.getMessage(), fs1.getMessage()); + Assert.assertEquals(flowExecution.getExecutionStatus(), ExecutionStatus.RUNNING); - JobStatusArray jobStatuses = flowStatus.getJobStatuses(); + JobStatusArray jobStatuses = flowExecution.getJobStatuses(); Assert.assertEquals(jobStatusList.size(), jobStatuses.size() + 1); for (int i = 0; i < jobStatuses.size(); i++) { org.apache.gobblin.service.monitoring.JobStatus mjs = jobStatusList.get(i); JobStatus js = jobStatuses.get(i); - compareJobStatus(js, mjs); } } @@ -328,16 +323,16 @@ public void testGetFailed() throws Exception { _listOfJobStatusLists.add(jobStatusList); FlowStatusId flowId = new FlowStatusId().setFlowGroup("fgroup1").setFlowName("flow1").setFlowExecutionId(0); - FlowStatus flowStatus = _client.getFlowStatus(flowId); + FlowExecution flowExecution = client.getFlowExecution(flowId); - Assert.assertEquals(flowStatus.getId().getFlowGroup(), "fgroup1"); - Assert.assertEquals(flowStatus.getId().getFlowName(), "flow1"); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionStartTime().longValue(), 0L); - Assert.assertEquals(flowStatus.getExecutionStatistics().getExecutionEndTime().longValue(), 7000L); - Assert.assertEquals(flowStatus.getMessage(), fs1.getMessage()); - Assert.assertEquals(flowStatus.getExecutionStatus(), ExecutionStatus.FAILED); + Assert.assertEquals(flowExecution.getId().getFlowGroup(), "fgroup1"); + Assert.assertEquals(flowExecution.getId().getFlowName(), "flow1"); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionStartTime().longValue(), 0L); + Assert.assertEquals(flowExecution.getExecutionStatistics().getExecutionEndTime().longValue(), 7000L); + Assert.assertEquals(flowExecution.getMessage(), fs1.getMessage()); + Assert.assertEquals(flowExecution.getExecutionStatus(), ExecutionStatus.FAILED); - JobStatusArray jobStatuses = flowStatus.getJobStatuses(); + JobStatusArray jobStatuses = flowExecution.getJobStatuses(); Assert.assertEquals(jobStatusList.size(), jobStatuses.size() + 1); @@ -351,8 +346,8 @@ public void testGetFailed() throws Exception { @AfterClass(alwaysRun = true) public void tearDown() throws Exception { - if (_client != null) { - _client.close(); + if (client != null) { + client.close(); } if (_server != null) { _server.stopAsync(); diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/GobblinServiceManagerTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/GobblinServiceManagerTest.java index d123d82d486..01e743f9cdc 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/GobblinServiceManagerTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/GobblinServiceManagerTest.java @@ -70,6 +70,7 @@ import org.apache.gobblin.service.modules.orchestration.MysqlDagStateStore; import org.apache.gobblin.service.modules.orchestration.MysqlMultiActiveLeaseArbiterTest; import org.apache.gobblin.service.modules.orchestration.ServiceAzkabanConfigKeys; +import org.apache.gobblin.service.modules.restli.FlowConfigsV2ResourceHandler; import org.apache.gobblin.service.modules.utils.FlowCompilationValidationHelper; import org.apache.gobblin.service.monitoring.DagManagementDagActionStoreChangeMonitor; import org.apache.gobblin.service.monitoring.FsJobStatusRetriever; @@ -336,11 +337,11 @@ public void testRestart() throws Exception { FlowConfig uncompilableFlowConfig = new FlowConfig().setId(UNCOMPILABLE_FLOW_ID).setTemplateUris(TEST_TEMPLATE_URI) .setSchedule(new Schedule().setCronSchedule(TEST_SCHEDULE).setRunImmediately(true)) .setProperties(new StringMap(flowProperties)); - FlowSpec uncompilableSpec = FlowConfigResourceLocalHandler.createFlowSpecForConfig(uncompilableFlowConfig); + FlowSpec uncompilableSpec = FlowConfigsV2ResourceHandler.createFlowSpecForConfig(uncompilableFlowConfig); FlowId flowId = createFlowIdWithUniqueName(TEST_GROUP_NAME); FlowConfig runOnceFlowConfig = new FlowConfig().setId(flowId) .setTemplateUris(TEST_TEMPLATE_URI).setProperties(new StringMap(flowProperties)); - FlowSpec runOnceSpec = FlowConfigResourceLocalHandler.createFlowSpecForConfig(runOnceFlowConfig); + FlowSpec runOnceSpec = FlowConfigsV2ResourceHandler.createFlowSpecForConfig(runOnceFlowConfig); // add the non compilable flow directly to the spec store skipping flow catalog which would not allow this this.gobblinServiceManager.getFlowCatalog().getSpecStore().addSpec(uncompilableSpec); @@ -377,7 +378,7 @@ public void testRestart() throws Exception { Assert.assertEquals(specs.size(), 0); } - @Test //(dependsOnMethods = "testRestart") + @Test (dependsOnMethods = "testRestart") public void testUncompilableJob() throws Exception { FlowId flowId = new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(MockedSpecCompiler.UNCOMPILABLE_FLOW); URI uri = FlowSpec.Utils.createFlowSpecUri(flowId); diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.java new file mode 100644 index 00000000000..b07ebe8f033 --- /dev/null +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gobblin.service.modules.orchestration; + +import java.util.concurrent.TimeUnit; + +import org.testng.Assert; +import org.testng.annotations.Test; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.service.ServiceConfigKeys; +import org.apache.gobblin.service.modules.flowgraph.Dag; +import org.apache.gobblin.service.modules.spec.JobExecutionPlan; + + +/** + * Tests the state updates (including updating in-memory state and MysqlDagActionStore) after performing add or cancel + * operations by calling addDag, stopDag, kill, and resume. It also tests flows with and without sla configs. + */ +public class DagFlowTest { + + @Test + void slaConfigCheck() throws Exception { + Dag dag = DagTestUtils.buildDag("5", 123456783L, "FINISH_RUNNING", 1); + Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), ServiceConfigKeys.DEFAULT_FLOW_FINISH_DEADLINE_MILLIS); + + Config jobConfig = dag.getStartNodes().get(0).getValue().getJobSpec().getConfig(); + jobConfig = jobConfig + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME, ConfigValueFactory.fromAnyRef("7")) + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME_UNIT, ConfigValueFactory.fromAnyRef(TimeUnit.SECONDS.name())); + dag.getStartNodes().get(0).getValue().getJobSpec().setConfig(jobConfig); + Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), TimeUnit.SECONDS.toMillis(7L)); + + jobConfig = jobConfig + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME, ConfigValueFactory.fromAnyRef("8")) + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME_UNIT, ConfigValueFactory.fromAnyRef(TimeUnit.MINUTES.name())); + dag.getStartNodes().get(0).getValue().getJobSpec().setConfig(jobConfig); + Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), TimeUnit.MINUTES.toMillis(8L)); + } +} 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 17915d9bf2e..9652e0faacf 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 @@ -83,10 +83,8 @@ public class OrchestratorTest { private ServiceBasedAppLauncher serviceLauncher; private TopologyCatalog topologyCatalog; private TopologySpec topologySpec; - private FlowCatalog flowCatalog; private FlowSpec flowSpec; - private ITestMetastoreDatabase testMetastoreDatabase; private Orchestrator dagMgrNotFlowLaunchHandlerBasedOrchestrator; @@ -119,7 +117,6 @@ public void setUp() throws Exception { this.flowCatalog = new FlowCatalog(ConfigUtils.propertiesToConfig(flowProperties), Optional.of(logger), Optional.absent(), true); this.serviceLauncher.addService(flowCatalog); - FlowStatusGenerator mockFlowStatusGenerator = mock(FlowStatusGenerator.class); MySqlDagManagementStateStore dagManagementStateStore = spy(MySqlDagManagementStateStoreTest.getDummyDMSS(this.testMetastoreDatabase)); @@ -127,7 +124,7 @@ public void setUp() throws Exception { SharedFlowMetricsSingleton sharedFlowMetricsSingleton = new SharedFlowMetricsSingleton(ConfigUtils.propertiesToConfig(orchestratorProperties)); FlowCompilationValidationHelper flowCompilationValidationHelper = new FlowCompilationValidationHelper(ConfigFactory.empty(), - sharedFlowMetricsSingleton, mock(UserQuotaManager.class), mockFlowStatusGenerator); + sharedFlowMetricsSingleton, mock(UserQuotaManager.class), mock(FlowStatusGenerator.class)); this.dagMgrNotFlowLaunchHandlerBasedOrchestrator = new Orchestrator(ConfigUtils.propertiesToConfig(orchestratorProperties), this.topologyCatalog, Optional.of(logger), mock(FlowLaunchHandler.class), sharedFlowMetricsSingleton, dagManagementStateStore, flowCompilationValidationHelper, mock(JobStatusRetriever.class)); diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/restli/FlowConfigUtilsTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/restli/FlowConfigUtilsTest.java index 5be7c5c53b8..94ab199ccc5 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/restli/FlowConfigUtilsTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/restli/FlowConfigUtilsTest.java @@ -29,7 +29,6 @@ import org.apache.gobblin.service.FlowConfig; import org.apache.gobblin.service.FlowConfigLoggedException; -import org.apache.gobblin.service.FlowConfigResourceLocalHandler; import org.apache.gobblin.service.FlowId; import org.apache.gobblin.service.Schedule; @@ -38,7 +37,7 @@ public class FlowConfigUtilsTest { private void testFlowSpec(FlowConfig flowConfig) { try { - FlowConfigResourceLocalHandler.createFlowSpecForConfig(flowConfig); + FlowConfigsV2ResourceHandler.createFlowSpecForConfig(flowConfig); } catch (FlowConfigLoggedException e) { Assert.fail("Should not get to here"); } @@ -155,7 +154,7 @@ public void testFlowConfigWithoutTemplateUri() { flowConfig.setProperties(new StringMap(Maps.fromProperties(properties))); try { - FlowConfigResourceLocalHandler.createFlowSpecForConfig(flowConfig); + FlowConfigsV2ResourceHandler.createFlowSpecForConfig(flowConfig); Assert.fail("Should not get to here"); } catch (RequiredFieldNotPresentException e) { Assert.assertTrue(true, "templateUri cannot be empty"); From cf2d4e8af7b077482affc6282ee0a91475e60b76 Mon Sep 17 00:00:00 2001 From: Arjun Singh Bora Date: Mon, 9 Sep 2024 15:24:09 -0700 Subject: [PATCH 2/4] address review comments --- ...gobblin.service.flowstatuses.restspec.json | 2 +- ...gobblin.service.flowstatuses.snapshot.json | 2 +- ...FlowExecutionResourceHandlerInterface.java | 2 +- ... => FlowConfigsV2ResourceHandlerTest.java} | 2 +- ... => FlowExecutionResourceHandlerTest.java} | 2 +- .../modules/core/GobblinServiceManager.java | 2 +- .../restli/FlowExecutionResourceHandler.java | 3 + ...Test.java => FlowExecutionClientTest.java} | 2 +- .../modules/orchestration/DagFlowTest.java | 58 ------------------- .../modules/orchestration/DagUtilsTest.java | 19 ++++++ 10 files changed, 29 insertions(+), 65 deletions(-) rename gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/{FlowConfigResourceLocalHandlerTest.java => FlowConfigsV2ResourceHandlerTest.java} (98%) rename gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/{FlowExecutionResourceLocalHandlerTest.java => FlowExecutionResourceHandlerTest.java} (97%) rename gobblin-service/src/test/java/org/apache/gobblin/service/{FlowExecutionTest.java => FlowExecutionClientTest.java} (99%) delete mode 100644 gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.java diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json index b6a0ee0eef8..f84720ab487 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/idl/org.apache.gobblin.service.flowstatuses.restspec.json @@ -6,7 +6,7 @@ "namespace" : "org.apache.gobblin.service", "path" : "/flowstatuses", "schema" : "org.apache.gobblin.service.FlowStatus", - "doc" : "Resource for handling flow status requests. Deprecated, use {@link FlowExecutionResource}\n\ngenerated from: org.apache.gobblin.service.FlowStatusResource", + "doc" : "Deprecated, use {@link FlowExecutionResource}\n\nResource for handling flow status requests. generated from: org.apache.gobblin.service.FlowStatusResource", "collection" : { "identifier" : { "name" : "id", diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json index e85d93df793..df4d02067a5 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json @@ -271,7 +271,7 @@ "namespace" : "org.apache.gobblin.service", "path" : "/flowstatuses", "schema" : "org.apache.gobblin.service.FlowStatus", - "doc" : "Resource for handling flow status requests. Deprecated, use {@link FlowExecutionResource}\n\ngenerated from: org.apache.gobblin.service.FlowStatusResource", + "doc" : "Deprecated, use {@link FlowExecutionResource}\n\nResource for handling flow status requests. generated from: org.apache.gobblin.service.FlowStatusResource", "collection" : { "identifier" : { "name" : "id", diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java index 48e3ac149d7..cc9b0e34b5f 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java @@ -26,7 +26,7 @@ // Unlike FlowConfigsV2ResourceHandler, this is an interface rather than a class because it's implementation needs -// classes from gobblin-service module, and adding gobblin-service as a dependency will cause circular dependency, +// classes from gobblin-service module, and adding gobblin-service as a dependency will cause circular dependency public interface FlowExecutionResourceHandlerInterface { /** * Get {@link FlowExecution} diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigResourceLocalHandlerTest.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigsV2ResourceHandlerTest.java similarity index 98% rename from gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigResourceLocalHandlerTest.java rename to gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigsV2ResourceHandlerTest.java index bfb24a2df6e..8675c6c9dc3 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigResourceLocalHandlerTest.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowConfigsV2ResourceHandlerTest.java @@ -32,7 +32,7 @@ import org.apache.gobblin.service.modules.restli.FlowConfigsV2ResourceHandler; -public class FlowConfigResourceLocalHandlerTest { +public class FlowConfigsV2ResourceHandlerTest { private static final String TEST_GROUP_NAME = "testGroup1"; private static final String TEST_FLOW_NAME = "testFlow1"; private static final String TEST_SCHEDULE = "0 1/0 * ? * *"; diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandlerTest.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceHandlerTest.java similarity index 97% rename from gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandlerTest.java rename to gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceHandlerTest.java index 8215135bf4d..0333a167b73 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceLocalHandlerTest.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/test/java/org/apache/gobblin/service/FlowExecutionResourceHandlerTest.java @@ -21,7 +21,7 @@ import org.testng.annotations.Test; -public class FlowExecutionResourceLocalHandlerTest { +public class FlowExecutionResourceHandlerTest { @Test public void testEstimateCopyTimeLeftSanityCheck() { diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java index d9141a01344..d36afdeeec4 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java @@ -280,7 +280,7 @@ private void registerServicesInLauncher(){ private void configureServices(){ if (configuration.isRestLIServerEnabled()) { this.restliServer = EmbeddedRestliServer.builder() - .resources(Lists.newArrayList(FlowConfigsV2Resource.class, FlowConfigsV2Resource.class)) + .resources(Lists.newArrayList(FlowConfigsV2Resource.class)) .injector(injector) .build(); diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java index 85bb3851e84..8dc2802dc55 100644 --- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java +++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/restli/FlowExecutionResourceHandler.java @@ -45,6 +45,9 @@ import org.apache.gobblin.service.monitoring.FlowStatusGenerator; +/** + * This is associated with {@link FlowExecutionResource} and handles all the requests FlowExecutionResource get. + */ @Slf4j public class FlowExecutionResourceHandler implements FlowExecutionResourceHandlerInterface { private final DagManagementStateStore dagManagementStateStore; diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionClientTest.java similarity index 99% rename from gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionTest.java rename to gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionClientTest.java index 15380be091c..c6d88591bba 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/FlowExecutionClientTest.java @@ -46,7 +46,7 @@ @Test(groups = { "gobblin.service" }, singleThreaded = true) -public class FlowExecutionTest { +public class FlowExecutionClientTest { private FlowExecutionClient client; private EmbeddedRestliServer _server; private List> _listOfJobStatusLists; diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.java deleted file mode 100644 index b07ebe8f033..00000000000 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagFlowTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.gobblin.service.modules.orchestration; - -import java.util.concurrent.TimeUnit; - -import org.testng.Assert; -import org.testng.annotations.Test; - -import com.typesafe.config.Config; -import com.typesafe.config.ConfigValueFactory; - -import org.apache.gobblin.configuration.ConfigurationKeys; -import org.apache.gobblin.service.ServiceConfigKeys; -import org.apache.gobblin.service.modules.flowgraph.Dag; -import org.apache.gobblin.service.modules.spec.JobExecutionPlan; - - -/** - * Tests the state updates (including updating in-memory state and MysqlDagActionStore) after performing add or cancel - * operations by calling addDag, stopDag, kill, and resume. It also tests flows with and without sla configs. - */ -public class DagFlowTest { - - @Test - void slaConfigCheck() throws Exception { - Dag dag = DagTestUtils.buildDag("5", 123456783L, "FINISH_RUNNING", 1); - Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), ServiceConfigKeys.DEFAULT_FLOW_FINISH_DEADLINE_MILLIS); - - Config jobConfig = dag.getStartNodes().get(0).getValue().getJobSpec().getConfig(); - jobConfig = jobConfig - .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME, ConfigValueFactory.fromAnyRef("7")) - .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME_UNIT, ConfigValueFactory.fromAnyRef(TimeUnit.SECONDS.name())); - dag.getStartNodes().get(0).getValue().getJobSpec().setConfig(jobConfig); - Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), TimeUnit.SECONDS.toMillis(7L)); - - jobConfig = jobConfig - .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME, ConfigValueFactory.fromAnyRef("8")) - .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME_UNIT, ConfigValueFactory.fromAnyRef(TimeUnit.MINUTES.name())); - dag.getStartNodes().get(0).getValue().getJobSpec().setConfig(jobConfig); - Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), TimeUnit.MINUTES.toMillis(8L)); - } -} diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagUtilsTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagUtilsTest.java index 4d710fb79f2..fe307c35b4d 100644 --- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagUtilsTest.java +++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/DagUtilsTest.java @@ -63,6 +63,25 @@ public class DagUtilsTest { .withValue(ConfigurationKeys.SPECEXECUTOR_INSTANCE_URI_KEY, ConfigValueFactory.fromAnyRef( MySqlDagManagementStateStoreTest.TEST_SPEC_EXECUTOR_URI)); + @Test + void slaConfigCheck() throws Exception { + Dag dag = DagTestUtils.buildDag("5", 123456783L, "FINISH_RUNNING", 1); + Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), ServiceConfigKeys.DEFAULT_FLOW_FINISH_DEADLINE_MILLIS); + + Config jobConfig = dag.getStartNodes().get(0).getValue().getJobSpec().getConfig(); + jobConfig = jobConfig + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME, ConfigValueFactory.fromAnyRef("7")) + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME_UNIT, ConfigValueFactory.fromAnyRef(TimeUnit.SECONDS.name())); + dag.getStartNodes().get(0).getValue().getJobSpec().setConfig(jobConfig); + Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), TimeUnit.SECONDS.toMillis(7L)); + + Config jobConfig2 = jobConfig + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME, ConfigValueFactory.fromAnyRef("8")) + .withValue(ConfigurationKeys.GOBBLIN_FLOW_FINISH_DEADLINE_TIME_UNIT, ConfigValueFactory.fromAnyRef(TimeUnit.MINUTES.name())); + dag.getStartNodes().get(0).getValue().getJobSpec().setConfig(jobConfig2); + Assert.assertEquals(DagUtils.getFlowFinishDeadline(dag.getStartNodes().get(0)), TimeUnit.MINUTES.toMillis(8L)); + } + @Test void deadlineConfigCheck() throws Exception { Dag dag = DagTestUtils.buildDag("5", 123456783L, "FINISH_RUNNING", 1); From 6ef073ab353d34a7b81b84dcf58a1fc0ec7f1f40 Mon Sep 17 00:00:00 2001 From: Arjun Singh Bora Date: Mon, 9 Sep 2024 17:40:19 -0700 Subject: [PATCH 3/4] address review comments --- .../FlowConfigsResourceHandlerInterface.java | 56 +++++++++++++++++++ ...FlowExecutionResourceHandlerInterface.java | 4 +- .../restli/FlowConfigsV2ResourceHandler.java | 9 ++- 3 files changed, 62 insertions(+), 7 deletions(-) create mode 100644 gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java new file mode 100644 index 00000000000..cd6fbff36ec --- /dev/null +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java @@ -0,0 +1,56 @@ +/* + * 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; + +import java.util.Collection; +import java.util.Properties; + +import com.linkedin.restli.common.ComplexResourceKey; +import com.linkedin.restli.common.PatchRequest; +import com.linkedin.restli.server.CreateKVResponse; +import com.linkedin.restli.server.UpdateResponse; + +import org.apache.gobblin.runtime.api.FlowSpecSearchObject; + + +// This is an interface rather than a class because implementation may need resources from the packages it cannot have +// direct dependency on +public interface FlowConfigsResourceHandlerInterface { + + FlowConfig getFlowConfig(FlowId flowId) throws FlowConfigLoggedException; + + Collection getFlowConfig(FlowSpecSearchObject flowSpecSearchObject) throws FlowConfigLoggedException; + + /** + * Get all flow configs + */ + Collection getAllFlowConfigs(); + + /** + * Get all flow configs in between start and start + count - 1 + */ + Collection getAllFlowConfigs(int start, int count); + + UpdateResponse deleteFlowConfig(FlowId flowId, Properties header) throws FlowConfigLoggedException; + + UpdateResponse partialUpdateFlowConfig(FlowId flowId, PatchRequest flowConfigPatch) throws FlowConfigLoggedException; + + UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig) throws FlowConfigLoggedException; + + CreateKVResponse, FlowConfig> createFlowConfig(FlowConfig flowConfig) throws FlowConfigLoggedException; +} diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java index cc9b0e34b5f..971c02b6407 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java @@ -25,8 +25,8 @@ import com.linkedin.restli.server.UpdateResponse; -// Unlike FlowConfigsV2ResourceHandler, this is an interface rather than a class because it's implementation needs -// classes from gobblin-service module, and adding gobblin-service as a dependency will cause circular dependency +// This is an interface rather than a class because implementation may need resources from the packages it cannot have +// direct dependency on public interface FlowExecutionResourceHandlerInterface { /** * Get {@link FlowExecution} diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/modules/restli/FlowConfigsV2ResourceHandler.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/modules/restli/FlowConfigsV2ResourceHandler.java index 3b3869f4357..927909e57dd 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/modules/restli/FlowConfigsV2ResourceHandler.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/modules/restli/FlowConfigsV2ResourceHandler.java @@ -66,6 +66,7 @@ import org.apache.gobblin.runtime.util.InjectionNames; import org.apache.gobblin.service.FlowConfig; import org.apache.gobblin.service.FlowConfigLoggedException; +import org.apache.gobblin.service.FlowConfigsResourceHandlerInterface; import org.apache.gobblin.service.FlowId; import org.apache.gobblin.service.FlowStatusId; import org.apache.gobblin.service.RequesterService; @@ -75,7 +76,7 @@ @Slf4j -public class FlowConfigsV2ResourceHandler { +public class FlowConfigsV2ResourceHandler implements FlowConfigsResourceHandlerInterface { @Getter private String serviceName; @@ -164,16 +165,14 @@ public UpdateResponse partialUpdateFlowConfig(FlowId flowId, return updateFlowConfig(flowId, flowConfig, modifiedWatermark); } - public UpdateResponse updateFlowConfig(FlowId flowId, - FlowConfig flowConfig) throws FlowConfigLoggedException { + public UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig) throws FlowConfigLoggedException { // We have modifiedWatermark here to avoid update config happens at the same time on different hosts overwrite each other // timestamp here will be treated as largest modifiedWatermark that we can update long version = System.currentTimeMillis() / 1000; return updateFlowConfig(flowId, flowConfig, version); } - public UpdateResponse updateFlowConfig(FlowId flowId, - FlowConfig flowConfig, long modifiedWatermark) throws FlowConfigLoggedException { + private UpdateResponse updateFlowConfig(FlowId flowId, FlowConfig flowConfig, long modifiedWatermark) throws FlowConfigLoggedException { String flowName = flowId.getFlowName(); String flowGroup = flowId.getFlowGroup(); From 6be4fe11b1cc8734645f6d1a6d10940315736d75 Mon Sep 17 00:00:00 2001 From: Arjun Singh Bora Date: Tue, 10 Sep 2024 13:20:41 -0700 Subject: [PATCH 4/4] address review comments --- .../service/FlowConfigsResourceHandlerInterface.java | 3 +++ .../service/FlowExecutionResourceHandlerInterface.java | 4 ++++ .../service/monitoring/DagActionStoreChangeMonitor.java | 7 +++++-- .../gobblin/service/monitoring/SpecStoreChangeMonitor.java | 5 ++--- 4 files changed, 14 insertions(+), 5 deletions(-) diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java index cd6fbff36ec..dd1b8c77230 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsResourceHandlerInterface.java @@ -30,6 +30,9 @@ // This is an interface rather than a class because implementation may need resources from the packages it cannot have // direct dependency on +/** + * It is closely coupled with {@link FlowConfigsV2Resource} and handle all the requests it get. + */ public interface FlowConfigsResourceHandlerInterface { FlowConfig getFlowConfig(FlowId flowId) throws FlowConfigLoggedException; diff --git a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java index 971c02b6407..a267f9ab4ea 100644 --- a/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java +++ b/gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResourceHandlerInterface.java @@ -27,6 +27,10 @@ // This is an interface rather than a class because implementation may need resources from the packages it cannot have // direct dependency on + +/** + * It is closely coupled with {@link FlowExecutionResource} and handle all the requests it get. + */ public interface FlowExecutionResourceHandlerInterface { /** * Get {@link FlowExecution} 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 44005afa752..5b35b0bf7d4 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 @@ -45,6 +45,7 @@ import org.apache.gobblin.runtime.metrics.RuntimeMetrics; import org.apache.gobblin.runtime.spec_catalog.FlowCatalog; import org.apache.gobblin.service.modules.orchestration.DagActionStore; +import org.apache.gobblin.service.modules.orchestration.DagManagement; import org.apache.gobblin.service.modules.orchestration.DagManagementStateStore; import org.apache.gobblin.service.modules.orchestration.Orchestrator; import org.apache.gobblin.service.modules.orchestration.task.DagProcessingEngineMetrics; @@ -112,8 +113,8 @@ Metrics need to be created before initializeMonitor() below is called (or more s @Override protected void assignTopicPartitions() { - // Expects underlying consumer to handle initializing partitions and offset for the topic - - // subscribe to all partitions from latest offset + // This implementation expects underlying consumer (HighLevelConsumer::GobblinKafkaConsumerClient) to handle + // initializing partitions and offset for the topic. It should subscribe to all partitions from latest offset } /** @@ -138,6 +139,8 @@ protected void initializeMonitor() { this.unexpectedErrors.mark(); } } + executorService.shutdown(); + try { boolean executedSuccessfully = executorService.awaitTermination(ConfigUtils.getInt(this.config, ConfigurationKeys.DAG_ACTION_STORE_MONITOR_EXECUTOR_TIMEOUT_SECONDS, 30), TimeUnit.SECONDS); 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 29111dbfb98..5c183ea326d 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 @@ -93,9 +93,8 @@ public SpecStoreChangeMonitor(String topic, Config config, FlowCatalog flowCatal @Override protected void assignTopicPartitions() { - // Expects underlying consumer to handle initializing partitions and offset for the topic - - // subscribe to all partitions from latest offset - return; + // This implementation expects underlying consumer (HighLevelConsumer::GobblinKafkaConsumerClient) to handle + // initializing partitions and offset for the topic. It should subscribe to all partitions from latest offset } /*