You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by ab...@apache.org on 2017/09/12 09:30:01 UTC
[1/4] incubator-gobblin git commit: [GOBBLIN-3] Multi-hop flow
compiler implementation
Repository: incubator-gobblin
Updated Branches:
refs/heads/master ea5047ea2 -> 9402a9037
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java
new file mode 100644
index 0000000..456f3a3
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java
@@ -0,0 +1,313 @@
+/*
+ * 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.flow;
+
+import com.google.common.base.Splitter;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
+import com.typesafe.config.Config;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor;
+import org.apache.gobblin.service.modules.policy.ServicePolicy;
+import org.apache.gobblin.util.ClassAliasResolver;
+import org.jgrapht.graph.DirectedWeightedMultigraph;
+import org.slf4j.Logger;
+import org.apache.gobblin.runtime.api.FlowEdge;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.apache.gobblin.runtime.api.FlowSpec;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.TopologySpec;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.runtime.spec_executorInstance.BaseServiceNodeImpl;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.JobTemplate;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecNotFoundException;
+import org.apache.gobblin.runtime.job_spec.ResolvedJobSpec;
+
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import static org.apache.gobblin.service.ServiceConfigKeys.*;
+import static org.apache.gobblin.service.modules.utils.FindPathUtils.*;
+
+// Users are capable to inject hints/prioritization into route selection, in two forms:
+// 1. PolicyBasedBlockedConnection: Define some undesired routes
+// 2. Specified a complete path. FlowCompiler is responsible to verify if the path given is valid.
+
+// TODO: Flow monitoring, injecting weight for flowEdge:ETL-6213
+@Slf4j
+public class MultiHopsFlowToJobSpecCompiler extends BaseFlowToJobSpecCompiler {
+
+ private static final Splitter SPLIT_BY_COMMA = Splitter.on(",").omitEmptyStrings().trimResults();
+
+ @Getter
+ private DirectedWeightedMultigraph<ServiceNode, FlowEdge> weightedGraph =
+ new DirectedWeightedMultigraph<>(LoadBasedFlowEdgeImpl.class);
+
+ public ServicePolicy servicePolicy;
+
+ // Contains user-specified complete path of how the data movement is executed from source to sink.
+ private Optional<String> optionalUserSpecifiedPath;
+
+ private FlowEdgeProps defaultFlowEdgeProps = new FlowEdgeProps();
+
+ public MultiHopsFlowToJobSpecCompiler(Config config) {
+ this(config, Optional.absent(), true);
+ }
+
+ public MultiHopsFlowToJobSpecCompiler(Config config, Optional<Logger> log) {
+ this(config, log, true);
+ }
+
+ public MultiHopsFlowToJobSpecCompiler(Config config, Optional<Logger> log, boolean instrumentationEnabled) {
+ super(config, log, instrumentationEnabled);
+ String policyClassName = config.hasPath(SERVICE_POLICY_NAME) ? config.getString(SERVICE_POLICY_NAME)
+ : ServiceConfigKeys.DEFAULT_SERVICE_POLICY;
+ ClassAliasResolver<ServicePolicy> classResolver = new ClassAliasResolver<>(ServicePolicy.class);
+ try {
+ servicePolicy = classResolver.resolveClass(policyClassName).newInstance();
+ } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
+ throw new RuntimeException("Error happen when resolving class for :" + policyClassName, e);
+ }
+
+ if (config.hasPath(ServiceConfigKeys.POLICY_BASED_BLOCKED_CONNECTION)
+ && config.getStringList(ServiceConfigKeys.POLICY_BASED_BLOCKED_CONNECTION).size() > 0) {
+ try {
+ for (String sourceSinkPair : config.getStringList(ServiceConfigKeys.POLICY_BASED_BLOCKED_CONNECTION)) {
+ BaseServiceNodeImpl source = new BaseServiceNodeImpl(sourceSinkPair.split(":")[0]);
+ BaseServiceNodeImpl sink = new BaseServiceNodeImpl(sourceSinkPair.split(":")[1]);
+ URI specExecutorURI = new URI(sourceSinkPair.split(":")[2]);
+ servicePolicy.addFlowEdge(
+ new LoadBasedFlowEdgeImpl(source, sink, InMemorySpecExecutor.createDummySpecExecutor(specExecutorURI)));
+ }
+ } catch (URISyntaxException e) {
+ this.log.warn("Constructing of FlowEdge in ServicePolicy Failed");
+ }
+ }
+
+ if (config.hasPath(ServiceConfigKeys.POLICY_BASED_BLOCKED_NODES) &&
+ StringUtils.isNotBlank(config.getString(ServiceConfigKeys.POLICY_BASED_BLOCKED_NODES))) {
+ for (String blacklistedNode : SPLIT_BY_COMMA.splitToList(
+ config.getString(ServiceConfigKeys.POLICY_BASED_BLOCKED_NODES))) {
+ servicePolicy.addServiceNode(new BaseServiceNodeImpl(blacklistedNode));
+ }
+ }
+
+ if (config.hasPath(ServiceConfigKeys.POLICY_BASED_DATA_MOVEMENT_PATH) && StringUtils.isNotBlank(
+ config.getString(ServiceConfigKeys.POLICY_BASED_DATA_MOVEMENT_PATH))) {
+ optionalUserSpecifiedPath = Optional.of(config.getString(ServiceConfigKeys.POLICY_BASED_DATA_MOVEMENT_PATH));
+ } else {
+ optionalUserSpecifiedPath = Optional.absent();
+ }
+ }
+
+ @Override
+ public Map<Spec, SpecExecutor> compileFlow(Spec spec) {
+ // A Map from JobSpec to SpexExecutor, as the output of Flow Compiler.
+ Map<Spec, SpecExecutor> specExecutorInstanceMap = Maps.newLinkedHashMap();
+ findPath(specExecutorInstanceMap, spec);
+ return specExecutorInstanceMap;
+ }
+
+ /**
+ * @return Transform a set of {@link TopologySpec} into a instance of {@link org.jgrapht.graph.WeightedMultigraph}
+ * and filter out connections between blacklisted vertices that user specified.
+ * The output of this function only stays in memory, so each time a logical flow is compiled, the multigraph will
+ * be re-calculated.
+ *
+ */
+ private void inMemoryWeightGraphGenerator() {
+ for (TopologySpec topologySpec : topologySpecMap.values()) {
+ weightGraphGenerateHelper(topologySpec);
+ }
+
+ // Filter out connection appearing in servicePolicy.
+ // This is where servicePolicy is enforced.
+ servicePolicy.populateBlackListedEdges(this.weightedGraph);
+ if (servicePolicy.getBlacklistedEdges().size() > 0) {
+ for (FlowEdge toDeletedEdge : servicePolicy.getBlacklistedEdges()) {
+ weightedGraph.removeEdge(toDeletedEdge);
+ }
+ }
+ }
+
+ // Basically a dijkstra path finding for connecting source and sink by multiple hops in between.
+ // If there's any user-specified prioritization, conduct the DFS and see if the user-specified path is available.
+
+ // there's no updates on TopologySpec, or user should be aware of the possibility
+ // that a topologySpec not being reflected in findPath.
+ private void findPath(Map<Spec, SpecExecutor> specExecutorInstanceMap, Spec spec) {
+ inMemoryWeightGraphGenerator();
+ FlowSpec flowSpec = (FlowSpec) spec;
+ if (optionalUserSpecifiedPath.isPresent()) {
+ log.info("Starting to evaluate user's specified path ... ");
+ if (userSpecifiedPathVerificator(specExecutorInstanceMap, flowSpec)) {
+ log.info("User specified path[ " + optionalUserSpecifiedPath.get() + "] successfully verified.");
+ return;
+ } else {
+ log.error("Will not execute user specified path[ " + optionalUserSpecifiedPath.get() + "]");
+ log.info("Start to execute FlowCompiler's algorithm for valid data movement path");
+ }
+ }
+
+ ServiceNode sourceNode =
+ new BaseServiceNodeImpl(flowSpec.getConfig().getString(ServiceConfigKeys.FLOW_SOURCE_IDENTIFIER_KEY));
+
+ ServiceNode targetNode =
+ new BaseServiceNodeImpl(flowSpec.getConfig().getString(ServiceConfigKeys.FLOW_DESTINATION_IDENTIFIER_KEY));
+
+ List<FlowEdge> resultEdgePath = dijkstraBasedPathFindingHelper(sourceNode, targetNode, this.weightedGraph);
+ for (int i = 0; i < resultEdgePath.size() ; i++) {
+ FlowEdge tmpFlowEdge = resultEdgePath.get(i);
+ ServiceNode edgeSrcNode = ((LoadBasedFlowEdgeImpl) tmpFlowEdge).getSourceNode();
+ ServiceNode edgeTgtNode = ((LoadBasedFlowEdgeImpl) tmpFlowEdge).getTargetNode();
+ specExecutorInstanceMap.put(jobSpecGenerator(edgeSrcNode, edgeTgtNode, flowSpec),
+ ((LoadBasedFlowEdgeImpl) (resultEdgePath.get(i))).getSpecExecutorInstance());
+ }
+ }
+
+ /**
+ * As the base implementation here, all templates will be considered for each edge.
+ */
+ @Override
+ protected void populateEdgeTemplateMap() {
+ if (templateCatalog.isPresent()) {
+ for (FlowEdge flowEdge : this.weightedGraph.edgeSet()) {
+ edgeTemplateMap.put(flowEdge.getEdgeIdentity(), templateCatalog.get().
+ getAllTemplates().
+ stream().map(jobTemplate -> jobTemplate.getUri()).collect(Collectors.toList()));
+ }
+ }
+ }
+
+ // If path specified not existed, return false;
+ // else return true.
+ private boolean userSpecifiedPathVerificator(Map<Spec, SpecExecutor> specExecutorInstanceMap, FlowSpec flowSpec) {
+ Map<Spec, SpecExecutor> tmpSpecExecutorInstanceMap = new HashMap<>();
+ List<String> userSpecfiedPath = Arrays.asList(optionalUserSpecifiedPath.get().split(","));
+ for (int i = 0; i < userSpecfiedPath.size() - 1; i++) {
+ ServiceNode sourceNode = new BaseServiceNodeImpl(userSpecfiedPath.get(i));
+ ServiceNode targetNode = new BaseServiceNodeImpl(userSpecfiedPath.get(i + 1));
+ if (weightedGraph.containsVertex(sourceNode) && weightedGraph.containsVertex(targetNode)
+ && weightedGraph.containsEdge(sourceNode, targetNode)) {
+ tmpSpecExecutorInstanceMap.put(jobSpecGenerator(sourceNode, targetNode, flowSpec),
+ (((LoadBasedFlowEdgeImpl) weightedGraph.getEdge(sourceNode, targetNode)).getSpecExecutorInstance()));
+ } else {
+ log.error("User Specified Path is invalid");
+ return false;
+ }
+ }
+ specExecutorInstanceMap.putAll(tmpSpecExecutorInstanceMap);
+ return true;
+ }
+
+ // Helper function for transform TopologySpecMap into a weightedDirectedGraph.
+ private void weightGraphGenerateHelper(TopologySpec topologySpec) {
+ try {
+ Map<ServiceNode, ServiceNode> capabilities = topologySpec.getSpecExecutor().getCapabilities().get();
+ for (Map.Entry<ServiceNode, ServiceNode> capability : capabilities.entrySet()) {
+
+ BaseServiceNodeImpl sourceNode = new BaseServiceNodeImpl(capability.getKey().getNodeName());
+ BaseServiceNodeImpl targetNode = new BaseServiceNodeImpl(capability.getValue().getNodeName());
+
+ if (!weightedGraph.containsVertex(sourceNode)) {
+ weightedGraph.addVertex(sourceNode);
+ }
+ if (!weightedGraph.containsVertex(targetNode)) {
+ weightedGraph.addVertex(targetNode);
+ }
+
+ FlowEdge flowEdge =
+ new LoadBasedFlowEdgeImpl(sourceNode, targetNode, defaultFlowEdgeProps, topologySpec.getSpecExecutor());
+
+ // In Multi-Graph if flowEdge existed, just skip it.
+ if (!weightedGraph.containsEdge(flowEdge)) {
+ weightedGraph.addEdge(sourceNode, targetNode, flowEdge);
+ }
+ }
+ } catch (InterruptedException | ExecutionException e) {
+ Instrumented.markMeter(this.flowCompilationFailedMeter);
+ throw new RuntimeException("Cannot determine topology capabilities", e);
+ }
+ }
+
+ /**
+ * Generate JobSpec based on the #templateURI that user specified.
+ */
+ private JobSpec jobSpecGenerator(ServiceNode sourceNode, ServiceNode targetNode, FlowEdge flowEdge, URI templateURI,
+ FlowSpec flowSpec) {
+ JobSpec jobSpec;
+ JobSpec.Builder jobSpecBuilder = JobSpec.builder(jobSpecURIGenerator(flowSpec, sourceNode, targetNode))
+ .withConfig(flowSpec.getConfig())
+ .withDescription(flowSpec.getDescription())
+ .withVersion(flowSpec.getVersion());
+ if (edgeTemplateMap.containsKey(flowEdge.getEdgeIdentity()) && edgeTemplateMap.get(flowEdge.getEdgeIdentity())
+ .contains(templateURI)) {
+ jobSpecBuilder.withTemplate(templateURI);
+ try {
+ jobSpec = new ResolvedJobSpec(jobSpecBuilder.build(), templateCatalog.get());
+ log.info("Resolved JobSpec properties are: " + jobSpec.getConfigAsProperties());
+ } catch (SpecNotFoundException | JobTemplate.TemplateException e) {
+ throw new RuntimeException("Could not resolve template in JobSpec from TemplateCatalog", e);
+ }
+ } else {
+ jobSpec = jobSpecBuilder.build();
+ log.info("Unresolved JobSpec properties are: " + jobSpec.getConfigAsProperties());
+ }
+ return jobSpec;
+ }
+
+ /**
+ * A naive implementation of resolving templates in each JobSpec among Multi-hop FlowSpec.
+ * Handle the case when edge is not specified.
+ * Always select the first available template.
+ */
+ private JobSpec jobSpecGenerator(ServiceNode sourceNode, ServiceNode targetNode, FlowSpec flowSpec) {
+ FlowEdge flowEdge = weightedGraph.getAllEdges(sourceNode, targetNode).iterator().next();
+ URI firstTemplateURI =
+ (edgeTemplateMap != null && edgeTemplateMap.containsKey(flowEdge.getEdgeIdentity())) ? edgeTemplateMap.get(
+ flowEdge.getEdgeIdentity()).get(0) : jobSpecGenerator(flowSpec).getUri();
+ return this.jobSpecGenerator(sourceNode, targetNode, flowEdge, firstTemplateURI, flowSpec);
+ }
+
+ /**
+ * A naive implementation of generating a jobSpec's URI within a multi-hop logical Flow.
+ */
+ public static URI jobSpecURIGenerator(FlowSpec flowSpec, ServiceNode sourceNode, ServiceNode targetNode) {
+ try {
+ return new URI(flowSpec.getUri().getScheme(), flowSpec.getUri().getAuthority(),
+ "/" + sourceNode.getNodeName() + "-" + targetNode.getNodeName(), null);
+ } catch (URISyntaxException e) {
+ log.error(
+ "URI construction failed when jobSpec from " + sourceNode.getNodeName() + " to " + targetNode.getNodeName());
+ throw new RuntimeException();
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java
index 65deeec..261ce6e 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/Orchestrator.java
@@ -22,12 +22,8 @@ import java.net.URI;
import java.util.Collections;
import java.util.List;
import java.util.Map;
-
import java.util.concurrent.TimeUnit;
import javax.annotation.Nonnull;
-import lombok.Getter;
-import org.apache.commons.lang3.reflect.ConstructorUtils;
-import org.slf4j.Logger;
import com.codahale.metrics.Meter;
import com.codahale.metrics.Timer;
@@ -40,10 +36,8 @@ import org.apache.gobblin.instrumented.Instrumented;
import org.apache.gobblin.instrumented.Instrumentable;
import org.apache.gobblin.metrics.MetricContext;
import org.apache.gobblin.metrics.Tag;
-
import org.apache.gobblin.runtime.api.FlowSpec;
import org.apache.gobblin.runtime.api.SpecCompiler;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
import org.apache.gobblin.runtime.api.TopologySpec;
import org.apache.gobblin.runtime.api.Spec;
import org.apache.gobblin.runtime.api.SpecCatalogListener;
@@ -54,6 +48,13 @@ import org.apache.gobblin.service.modules.flow.IdentityFlowToJobSpecCompiler;
import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ConfigUtils;
import org.slf4j.LoggerFactory;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.commons.lang3.reflect.ConstructorUtils;
+import org.apache.gobblin.configuration.State;
+import org.slf4j.Logger;
+
+import lombok.Getter;
/**
@@ -179,7 +180,7 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable {
public void orchestrate(Spec spec) throws Exception {
long startTime = System.nanoTime();
if (spec instanceof FlowSpec) {
- Map<Spec, SpecExecutorInstanceProducer> specExecutorInstanceMap = specCompiler.compileFlow(spec);
+ Map<Spec, SpecExecutor> specExecutorInstanceMap = specCompiler.compileFlow(spec);
if (specExecutorInstanceMap.isEmpty()) {
_log.warn("Cannot determine an executor to run on for Spec: " + spec);
@@ -187,18 +188,18 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable {
}
// Schedule all compiled JobSpecs on their respective Executor
- for (Map.Entry<Spec, SpecExecutorInstanceProducer> specsToExecute : specExecutorInstanceMap.entrySet()) {
+ for (Map.Entry<Spec, SpecExecutor> specsToExecute : specExecutorInstanceMap.entrySet()) {
// Run this spec on selected executor
- SpecExecutorInstanceProducer producer = null;
+ SpecProducer producer = null;
try {
- producer = specsToExecute.getValue();
+ producer = specsToExecute.getValue().getProducer().get();
Spec jobSpec = specsToExecute.getKey();
_log.info(String.format("Going to orchestrate JobSpc: %s on Executor: %s", jobSpec, producer));
producer.addSpec(jobSpec);
} catch(Exception e) {
_log.error("Cannot successfully setup spec: " + specsToExecute.getKey() + " on executor: " + producer +
- " for flow: " + spec, e);
+ " for flow: " + spec, e);
}
}
} else {
@@ -221,7 +222,7 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable {
}
@Override
- public List<Tag<?>> generateTags(org.apache.gobblin.configuration.State state) {
+ public List<Tag<?>> generateTags(State state) {
return Collections.emptyList();
}
@@ -234,4 +235,4 @@ public class Orchestrator implements SpecCatalogListener, Instrumentable {
public void switchMetricContext(MetricContext context) {
throw new UnsupportedOperationException();
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/ServicePolicy.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/ServicePolicy.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/ServicePolicy.java
new file mode 100644
index 0000000..540b13d
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/ServicePolicy.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.policy;
+
+import java.util.Set;
+import org.apache.gobblin.runtime.api.FlowEdge;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.jgrapht.graph.DirectedWeightedMultigraph;
+
+
+/**
+ * ServicePolicy will be firstly checked before the compilation happen.
+ * unexpcted edges will not be considered in compilation process.
+ */
+public interface ServicePolicy {
+
+ /**
+ * After initialization of {@link ServicePolicy}, the populating method need to invoked before
+ * {@link #getBlacklistedEdges()} can return the expected result.
+ *
+ * This requirement exists because when {@link ServicePolicy} is initialized it is not necessary that a
+ * {@link org.jgrapht.graph.WeightedMultigraph} has been constructed, neither we cannot know if user-specified edges
+ * and nodes exist in {@link org.jgrapht.graph.WeightedMultigraph}.
+ * The population of blacklisted Edges make sense after graph has been constructed.
+ */
+ public void populateBlackListedEdges(DirectedWeightedMultigraph<ServiceNode, FlowEdge> graph);
+
+ /**
+ * Should return all edges that being blacklisted by this policy.
+ */
+ public Set<FlowEdge> getBlacklistedEdges();
+
+ public void addServiceNode(ServiceNode serviceNode);
+
+ public void addFlowEdge(FlowEdge flowEdge);
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/StaticServicePolicy.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/StaticServicePolicy.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/StaticServicePolicy.java
new file mode 100644
index 0000000..aabc67e
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/policy/StaticServicePolicy.java
@@ -0,0 +1,98 @@
+/*
+ * 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.policy;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.gobblin.annotation.Alias;
+import org.apache.gobblin.runtime.api.FlowEdge;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.jgrapht.graph.DirectedWeightedMultigraph;
+
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * Defines {@link ServiceNode}s or {@link FlowEdge}s that should be blacklisted from the Flow-compiler process,
+ * obtained only from configuration, which is the reason it is named as static.
+ *
+ * TODO: DynamicServicePolicy can obtain new blacklist candidate through Flow monitoring process, which is responsible
+ * for monitoring the flow execution and react accordingly.
+ *
+ * Either user specify {@link ServiceNode} or {@link FlowEdge} to blacklist will all end up with a list of
+ * {@link FlowEdge}s that won't be considered when selecting path for data transformation.
+ */
+@Slf4j
+@Alias("static")
+public class StaticServicePolicy implements ServicePolicy {
+
+ @Getter
+ Set<FlowEdge> blacklistedEdges;
+
+ List<ServiceNode> serviceNodes;
+ List<FlowEdge> flowEdges;
+
+ public StaticServicePolicy() {
+ serviceNodes = new ArrayList<>();
+ flowEdges = new ArrayList<>();
+ blacklistedEdges = new HashSet<>();
+ }
+
+ public StaticServicePolicy(List<ServiceNode> serviceNodes, List<FlowEdge> flowEdges) {
+ Preconditions.checkNotNull(serviceNodes);
+ Preconditions.checkNotNull(flowEdges);
+ blacklistedEdges = new HashSet<>();
+ this.serviceNodes = serviceNodes;
+ this.flowEdges = flowEdges;
+ }
+
+ public void addServiceNode(ServiceNode serviceNode) {
+ this.serviceNodes.add(serviceNode);
+ }
+
+ public void addFlowEdge(FlowEdge flowEdge){
+ this.flowEdges.add(flowEdge);
+ }
+
+ @Override
+ public void populateBlackListedEdges(DirectedWeightedMultigraph<ServiceNode, FlowEdge> graph) {
+ for (ServiceNode node: serviceNodes) {
+ if (graph.containsVertex(node)) {
+ blacklistedEdges.addAll(graph.incomingEdgesOf(node));
+ blacklistedEdges.addAll(graph.outgoingEdgesOf(node));
+ } else {
+ log.info("The graph " + graph + " doesn't contains node " + node.toString());
+ }
+ }
+
+ for( FlowEdge flowEdge: flowEdges) {
+ if (graph.containsEdge(flowEdge)) {
+ blacklistedEdges.add(flowEdge);
+ } else {
+ log.info("The graph " + graph + "doesn't contains edge " + flowEdge.toString());
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
index d223f90..a625f36 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
@@ -53,7 +53,7 @@ import org.apache.gobblin.runtime.spec_catalog.TopologyCatalog;
import org.apache.gobblin.scheduler.BaseGobblinJob;
import org.apache.gobblin.scheduler.JobScheduler;
import org.apache.gobblin.scheduler.SchedulerService;
-import org.apache.gobblin.service.HelixUtils;
+import org.apache.gobblin.service.modules.utils.HelixUtils;
import org.apache.gobblin.service.ServiceConfigKeys;
import org.apache.gobblin.service.modules.orchestration.Orchestrator;
import org.apache.gobblin.util.ConfigUtils;
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactory.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactory.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactory.java
index 1a9b8f6..fb7c1b0 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactory.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactory.java
@@ -32,11 +32,12 @@ import com.google.common.collect.Lists;
import com.typesafe.config.Config;
import org.apache.gobblin.annotation.Alpha;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
import org.apache.gobblin.runtime.api.TopologySpec;
import org.apache.gobblin.service.ServiceConfigKeys;
import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+
@Alpha
@@ -45,7 +46,7 @@ public class ConfigBasedTopologySpecFactory implements TopologySpecFactory {
private static final Splitter SPLIT_BY_COMMA = Splitter.on(",").omitEmptyStrings().trimResults();
private final Config _config;
private final Logger _log;
- private final ClassAliasResolver<SpecExecutorInstanceProducer> _aliasResolver;
+ private final ClassAliasResolver<SpecExecutor> _aliasResolver;
public ConfigBasedTopologySpecFactory(Config config) {
this(config, Optional.<Logger>absent());
@@ -55,7 +56,7 @@ public class ConfigBasedTopologySpecFactory implements TopologySpecFactory {
Preconditions.checkNotNull(config, "Config should not be null");
_log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
_config = config;
- _aliasResolver = new ClassAliasResolver<>(SpecExecutorInstanceProducer.class);
+ _aliasResolver = new ClassAliasResolver<>(SpecExecutor.class);
}
@Override
@@ -70,21 +71,21 @@ public class ConfigBasedTopologySpecFactory implements TopologySpecFactory {
for (String topologyName : topologyNames) {
Preconditions.checkArgument(_config.hasPath(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + topologyName),
- "Config does not contain Topology Factory descriptor for Topology" + topologyName);
+ "Config does not contain Topology Factory descriptor for Topology " + topologyName);
Config topologyConfig = _config.getConfig(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + topologyName);
String description = ConfigUtils.getString(topologyConfig, ServiceConfigKeys.TOPOLOGYSPEC_DESCRIPTION_KEY, "NA");
String version = ConfigUtils.getString(topologyConfig, ServiceConfigKeys.TOPOLOGYSPEC_VERSION_KEY, "-1");
- String specExecutorInstanceProducerClass = ServiceConfigKeys.DEFAULT_SPEC_EXECUTOR_INSTANCE_PRODUCER;
- if (topologyConfig.hasPath(ServiceConfigKeys.SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY)) {
- specExecutorInstanceProducerClass = topologyConfig.getString(ServiceConfigKeys.SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY);
+ String specExecutorClass = ServiceConfigKeys.DEFAULT_SPEC_EXECUTOR;
+ if (topologyConfig.hasPath(ServiceConfigKeys.SPEC_EXECUTOR_KEY)) {
+ specExecutorClass = topologyConfig.getString(ServiceConfigKeys.SPEC_EXECUTOR_KEY);
}
- SpecExecutorInstanceProducer specExecutorInstanceProducer;
+ SpecExecutor specExecutor;
try {
- _log.info("Using SpecExecutorInstanceProducer class name/alias " + specExecutorInstanceProducerClass);
- specExecutorInstanceProducer = (SpecExecutorInstanceProducer) ConstructorUtils
+ _log.info("Using SpecProducer class name/alias " + specExecutorClass);
+ specExecutor = (SpecExecutor) ConstructorUtils
.invokeConstructor(Class.forName(_aliasResolver
- .resolve(specExecutorInstanceProducerClass)), topologyConfig);
+ .resolve(specExecutorClass)), topologyConfig);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException
| ClassNotFoundException e) {
throw new RuntimeException(e);
@@ -95,10 +96,10 @@ public class ConfigBasedTopologySpecFactory implements TopologySpecFactory {
.withConfig(topologyConfig)
.withDescription(description)
.withVersion(version)
- .withSpecExecutorInstanceProducer(specExecutorInstanceProducer);
+ .withSpecExecutor(specExecutor);
topologySpecs.add(topologySpecBuilder.build());
}
return topologySpecs;
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/DistancedNode.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/DistancedNode.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/DistancedNode.java
new file mode 100644
index 0000000..946fe10
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/DistancedNode.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.utils;
+import org.apache.gobblin.runtime.api.ServiceNode;
+
+/**
+ * This is a helping class(Basically a wrapper) for Shortest path finding process
+ * to keep the shortest distance from source to an arbitrary node.
+ */
+public class DistancedNode<T extends ServiceNode> {
+
+ /**
+ * The distance between {@link this} node to the src node in the shortest-distance finding problem.
+ */
+ private double distToSrc;
+
+ private T _serviceNode;
+
+
+ /**
+ * Max_Value represents no-connection.
+ */
+ public DistancedNode(T _serviceNode){
+ this(_serviceNode, Double.MAX_VALUE);
+ }
+
+ public DistancedNode(T _serviceNode, double dist){
+ this._serviceNode = _serviceNode;
+ this.distToSrc = dist;
+ }
+
+ public double getDistToSrc(){
+ return this.distToSrc;
+ }
+
+ public void setDistToSrc(double distToSrc){
+ this.distToSrc = distToSrc;
+ }
+
+ public T getNode(){
+ return this._serviceNode;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ DistancedNode<?> that = (DistancedNode<?>) o;
+
+ return _serviceNode.equals(that._serviceNode);
+ }
+
+ @Override
+ public int hashCode() {
+ return _serviceNode.hashCode();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/FindPathUtils.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/FindPathUtils.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/FindPathUtils.java
new file mode 100644
index 0000000..1481f78
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/FindPathUtils.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service.modules.utils;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+import org.apache.gobblin.runtime.api.FlowEdge;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.apache.gobblin.service.modules.flow.LoadBasedFlowEdgeImpl;
+import org.jgrapht.graph.DirectedWeightedMultigraph;
+
+import lombok.extern.slf4j.Slf4j;
+
+import avro.shaded.com.google.common.annotations.VisibleForTesting;
+
+@Slf4j
+public class FindPathUtils {
+ // Since Author{autumnust@gmail.com} couldn't find the proper way to conduct Library provided by JGraphT
+ // on the customized-edge Graph, here is the raw implementation of Dijkstra algorithm for finding shortest path.
+
+ /**
+ * Given sourceNode and targetNode, find the shortest path and return shortest path.
+ * @return Each edge on this shortest path, in order.
+ *
+ */
+ @VisibleForTesting
+ public static List<FlowEdge> dijkstraBasedPathFindingHelper(ServiceNode sourceNode, ServiceNode targetNode,
+ DirectedWeightedMultigraph<ServiceNode, FlowEdge> weightedGraph) {
+ Map<DistancedNode, ArrayList<FlowEdge>> shortestPath = new HashMap<>();
+ Map<DistancedNode, Double> shortestDist = new HashMap<>();
+ PriorityQueue<DistancedNode> pq = new PriorityQueue<>(new Comparator<DistancedNode>() {
+ @Override
+ public int compare(DistancedNode o1, DistancedNode o2) {
+ if (o1.getDistToSrc() < o2.getDistToSrc()) {
+ return -1;
+ } else {
+ return 1;
+ }
+ }
+ });
+ pq.add(new DistancedNode(sourceNode, 0.0));
+
+ Set<FlowEdge> visitedEdge = new HashSet<>();
+
+ while(!pq.isEmpty()) {
+ DistancedNode node = pq.poll();
+ if (node.getNode().getNodeName().equals(targetNode.getNodeName())) {
+ // Searching finished
+ return shortestPath.get(node);
+ }
+
+ Set<FlowEdge> outgoingEdges = weightedGraph.outgoingEdgesOf(node.getNode());
+ for (FlowEdge outGoingEdge:outgoingEdges) {
+ // Since it is a multi-graph problem, should use edge for deduplicaiton instead of vertex.
+ if (visitedEdge.contains(outGoingEdge)) {
+ continue;
+ }
+
+ DistancedNode adjacentNode = new DistancedNode(weightedGraph.getEdgeTarget(outGoingEdge));
+ if (shortestDist.containsKey(adjacentNode)) {
+ adjacentNode.setDistToSrc(shortestDist.get(adjacentNode));
+ }
+
+ double newDist = node.getDistToSrc() + ((LoadBasedFlowEdgeImpl) outGoingEdge).getEdgeLoad();
+
+ if (newDist < adjacentNode.getDistToSrc()) {
+ if (pq.contains(adjacentNode)) {
+ pq.remove(adjacentNode);
+ }
+
+ // Update the shortest path.
+ ArrayList<FlowEdge> path = shortestPath.containsKey(node)
+ ? new ArrayList<>(shortestPath.get(node)) : new ArrayList<>();
+ path.add(outGoingEdge);
+ shortestPath.put(adjacentNode, path);
+ shortestDist.put(adjacentNode, newDist);
+
+ adjacentNode.setDistToSrc(newDist);
+ pq.add(adjacentNode);
+ }
+ visitedEdge.add(outGoingEdge);
+ }
+ }
+ log.error("No path found");
+ return new ArrayList<>();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/HelixUtils.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f2c1c84
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/utils/HelixUtils.java
@@ -0,0 +1,110 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import java.util.UUID;
+import org.apache.helix.Criteria;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.manager.zk.ZKHelixManager;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.Message;
+import org.apache.helix.tools.ClusterSetup;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.slf4j.Logger;
+
+
+@Alpha
+public class HelixUtils {
+
+ /***
+ * 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));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceHATest.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceHATest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceHATest.java
index 01d0285..289e212 100644
--- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceHATest.java
+++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceHATest.java
@@ -49,8 +49,7 @@ import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
import org.apache.gobblin.runtime.spec_catalog.TopologyCatalog;
import org.apache.gobblin.service.FlowConfig;
import org.apache.gobblin.service.FlowConfigClient;
-import org.apache.gobblin.service.FlowId;
-import org.apache.gobblin.service.HelixUtils;
+import org.apache.gobblin.service.modules.utils.HelixUtils;
import org.apache.gobblin.service.ServiceConfigKeys;
import org.apache.gobblin.service.modules.orchestration.Orchestrator;
import org.apache.gobblin.util.ConfigUtils;
@@ -139,8 +138,8 @@ public class GobblinServiceHATest {
"1");
commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".uri",
"gobblinExecutor");
- commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".specExecutorInstanceProducer",
- "org.apache.gobblin.service.InMemorySpecExecutorInstanceProducer");
+ commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".specExecutorInstance",
+ "org.gobblin.service.InMemorySpecExecutor");
commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".specExecInstance.capabilities",
TEST_SOURCE_NAME + ":" + TEST_SINK_NAME);
@@ -506,4 +505,4 @@ public class GobblinServiceHATest {
Assert.assertTrue(assertSuccess, "New master should take over all old master jobs.");
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceManagerTest.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceManagerTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceManagerTest.java
index 314dc66..b40792e 100644
--- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceManagerTest.java
+++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceManagerTest.java
@@ -103,8 +103,8 @@ public class GobblinServiceManagerTest {
"1");
serviceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".uri",
"gobblinExecutor");
- serviceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".specExecutorInstanceProducer",
- "org.apache.gobblin.service.InMemorySpecExecutorInstanceProducer");
+ serviceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".specExecutorInstance",
+ "org.apache.gobblin.service.InMemorySpecExecutor");
serviceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX + TEST_GOBBLIN_EXECUTOR_NAME + ".specExecInstance.capabilities",
TEST_SOURCE_NAME + ":" + TEST_SINK_NAME);
@@ -339,4 +339,4 @@ public class GobblinServiceManagerTest {
Assert.fail("Get should have raised a 404 error");
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/IdentityFlowToJobSpecCompilerTest.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/IdentityFlowToJobSpecCompilerTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/IdentityFlowToJobSpecCompilerTest.java
index 0b3dc15..864b238 100644
--- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/IdentityFlowToJobSpecCompilerTest.java
+++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/IdentityFlowToJobSpecCompilerTest.java
@@ -16,6 +16,7 @@
*/
package org.apache.gobblin.service.modules.core;
+
import java.io.File;
import java.net.URI;
import java.net.URISyntaxException;
@@ -39,14 +40,14 @@ import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.runtime.api.FlowSpec;
import org.apache.gobblin.runtime.api.JobSpec;
import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
import org.apache.gobblin.runtime.api.TopologySpec;
-import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutorInstanceProducer;
import org.apache.gobblin.service.ServiceConfigKeys;
import org.apache.gobblin.service.modules.flow.IdentityFlowToJobSpecCompiler;
import org.apache.gobblin.util.ConfigUtils;
import org.apache.gobblin.util.PathUtils;
-
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor;
public class IdentityFlowToJobSpecCompilerTest {
private static final Logger logger = LoggerFactory.getLogger(IdentityFlowToJobSpecCompilerTest.class);
@@ -114,14 +115,14 @@ public class IdentityFlowToJobSpecCompilerTest {
properties.put("specStore.fs.dir", TOPOLOGY_SPEC_STORE_DIR);
properties.put("specExecInstance.capabilities", TEST_SOURCE_NAME + ":" + TEST_SINK_NAME);
Config config = ConfigUtils.propertiesToConfig(properties);
- SpecExecutorInstanceProducer specExecutorInstanceProducer = new InMemorySpecExecutorInstanceProducer(config);
+ SpecExecutor specExecutorInstance = new InMemorySpecExecutor(config);
TopologySpec.Builder topologySpecBuilder = TopologySpec.builder(computeTopologySpecURI(SPEC_STORE_PARENT_DIR,
TOPOLOGY_SPEC_STORE_DIR))
.withConfig(config)
.withDescription(SPEC_DESCRIPTION)
.withVersion(SPEC_VERSION)
- .withSpecExecutorInstanceProducer(specExecutorInstanceProducer);
+ .withSpecExecutor(specExecutorInstance);
return topologySpecBuilder.build();
}
@@ -152,7 +153,7 @@ public class IdentityFlowToJobSpecCompilerTest {
return flowSpecBuilder.build();
}
- public URI computeTopologySpecURI(String parent, String current) {
+ public static URI computeTopologySpecURI(String parent, String current) {
// Make sure this is relative
return PathUtils.relativizePath(new Path(current), new Path(parent)).toUri();
}
@@ -186,7 +187,7 @@ public class IdentityFlowToJobSpecCompilerTest {
FlowSpec flowSpec = initFlowSpec();
// Run compiler on flowSpec
- Map<Spec, SpecExecutorInstanceProducer> specExecutorMapping = this.compilerWithTemplateCalague.compileFlow(flowSpec);
+ Map<Spec, SpecExecutor> specExecutorMapping = this.compilerWithTemplateCalague.compileFlow(flowSpec);
// Assert pre-requisites
Assert.assertNotNull(specExecutorMapping, "Expected non null mapping.");
@@ -215,7 +216,7 @@ public class IdentityFlowToJobSpecCompilerTest {
FlowSpec flowSpec = initFlowSpec();
// Run compiler on flowSpec
- Map<Spec, SpecExecutorInstanceProducer> specExecutorMapping = this.compilerWithoutTemplateCalague.compileFlow(flowSpec);
+ Map<Spec, SpecExecutor> specExecutorMapping = this.compilerWithoutTemplateCalague.compileFlow(flowSpec);
// Assert pre-requisites
Assert.assertNotNull(specExecutorMapping, "Expected non null mapping.");
@@ -244,10 +245,10 @@ public class IdentityFlowToJobSpecCompilerTest {
FlowSpec flowSpec = initFlowSpec(TEST_FLOW_GROUP, TEST_FLOW_NAME, "unsupportedSource", "unsupportedSink");
// Run compiler on flowSpec
- Map<Spec, SpecExecutorInstanceProducer> specExecutorMapping = this.compilerWithTemplateCalague.compileFlow(flowSpec);
+ Map<Spec, SpecExecutor> specExecutorMapping = this.compilerWithTemplateCalague.compileFlow(flowSpec);
// Assert pre-requisites
Assert.assertNotNull(specExecutorMapping, "Expected non null mapping.");
Assert.assertTrue(specExecutorMapping.size() == 0, "Exepected 1 executor for FlowSpec.");
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/MultiHopsFlowToJobSpecCompilerTest.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/MultiHopsFlowToJobSpecCompilerTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/MultiHopsFlowToJobSpecCompilerTest.java
new file mode 100644
index 0000000..cc722eb
--- /dev/null
+++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/MultiHopsFlowToJobSpecCompilerTest.java
@@ -0,0 +1,326 @@
+/*
+ * 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.File;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import com.typesafe.config.Config;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.Path;
+import org.jgrapht.graph.DirectedWeightedMultigraph;
+import org.jgrapht.graph.WeightedMultigraph;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.spec_executorInstance.BaseServiceNodeImpl;
+import org.apache.gobblin.runtime.api.FlowEdge;
+import org.apache.gobblin.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.TopologySpec;
+import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.flow.LoadBasedFlowEdgeImpl;
+import org.apache.gobblin.service.modules.flow.MultiHopsFlowToJobSpecCompiler;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.PathUtils;
+
+import static org.apache.gobblin.service.modules.utils.FindPathUtils.*;
+
+
+// All unit tests here will be with templateCatelogue.
+public class MultiHopsFlowToJobSpecCompilerTest {
+ private static final Logger logger = LoggerFactory.getLogger(MultiHopsFlowToJobSpecCompilerTest.class);
+
+ private static final String TEST_TEMPLATE_CATALOG_PATH = "/tmp/gobblinTestTemplateCatalog_" + System.currentTimeMillis();
+ private static final String TEST_TEMPLATE_CATALOG_URI = "file://" + TEST_TEMPLATE_CATALOG_PATH;
+
+ private static final String TEST_TEMPLATE_NAME = "test.template";
+ private static final String TEST_TEMPLATE_URI = "FS:///test.template";
+
+ // The path to be discovered is TEST_SOURCE_NAME -> TEST_HOP_NAME_A -> TEST_HOP_NAME_B -> TEST_SINK_NAME
+ private static final String TEST_SOURCE_NAME = "testSource";
+ private static final String TEST_HOP_NAME_A = "testHopA";
+ private static final String TEST_HOP_NAME_B = "testHopB";
+ private static final String TEST_HOP_NAME_C = "testHopC";
+ private static final String TEST_SINK_NAME = "testSink";
+ private static final String TEST_FLOW_GROUP = "testFlowGroup";
+ private static final String TEST_FLOW_NAME = "testFlowName";
+
+ private static final String SPEC_STORE_PARENT_DIR = "/tmp/orchestrator/";
+ private static final String SPEC_DESCRIPTION = "Test Orchestrator";
+ private static final String SPEC_VERSION = "1";
+ private static final String TOPOLOGY_SPEC_STORE_DIR = "/tmp/orchestrator/topologyTestSpecStore_" + System.currentTimeMillis();
+ private static final String TOPOLOGY_SPEC_STORE_DIR_SECOND = "/tmp/orchestrator/topologyTestSpecStore_" + System.currentTimeMillis() + "_2";
+ private static final String FLOW_SPEC_STORE_DIR = "/tmp/orchestrator/flowTestSpecStore_" + System.currentTimeMillis();
+
+ private ServiceNode vertexSource;
+ private ServiceNode vertexHopA;
+ private ServiceNode vertexHopB;
+ private ServiceNode vertexHopC;
+ private ServiceNode vertexSink;
+
+ private MultiHopsFlowToJobSpecCompiler compilerWithTemplateCalague;
+ private Map<String, List<URI>> edgeTemplateMap;
+
+
+ @BeforeClass
+ public void setUp() throws Exception{
+ // Create dir for template catalog
+ FileUtils.forceMkdir(new File(TEST_TEMPLATE_CATALOG_PATH));
+
+ // Create template to use in test
+ List<String> templateEntries = new ArrayList<>();
+ templateEntries.add("testProperty1 = \"testValue1\"");
+ templateEntries.add("testProperty2 = \"test.Value1\"");
+ templateEntries.add("testProperty3 = 100");
+ FileUtils.writeLines(new File(TEST_TEMPLATE_CATALOG_PATH + "/" + TEST_TEMPLATE_NAME), templateEntries);
+
+ // Initialize complier with template catalog
+ Properties compilerWithTemplateCatalogProperties = new Properties();
+ compilerWithTemplateCatalogProperties.setProperty(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY, TEST_TEMPLATE_CATALOG_URI);
+
+ // Initialize compiler with common useful properties
+ String testPath = TEST_SOURCE_NAME + "," + TEST_HOP_NAME_A + "," + TEST_HOP_NAME_B + "," + TEST_SINK_NAME;
+ compilerWithTemplateCatalogProperties.setProperty(ServiceConfigKeys.POLICY_BASED_DATA_MOVEMENT_PATH, testPath);
+
+ this.compilerWithTemplateCalague = new MultiHopsFlowToJobSpecCompiler(ConfigUtils.propertiesToConfig(compilerWithTemplateCatalogProperties));
+
+ vertexSource = new BaseServiceNodeImpl(TEST_SOURCE_NAME);
+ vertexHopA = new BaseServiceNodeImpl(TEST_HOP_NAME_A);
+ vertexHopB = new BaseServiceNodeImpl(TEST_HOP_NAME_B);
+ vertexHopC = new BaseServiceNodeImpl(TEST_HOP_NAME_C);
+ vertexSink = new BaseServiceNodeImpl(TEST_SINK_NAME);
+
+ }
+
+ @AfterClass
+ public void cleanUp() throws Exception {
+ // Cleanup Template Catalog
+ try {
+ cleanUpDir(TEST_TEMPLATE_CATALOG_PATH);
+ } catch (Exception e) {
+ logger.warn("Could not completely cleanup Template catalog dir");
+ }
+
+ // Cleanup ToplogySpec Dir
+ try {
+ cleanUpDir(TOPOLOGY_SPEC_STORE_DIR);
+ } catch (Exception e) {
+ logger.warn("Could not completely cleanup ToplogySpec catalog dir");
+ }
+
+ // Cleanup FlowSpec Dir
+ try {
+ cleanUpDir(FLOW_SPEC_STORE_DIR);
+ } catch (Exception e) {
+ logger.warn("Could not completely cleanup FlowSpec catalog dir");
+ }
+ }
+
+ @Test
+ public void testWeightedGraphConstruction(){
+ FlowSpec flowSpec = initFlowSpec();
+ TopologySpec topologySpec = initTopologySpec(TOPOLOGY_SPEC_STORE_DIR, TEST_SOURCE_NAME, TEST_HOP_NAME_A, TEST_HOP_NAME_B, TEST_SINK_NAME);
+ this.compilerWithTemplateCalague.onAddSpec(topologySpec);
+
+ // invocation of compileFlow trigger the weighedGraph construction
+ this.compilerWithTemplateCalague.compileFlow(flowSpec);
+ DirectedWeightedMultigraph<ServiceNode, FlowEdge> weightedGraph = compilerWithTemplateCalague.getWeightedGraph();
+
+ Assert.assertTrue(weightedGraph.containsVertex(vertexSource));
+ Assert.assertTrue(weightedGraph.containsVertex(vertexHopA));
+ Assert.assertTrue(weightedGraph.containsVertex(vertexHopB));
+ Assert.assertTrue(weightedGraph.containsVertex(vertexSink));
+
+ FlowEdge edgeSrc2A = new LoadBasedFlowEdgeImpl(vertexSource, vertexHopA, topologySpec.getSpecExecutor());
+ FlowEdge edgeA2B = new LoadBasedFlowEdgeImpl(vertexHopA, vertexHopB, topologySpec.getSpecExecutor());
+ FlowEdge edgeB2Sink = new LoadBasedFlowEdgeImpl(vertexHopB, vertexSink, topologySpec.getSpecExecutor());
+
+ Assert.assertTrue(weightedGraph.containsEdge(edgeSrc2A));
+ Assert.assertTrue(weightedGraph.containsEdge(edgeA2B));
+ Assert.assertTrue(weightedGraph.containsEdge(edgeB2Sink));
+
+ Assert.assertTrue(edgeEqual(weightedGraph.getEdge(vertexSource, vertexHopA), edgeSrc2A));
+ Assert.assertTrue(edgeEqual(weightedGraph.getEdge(vertexHopA, vertexHopB), edgeA2B));
+ Assert.assertTrue(edgeEqual(weightedGraph.getEdge(vertexHopB, vertexSink), edgeB2Sink));
+
+ this.compilerWithTemplateCalague.onDeleteSpec(topologySpec.getUri(), "");
+ }
+
+ @Test
+ public void testUserSpecifiedPathCompilation(){
+ // TODO
+ }
+
+ @Test
+ public void testServicePolicy(){
+ // Initialize compiler with some blacklist properties
+ Properties properties = new Properties();
+ properties.setProperty(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY, TEST_TEMPLATE_CATALOG_URI);
+ String testPath = TEST_SOURCE_NAME + "," + TEST_HOP_NAME_A + "," + TEST_HOP_NAME_B + "," + TEST_SINK_NAME;
+ properties.setProperty(ServiceConfigKeys.POLICY_BASED_DATA_MOVEMENT_PATH, testPath);
+ properties.setProperty(ServiceConfigKeys.POLICY_BASED_BLOCKED_NODES,
+ "testHopA");
+ MultiHopsFlowToJobSpecCompiler compiler = new MultiHopsFlowToJobSpecCompiler(ConfigUtils.propertiesToConfig(properties));
+
+
+ FlowSpec flowSpec = initFlowSpec();
+ TopologySpec topologySpec = initTopologySpec(TOPOLOGY_SPEC_STORE_DIR, TEST_SOURCE_NAME, TEST_HOP_NAME_A, TEST_HOP_NAME_B, TEST_SINK_NAME);
+ compiler.onAddSpec(topologySpec);
+
+ // invocation of compileFlow trigger the weighedGraph construction
+ compiler.compileFlow(flowSpec);
+
+ compiler.servicePolicy.populateBlackListedEdges(compiler.getWeightedGraph());
+ Assert.assertEquals(compiler.servicePolicy.getBlacklistedEdges().size(), 2);
+
+ FlowEdge edgeSrc2A = new LoadBasedFlowEdgeImpl(vertexSource, vertexHopA, topologySpec.getSpecExecutor());
+ FlowEdge edgeA2B = new LoadBasedFlowEdgeImpl(vertexHopA, vertexHopB, topologySpec.getSpecExecutor());
+
+ Assert.assertTrue(compiler.servicePolicy.getBlacklistedEdges().contains(edgeSrc2A));
+ Assert.assertTrue(compiler.servicePolicy.getBlacklistedEdges().contains(edgeA2B));
+
+ }
+
+ @Test (dependsOnMethods = "testWeightedGraphConstruction")
+ public void testDijkstraPathFinding(){
+
+ FlowSpec flowSpec = initFlowSpec();
+ TopologySpec topologySpec_1 = initTopologySpec(TOPOLOGY_SPEC_STORE_DIR, TEST_SOURCE_NAME, TEST_HOP_NAME_A, TEST_HOP_NAME_B, TEST_SINK_NAME);
+ TopologySpec topologySpec_2 = initTopologySpec(TOPOLOGY_SPEC_STORE_DIR_SECOND, TEST_SOURCE_NAME, TEST_HOP_NAME_B, TEST_HOP_NAME_C, TEST_SINK_NAME);
+ this.compilerWithTemplateCalague.onAddSpec(topologySpec_1);
+ this.compilerWithTemplateCalague.onAddSpec(topologySpec_2);
+
+ // Get the edge -> Change the weight -> Materialized the edge change back to graph -> compile again -> Assertion
+ this.compilerWithTemplateCalague.compileFlow(flowSpec);
+ DirectedWeightedMultigraph<ServiceNode, FlowEdge> weightedGraph = compilerWithTemplateCalague.getWeightedGraph();
+ FlowEdge a2b= weightedGraph.getEdge(vertexHopA, vertexHopB);
+ FlowEdge b2c = weightedGraph.getEdge(vertexHopB, vertexHopC);
+ FlowEdge c2s = weightedGraph.getEdge(vertexHopC, vertexSink);
+ weightedGraph.setEdgeWeight(a2b, 1.99);
+ weightedGraph.setEdgeWeight(b2c, 0.1);
+ weightedGraph.setEdgeWeight(c2s, 0.2);
+
+ // Best route: Src - B(1) - C(0.1) - sink (0.2)
+ this.compilerWithTemplateCalague.compileFlow(flowSpec);
+ List<FlowEdge> edgeList = dijkstraBasedPathFindingHelper(vertexSource, vertexSink, weightedGraph);
+
+ FlowEdge src2b = weightedGraph.getEdge(vertexSource, vertexHopB);
+ FlowEdge b2C = weightedGraph.getEdge(vertexHopB, vertexHopC);
+ FlowEdge c2sink = weightedGraph.getEdge(vertexHopC, vertexSink);
+ Assert.assertEquals(edgeList.get(0).getEdgeIdentity(), src2b.getEdgeIdentity());
+ Assert.assertEquals(edgeList.get(1).getEdgeIdentity(), b2C.getEdgeIdentity());
+ Assert.assertEquals(edgeList.get(2).getEdgeIdentity(), c2sink.getEdgeIdentity());
+
+ this.compilerWithTemplateCalague.onDeleteSpec(topologySpec_1.getUri(), "");
+ this.compilerWithTemplateCalague.onDeleteSpec(topologySpec_2.getUri(), "");
+ }
+
+ // The topology is: Src - A - B - Dest
+ private TopologySpec initTopologySpec(String storeDir, String ... args) {
+ Properties properties = new Properties();
+ properties.put("specStore.fs.dir", storeDir);
+ String capabilitiesString = "";
+ for(int i =0 ; i < args.length - 1 ; i ++ ) {
+ capabilitiesString = capabilitiesString + ( args[i] + ":" + args[i+1] + ",");
+ }
+ Assert.assertEquals(capabilitiesString.charAt(capabilitiesString.length() - 1) , ',');
+ capabilitiesString = capabilitiesString.substring(0, capabilitiesString.length() - 1 );
+ properties.put("specExecInstance.capabilities", capabilitiesString);
+ properties.put("executorAttrs", new Properties());
+ Config config = ConfigUtils.propertiesToConfig(properties);
+ SpecExecutor specExecutorInstance = new InMemorySpecExecutor(config);
+
+ TopologySpec.Builder topologySpecBuilder = TopologySpec.builder(
+ IdentityFlowToJobSpecCompilerTest.computeTopologySpecURI(SPEC_STORE_PARENT_DIR,
+ storeDir))
+ .withConfig(config)
+ .withDescription(SPEC_DESCRIPTION)
+ .withVersion(SPEC_VERSION)
+ .withSpecExecutor(specExecutorInstance);
+ return topologySpecBuilder.build();
+ }
+
+ private FlowSpec initFlowSpec() {
+ return initFlowSpec(TEST_FLOW_GROUP, TEST_FLOW_NAME, TEST_SOURCE_NAME, TEST_SINK_NAME);
+ }
+
+ private FlowSpec initFlowSpec(String flowGroup, String flowName, String source, String destination) {
+ Properties properties = new Properties();
+ properties.put(ConfigurationKeys.JOB_SCHEDULE_KEY, "* * * * *");
+ properties.put(ConfigurationKeys.FLOW_GROUP_KEY, flowGroup);
+ properties.put(ConfigurationKeys.FLOW_NAME_KEY, flowName);
+ properties.put(ServiceConfigKeys.FLOW_SOURCE_IDENTIFIER_KEY, source);
+ properties.put(ServiceConfigKeys.FLOW_DESTINATION_IDENTIFIER_KEY, destination);
+ Config config = ConfigUtils.propertiesToConfig(properties);
+
+ FlowSpec.Builder flowSpecBuilder = null;
+ try {
+ flowSpecBuilder = FlowSpec.builder(computeTopologySpecURI(SPEC_STORE_PARENT_DIR,
+ FLOW_SPEC_STORE_DIR))
+ .withConfig(config)
+ .withDescription("dummy description")
+ .withVersion("1")
+ .withTemplate(new URI(TEST_TEMPLATE_URI));
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+ return flowSpecBuilder.build();
+ }
+
+ private void cleanUpDir(String dir) throws Exception {
+ File specStoreDir = new File(dir);
+ if (specStoreDir.exists()) {
+ FileUtils.deleteDirectory(specStoreDir);
+ }
+ }
+
+ // Criteria for FlowEdge to be equal in testing context
+ private boolean edgeEqual(FlowEdge a, FlowEdge b){
+ return (a.getEdgeIdentity().equals(b.getEdgeIdentity()) &&
+ ((LoadBasedFlowEdgeImpl)a).getEdgeLoad() == ((LoadBasedFlowEdgeImpl)b).getEdgeLoad());
+ }
+
+ // Use this function for
+ private void populateTemplateMap(WeightedMultigraph<ServiceNode, FlowEdge> weightedGraph, URI exempliedURI){
+ this.edgeTemplateMap.clear();
+ Set<FlowEdge> allEdges = weightedGraph.edgeSet();
+ for ( FlowEdge edge : allEdges ) {
+ this.edgeTemplateMap.put(edge.getEdgeIdentity(), Arrays.asList(exempliedURI)) ;
+ }
+ }
+
+ public static URI computeTopologySpecURI(String parent, String current) {
+ // Make sure this is relative
+ return PathUtils.relativizePath(new Path(current), new Path(parent)).toUri();
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/OrchestratorTest.java
----------------------------------------------------------------------
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 eb9974a..a933e85 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
@@ -17,6 +17,8 @@
package org.apache.gobblin.service.modules.orchestration;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor;
import java.io.File;
import java.net.URI;
import java.net.URISyntaxException;
@@ -40,13 +42,10 @@ import com.typesafe.config.Config;
import org.apache.gobblin.runtime.api.FlowSpec;
import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecCompiler;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
import org.apache.gobblin.runtime.api.TopologySpec;
import org.apache.gobblin.runtime.app.ServiceBasedAppLauncher;
import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
import org.apache.gobblin.runtime.spec_catalog.TopologyCatalog;
-import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutorInstanceProducer;
import org.apache.gobblin.service.modules.flow.IdentityFlowToJobSpecCompiler;
import org.apache.gobblin.util.ConfigUtils;
import org.apache.gobblin.util.PathUtils;
@@ -120,14 +119,14 @@ public class OrchestratorTest {
properties.put("specExecInstance.capabilities", "source:destination");
Config config = ConfigUtils.propertiesToConfig(properties);
- SpecExecutorInstanceProducer specExecutorInstanceProducer = new InMemorySpecExecutorInstanceProducer(config);
+ SpecExecutor specExecutorInstance = new InMemorySpecExecutor(config);
TopologySpec.Builder topologySpecBuilder = TopologySpec.builder(computeTopologySpecURI(SPEC_STORE_PARENT_DIR,
TOPOLOGY_SPEC_STORE_DIR))
.withConfig(config)
.withDescription(SPEC_DESCRIPTION)
.withVersion(SPEC_VERSION)
- .withSpecExecutorInstanceProducer(specExecutorInstanceProducer);
+ .withSpecExecutor(specExecutorInstance);
return topologySpecBuilder.build();
}
@@ -139,8 +138,6 @@ public class OrchestratorTest {
properties.put("gobblin.flow.destinationIdentifier", "destination");
Config config = ConfigUtils.propertiesToConfig(properties);
- SpecExecutorInstanceProducer specExecutorInstanceProducer = new InMemorySpecExecutorInstanceProducer(config);
-
FlowSpec.Builder flowSpecBuilder = null;
try {
flowSpecBuilder = FlowSpec.builder(computeTopologySpecURI(SPEC_STORE_PARENT_DIR,
@@ -209,10 +206,10 @@ public class OrchestratorTest {
@Test (dependsOnMethods = "createTopologySpec")
public void createFlowSpec() throws Exception {
- // Since only 1 Topology with 1 SpecExecutorInstanceProducer has been added in previous test
+ // Since only 1 Topology with 1 SpecProducer has been added in previous test
// .. it should be available and responsible for our new FlowSpec
IdentityFlowToJobSpecCompiler specCompiler = (IdentityFlowToJobSpecCompiler) this.orchestrator.getSpecCompiler();
- SpecExecutorInstanceProducer sei = specCompiler.getTopologySpecMap().values().iterator().next().getSpecExecutorInstanceProducer();
+ SpecExecutor sei = specCompiler.getTopologySpecMap().values().iterator().next().getSpecExecutor();
// List Current Specs
Collection<Spec> specs = flowCatalog.getSpecs();
@@ -225,7 +222,7 @@ public class OrchestratorTest {
// Make sure FlowCatalog is empty
Assert.assertTrue(specs.size() == 0, "Spec store should be empty before addition");
// Make sure FlowCatalog Listener is empty
- Assert.assertTrue(((List)(sei.listSpecs().get())).size() == 0, "SpecExecutorInstanceProducer should not know about "
+ Assert.assertTrue(((List)(sei.getProducer().get().listSpecs().get())).size() == 0, "SpecProducer should not know about "
+ "any Flow before addition");
// Create and add Spec
@@ -243,7 +240,7 @@ public class OrchestratorTest {
// Make sure FlowCatalog has the added Flow
Assert.assertTrue(specs.size() == 1, "Spec store should contain 1 Spec after addition");
// Orchestrator is a no-op listener for any new FlowSpecs
- Assert.assertTrue(((List)(sei.listSpecs().get())).size() == 0, "SpecExecutorInstanceProducer should contain 0 "
+ Assert.assertTrue(((List)(sei.getProducer().get().listSpecs().get())).size() == 0, "SpecProducer should contain 0 "
+ "Spec after addition");
}
@@ -251,7 +248,7 @@ public class OrchestratorTest {
public void deleteFlowSpec() throws Exception {
// Since only 1 Flow has been added in previous test it should be available
IdentityFlowToJobSpecCompiler specCompiler = (IdentityFlowToJobSpecCompiler) this.orchestrator.getSpecCompiler();
- SpecExecutorInstanceProducer sei = specCompiler.getTopologySpecMap().values().iterator().next().getSpecExecutorInstanceProducer();
+ SpecExecutor sei = specCompiler.getTopologySpecMap().values().iterator().next().getSpecExecutor();
// List Current Specs
Collection<Spec> specs = flowCatalog.getSpecs();
@@ -264,8 +261,8 @@ public class OrchestratorTest {
// Make sure FlowCatalog has the previously added Flow
Assert.assertTrue(specs.size() == 1, "Spec store should contain 1 Flow that was added in last test");
// Orchestrator is a no-op listener for any new FlowSpecs, so no FlowSpecs should be around
- int specsInSEI = ((List)(sei.listSpecs().get())).size();
- Assert.assertTrue(specsInSEI == 0, "SpecExecutorInstanceProducer should contain 0 "
+ int specsInSEI = ((List)(sei.getProducer().get().listSpecs().get())).size();
+ Assert.assertTrue(specsInSEI == 0, "SpecProducer should contain 0 "
+ "Spec after addition because Orchestrator is a no-op listener for any new FlowSpecs");
// Remove the flow
@@ -283,8 +280,8 @@ public class OrchestratorTest {
// Make sure FlowCatalog has the Flow removed
Assert.assertTrue(specs.size() == 0, "Spec store should not contain Spec after deletion");
// Make sure FlowCatalog Listener knows about the deletion
- specsInSEI = ((List)(sei.listSpecs().get())).size();
- Assert.assertTrue(specsInSEI == 0, "SpecExecutorInstanceProducer should not contain "
+ specsInSEI = ((List)(sei.getProducer().get().listSpecs().get())).size();
+ Assert.assertTrue(specsInSEI == 0, "SpecProducer should not contain "
+ "Spec after deletion");
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/test/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactoryTest.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactoryTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactoryTest.java
index c28e97b..36193dc 100644
--- a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactoryTest.java
+++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/topology/ConfigBasedTopologySpecFactoryTest.java
@@ -54,8 +54,8 @@ public class ConfigBasedTopologySpecFactoryTest {
properties.put(topology1Prefix + ServiceConfigKeys.TOPOLOGYSPEC_DESCRIPTION_KEY, "Topology for cluster");
properties.put(topology1Prefix + ServiceConfigKeys.TOPOLOGYSPEC_VERSION_KEY, "1");
properties.put(topology1Prefix + ServiceConfigKeys.TOPOLOGYSPEC_URI_KEY, "/mySpecs/" + topology1);
- properties.put(topology1Prefix + ServiceConfigKeys.SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY,
- ServiceConfigKeys.DEFAULT_SPEC_EXECUTOR_INSTANCE_PRODUCER);
+ properties.put(topology1Prefix + ServiceConfigKeys.SPEC_EXECUTOR_KEY,
+ ServiceConfigKeys.DEFAULT_SPEC_EXECUTOR);
properties.put(topology1Prefix + ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY, "salesforce:nosql");
// Topology Azkaban1 properties
@@ -63,8 +63,8 @@ public class ConfigBasedTopologySpecFactoryTest {
properties.put(topology2Prefix + ServiceConfigKeys.TOPOLOGYSPEC_DESCRIPTION_KEY, "Topology for Azkaban");
properties.put(topology2Prefix + ServiceConfigKeys.TOPOLOGYSPEC_VERSION_KEY, "2");
properties.put(topology2Prefix + ServiceConfigKeys.TOPOLOGYSPEC_URI_KEY, "/mySpecs/" + topology2);
- properties.put(topology2Prefix + ServiceConfigKeys.SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY,
- ServiceConfigKeys.DEFAULT_SPEC_EXECUTOR_INSTANCE_PRODUCER);
+ properties.put(topology2Prefix + ServiceConfigKeys.SPEC_EXECUTOR_KEY,
+ ServiceConfigKeys.DEFAULT_SPEC_EXECUTOR);
properties.put(topology2Prefix + ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY, "nosql:hdfs");
_config = ConfigUtils.propertiesToConfig(properties);
@@ -94,4 +94,4 @@ public class ConfigBasedTopologySpecFactoryTest {
"Version did not match with construction");
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gradle/scripts/dependencyDefinitions.gradle
----------------------------------------------------------------------
diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle
index 24b2473..cde546c 100644
--- a/gradle/scripts/dependencyDefinitions.gradle
+++ b/gradle/scripts/dependencyDefinitions.gradle
@@ -91,6 +91,7 @@ ext.externalDependency = [
"jacksonMapper": "org.codehaus.jackson:jackson-mapper-asl:1.9.13",
"jasypt": "org.jasypt:jasypt:1.9.2",
"jodaTime": "joda-time:joda-time:2.9.3",
+ "jgrapht": "org.jgrapht:jgrapht-core:0.9.2",
"metricsCore": "io.dropwizard.metrics:metrics-core:" + dropwizardMetricsVersion,
"metricsJvm": "io.dropwizard.metrics:metrics-jvm:" + dropwizardMetricsVersion,
"metricsGraphite": "io.dropwizard.metrics:metrics-graphite:" + dropwizardMetricsVersion,
[4/4] incubator-gobblin git commit: [GOBBLIN-3] Multi-hop flow
compiler implementation
Posted by ab...@apache.org.
[GOBBLIN-3] Multi-hop flow compiler implementation
Closes #2078 from autumnust/flowcompiler
Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/9402a903
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/9402a903
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/9402a903
Branch: refs/heads/master
Commit: 9402a9037554bcae4cc958a69a85eb4a16e8c179
Parents: ea5047e
Author: Lei Sun <au...@gmail.com>
Authored: Tue Sep 12 02:29:05 2017 -0700
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Tue Sep 12 02:29:21 2017 -0700
----------------------------------------------------------------------
conf/service/application.conf | 2 +-
.../apache/gobblin/runtime/api/FlowEdge.java | 51 +++
.../apache/gobblin/runtime/api/ServiceNode.java | 43 +++
.../gobblin/runtime/api/SpecConsumer.java | 35 ++
.../gobblin/runtime/api/SpecExecutor.java | 79 +++++
.../runtime/api/SpecExecutorInstance.java | 71 ----
.../api/SpecExecutorInstanceConsumer.java | 30 --
.../api/SpecExecutorInstanceProducer.java | 44 ---
.../gobblin/runtime/api/SpecProducer.java | 46 +++
.../gobblin/service/ServiceConfigKeys.java | 109 +++++++
.../GobblinClusterConfigurationKeys.java | 12 +-
.../ScheduledJobConfigurationManager.java | 49 ++-
.../StreamingJobConfigurationManager.java | 55 ++--
.../orchestration/AzkabanSpecExecutor.java | 76 +++++
.../AzkabanSpecExecutorInstance.java | 108 ------
.../AzkabanSpecExecutorInstanceProducer.java | 176 ----------
.../orchestration/AzkabanSpecProducer.java | 176 ++++++++++
.../service/SimpleKafkaSpecConsumer.java | 264 +++++++++++++++
.../service/SimpleKafkaSpecExecutor.java | 105 ++++++
.../SimpleKafkaSpecExecutorInstance.java | 131 --------
...SimpleKafkaSpecExecutorInstanceConsumer.java | 261 ---------------
...SimpleKafkaSpecExecutorInstanceProducer.java | 139 --------
.../service/SimpleKafkaSpecProducer.java | 140 ++++++++
.../service/StreamingKafkaSpecConsumer.java | 173 ++++++++++
...eamingKafkaSpecExecutorInstanceConsumer.java | 171 ----------
.../SimpleKafkaSpecExecutorInstanceTest.java | 180 ----------
.../service/SimpleKafkaSpecExecutorTest.java | 180 ++++++++++
.../StreamingKafkaSpecExecutorInstanceTest.java | 192 -----------
.../service/StreamingKafkaSpecExecutorTest.java | 191 +++++++++++
.../gobblin/runtime/api/SpecCompiler.java | 10 +-
.../gobblin/runtime/api/TopologySpec.java | 66 ++--
.../job_monitor/AvroJobSpecKafkaJobMonitor.java | 20 +-
.../AbstractSpecExecutor.java | 188 +++++++++++
.../BaseServiceNodeImpl.java | 100 ++++++
.../InMemorySpecExecutor.java | 93 ++++++
.../InMemorySpecExecutorInstanceProducer.java | 147 ---------
.../InMemorySpecProducer.java | 82 +++++
.../gobblin/spec_catalog/FlowCatalogTest.java | 8 +-
.../spec_catalog/TopologyCatalogTest.java | 10 +-
gobblin-service/build.gradle | 1 +
.../org/apache/gobblin/service/HelixUtils.java | 110 -------
.../gobblin/service/ServiceConfigKeys.java | 80 -----
.../modules/core/GobblinServiceManager.java | 2 +-
.../modules/flow/BaseFlowToJobSpecCompiler.java | 259 +++++++++++++++
.../service/modules/flow/FlowEdgeProps.java | 67 ++++
.../flow/IdentityFlowToJobSpecCompiler.java | 192 ++---------
.../modules/flow/LoadBasedFlowEdgeImpl.java | 180 ++++++++++
.../flow/MultiHopsFlowToJobSpecCompiler.java | 313 ++++++++++++++++++
.../modules/orchestration/Orchestrator.java | 27 +-
.../service/modules/policy/ServicePolicy.java | 51 +++
.../modules/policy/StaticServicePolicy.java | 98 ++++++
.../scheduler/GobblinServiceJobScheduler.java | 2 +-
.../ConfigBasedTopologySpecFactory.java | 27 +-
.../service/modules/utils/DistancedNode.java | 77 +++++
.../service/modules/utils/FindPathUtils.java | 109 +++++++
.../service/modules/utils/HelixUtils.java | 110 +++++++
.../modules/core/GobblinServiceHATest.java | 9 +-
.../modules/core/GobblinServiceManagerTest.java | 6 +-
.../core/IdentityFlowToJobSpecCompilerTest.java | 21 +-
.../MultiHopsFlowToJobSpecCompilerTest.java | 326 +++++++++++++++++++
.../modules/orchestration/OrchestratorTest.java | 31 +-
.../ConfigBasedTopologySpecFactoryTest.java | 10 +-
gradle/scripts/dependencyDefinitions.gradle | 1 +
63 files changed, 3921 insertions(+), 2201 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/conf/service/application.conf
----------------------------------------------------------------------
diff --git a/conf/service/application.conf b/conf/service/application.conf
index 3e292a0..3cb5b34 100644
--- a/conf/service/application.conf
+++ b/conf/service/application.conf
@@ -25,7 +25,7 @@ topologySpecFactory.topologyNames=localGobblinCluster
topologySpecFactory.localGobblinCluster.description="StandaloneClusterTopology"
topologySpecFactory.localGobblinCluster.version="1"
topologySpecFactory.localGobblinCluster.uri="gobblinCluster"
-topologySpecFactory.localGobblinCluster.specExecutorInstanceProducer.class="org.apache.gobblin.service.SimpleKafkaSpecExecutorInstanceProducer"
+topologySpecFactory.localGobblinCluster.specExecutorInstance.class="org.apache.gobblin.service.SimpleKafkaSpecProducer"
topologySpecFactory.localGobblinCluster.specExecInstance.capabilities="externalSource:InternalSink"
topologySpecFactory.localGobblinCluster.writer.kafka.topics="SimpleKafkaSpecExecutorInstanceTest"
topologySpecFactory.localGobblinCluster.writer.kafka.producerConfig.bootstrap.servers="localhost:9092"
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/FlowEdge.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/FlowEdge.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/FlowEdge.java
new file mode 100644
index 0000000..9dc6413
--- /dev/null
+++ b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/FlowEdge.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.api;
+
+import com.typesafe.config.Config;
+
+
+/**
+ * A typical edge consists of two types of attributes:
+ * - Numerical value based: Return an numerical value for evaluation.
+ * - Boolean value based: Return either true or false.
+ */
+public interface FlowEdge {
+
+ /**
+ * @return Uniqueness of an edge is defined by
+ * - sourceNode
+ * - targetNode
+ * - SpecExecutor
+ * hashCode and equals is required to implemented accordingly.
+ */
+ String getEdgeIdentity();
+
+ /**
+ * Return read-only Edge Properties .
+ * @return
+ */
+ Config getEdgeProperties();
+
+ /**
+ * @return If a edge should be considered as part of flow spec compilation result,
+ * based on all boolean-based properties like safety.
+ */
+ boolean isEdgeEnabled();
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/ServiceNode.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/ServiceNode.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/ServiceNode.java
new file mode 100644
index 0000000..eeb74c4
--- /dev/null
+++ b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/ServiceNode.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.api;
+
+import com.typesafe.config.Config;
+
+/**
+ * Abstraction of a Node in {@link SpecExecutor}
+ * 'Service' here refers to 'Service' in GaaS and it is not necessary related to a Service interface.
+ */
+public interface ServiceNode {
+ /**
+ * @return The name of node.
+ * It should be the identifier of a {@link ServiceNode}.
+ */
+ String getNodeName();
+
+ /**
+ * @return The attributes of a {@link ServiceNode}.
+ */
+ Config getNodeProps();
+
+ /**
+ * @return if the node is valid to use
+ */
+ boolean isNodeEnabled();
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecConsumer.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecConsumer.java
new file mode 100644
index 0000000..6dc16f4
--- /dev/null
+++ b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecConsumer.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.api;
+
+import java.util.List;
+import java.util.concurrent.Future;
+import org.apache.commons.lang3.tuple.Pair;
+
+
+/**
+ * A communication socket (receiving side for this class)
+ * for each {@link SpecExecutor} to receive spec to execute from Orchestrator.
+ * Implementation of this interface should specify communication channel (e.g. Kafka, REST, etc.)
+ */
+public interface SpecConsumer<V> {
+
+ /** List of newly changed {@link Spec}s for execution on {@link SpecExecutor}. */
+ Future<? extends List<Pair<SpecExecutor.Verb, V>>> changedSpecs();
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutor.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutor.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutor.java
new file mode 100644
index 0000000..cb5197a
--- /dev/null
+++ b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutor.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.api;
+
+import java.net.URI;
+import java.util.Map;
+import java.util.concurrent.Future;
+
+import com.typesafe.config.Config;
+
+
+/**
+ * Defines a representation of JobSpec-Executor in GaaS.
+ * A triplet of <Technology, location, communication mechanism> uniquely defines an object of SpecExecutor.
+ * e.g. <Lumos, Holdem, Rest> represents a Executor that moves data by Lumos, running on Holdem can be reached by Rest.
+ */
+public interface SpecExecutor {
+ /** An URI identifying the SpecExecutor. */
+ URI getUri();
+
+ /** Human-readable description of the SpecExecutor .*/
+ Future<String> getDescription();
+
+ /** SpecExecutor config as a typesafe config object. */
+ Future<Config> getConfig();
+
+ /** SpecExecutor attributes include Location of SpecExecutor and the Type of it (Technology it used for data movement,
+ * like, gobblin-standalone/gobblin-cluster
+ * SpecExecutor attributes are supposed to be read-only once instantiated.
+ * */
+ Config getAttrs();
+
+ /** Health of SpecExecutor. */
+ Future<String> getHealth();
+
+ /** Source : Destination processing capabilities of SpecExecutor. */
+ Future<? extends Map<ServiceNode, ServiceNode>> getCapabilities();
+
+ /** A communication socket for generating spec to assigned physical executors, paired with
+ * a consumer on the physical executor side. */
+ Future<? extends SpecProducer> getProducer();
+
+ public static enum Verb {
+ ADD(1, "add"),
+ UPDATE(2, "update"),
+ DELETE(3, "delete");
+
+ private int _id;
+ private String _verb;
+
+ Verb(int id, String verb) {
+ _id = id;
+ _verb = verb;
+ }
+
+ public int getId() {
+ return _id;
+ }
+
+ public String getVerb() {
+ return _verb;
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstance.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstance.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstance.java
deleted file mode 100644
index 47569f3..0000000
--- a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstance.java
+++ /dev/null
@@ -1,71 +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.runtime.api;
-
-import java.net.URI;
-import java.util.Map;
-import java.util.concurrent.Future;
-
-import com.typesafe.config.Config;
-
-import org.apache.gobblin.annotation.Alpha;
-
-
-/**
- * Defines a SpecExecutorInstance (typically a standalone instance, cluster or Azkaban deployment)
- * that can execute a {@link Spec}.
- */
-@Alpha
-public interface SpecExecutorInstance {
- /** An URI identifying the SpecExecutorInstance. */
- URI getUri();
-
- /** Human-readable description of the SpecExecutorInstance .*/
- Future<String> getDescription();
-
- /** SpecExecutorInstance config as a typesafe config object. */
- Future<Config> getConfig();
-
- /** Health of SpecExecutorInstance. */
- Future<String> getHealth();
-
- /** Source : Destination processing capabilities of SpecExecutorInstance. */
- Future<? extends Map<String, String>> getCapabilities();
-
- public static enum Verb {
- ADD(1, "add"),
- UPDATE(2, "update"),
- DELETE(3, "delete");
-
- private int _id;
- private String _verb;
-
- Verb(int id, String verb) {
- _id = id;
- _verb = verb;
- }
-
- public int getId() {
- return _id;
- }
-
- public String getVerb() {
- return _verb;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceConsumer.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceConsumer.java
deleted file mode 100644
index 475c5af..0000000
--- a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceConsumer.java
+++ /dev/null
@@ -1,30 +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.runtime.api;
-
-import java.util.List;
-import java.util.concurrent.Future;
-import org.apache.commons.lang3.tuple.Pair;
-
-
-public interface SpecExecutorInstanceConsumer<V> extends SpecExecutorInstance {
-
- /** List of newly changed {@link Spec}s for execution on {@link SpecExecutorInstance}. */
- Future<? extends List<Pair<Verb, V>>> changedSpecs();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceProducer.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceProducer.java
deleted file mode 100644
index 12508da..0000000
--- a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecExecutorInstanceProducer.java
+++ /dev/null
@@ -1,44 +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.runtime.api;
-
-import java.net.URI;
-import java.util.List;
-import java.util.concurrent.Future;
-
-import org.apache.gobblin.annotation.Alpha;
-
-
-/**
- * Defines a SpecExecutorInstanceProducer to produce jobs to {@link SpecExecutorInstance}
- * that can execute a {@link Spec}.
- */
-@Alpha
-public interface SpecExecutorInstanceProducer<V> extends SpecExecutorInstance {
- /** Add a {@link Spec} for execution on {@link SpecExecutorInstance}. */
- Future<?> addSpec(V addedSpec);
-
- /** Update a {@link Spec} being executed on {@link SpecExecutorInstance}. */
- Future<?> updateSpec(V updatedSpec);
-
- /** Delete a {@link Spec} being executed on {@link SpecExecutorInstance}. */
- Future<?> deleteSpec(URI deletedSpecURI);
-
- /** List all {@link Spec} being executed on {@link SpecExecutorInstance}. */
- Future<? extends List<V>> listSpecs();
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecProducer.java b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecProducer.java
new file mode 100644
index 0000000..9b9e504
--- /dev/null
+++ b/gobblin-api/src/main/java/org/apache/gobblin/runtime/api/SpecProducer.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.api;
+
+import java.net.URI;
+import java.util.List;
+import java.util.concurrent.Future;
+
+import org.apache.gobblin.annotation.Alpha;
+
+
+/**
+ * Defines a SpecProducer to produce jobs to {@link SpecExecutor}
+ * that can execute a {@link Spec}.
+ *
+ * A handle on the Orchestrator side to send {@link Spec}s.
+ */
+@Alpha
+public interface SpecProducer<V> {
+ /** Add a {@link Spec} for execution on {@link SpecExecutor}. */
+ Future<?> addSpec(V addedSpec);
+
+ /** Update a {@link Spec} being executed on {@link SpecExecutor}. */
+ Future<?> updateSpec(V updatedSpec);
+
+ /** Delete a {@link Spec} being executed on {@link SpecExecutor}. */
+ Future<?> deleteSpec(URI deletedSpecURI);
+
+ /** List all {@link Spec} being executed on {@link SpecExecutor}. */
+ Future<? extends List<V>> listSpecs();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.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
new file mode 100644
index 0000000..7231e0c
--- /dev/null
+++ b/gobblin-api/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.service;
+
+import org.apache.gobblin.annotation.Alpha;
+
+@Alpha
+public class ServiceConfigKeys {
+
+ private static final String GOBBLIN_SERVICE_PREFIX = "gobblin.service.";
+
+ // Gobblin Service Manager Keys
+ public static final String GOBBLIN_SERVICE_TOPOLOGY_CATALOG_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "topologyCatalog.enabled";
+ public static final String GOBBLIN_SERVICE_FLOW_CATALOG_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "flowCatalog.enabled";
+ public static final String GOBBLIN_SERVICE_SCHEDULER_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "scheduler.enabled";
+ public static final String GOBBLIN_SERVICE_ORCHESTRATOR_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "orchestrator.enabled";
+ public static final String GOBBLIN_SERVICE_RESTLI_SERVER_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "restliServer.enabled";
+ 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";
+
+ // 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";
+
+ // 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";
+ /**
+ * Directly use canonical class name here to avoid introducing additional dependency here.
+ */
+ public static final String DEFAULT_GOBBLIN_SERVICE_FLOWCOMPILER_CLASS =
+ "org.apache.gobblin.service.modules.flow.IdentityFlowToJobSpecCompiler";
+
+ // Flow specific Keys
+ public static final String FLOW_SOURCE_IDENTIFIER_KEY = "gobblin.flow.sourceIdentifier";
+ public static final String FLOW_DESTINATION_IDENTIFIER_KEY = "gobblin.flow.destinationIdentifier";
+
+ // Command line options
+ public static final String SERVICE_NAME_OPTION_NAME = "service_name";
+
+ // Topology Factory Keys (for overall factory)
+ public static final String TOPOLOGY_FACTORY_PREFIX = "topologySpecFactory.";
+ public static final String DEFAULT_TOPOLOGY_SPEC_FACTORY =
+ "org.apache.gobblin.service.modules.topology.ConfigBasedTopologySpecFactory";
+ public static final String TOPOLOGYSPEC_FACTORY_KEY = TOPOLOGY_FACTORY_PREFIX + "class";
+ public static final String TOPOLOGY_FACTORY_TOPOLOGY_NAMES_KEY = TOPOLOGY_FACTORY_PREFIX + "topologyNames";
+
+ // Topology Factory Keys (for individual topologies)
+ public static final String TOPOLOGYSPEC_DESCRIPTION_KEY = "description";
+ public static final String TOPOLOGYSPEC_VERSION_KEY = "version";
+ public static final String TOPOLOGYSPEC_URI_KEY = "uri";
+
+ public static final String DEFAULT_SPEC_EXECUTOR =
+ "org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor";
+ public static final String SPEC_EXECUTOR_KEY = "specExecutorInstance.class";
+ public static final String EDGE_SECURITY_KEY = "edge.secured";
+
+
+ // Template Catalog Keys
+ public static final String TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY = GOBBLIN_SERVICE_PREFIX + "templateCatalogs.fullyQualifiedPath";
+
+ // Keys related to user-specified policy on route selection.
+ // Undesired connection to form an executable JobSpec.
+ // Formatted as a String list, each entry contains a string in the format of "Source1:Sink1:URI",
+ // which indicates that data movement from source1 to sink1 with specific URI of specExecutor should be avoided.
+ public static final String POLICY_BASED_BLOCKED_CONNECTION = GOBBLIN_SERVICE_PREFIX + "blockedConnections";
+
+ // Comma separated list of nodes that is blacklisted. Names put here will become the nodeName which is the ID of a serviceNode.
+ public static final String POLICY_BASED_BLOCKED_NODES = GOBBLIN_SERVICE_PREFIX + "blockedNodes";
+ // Complete path of how the data movement is executed from source to sink.
+ // Formatted as a String, each hop separated by comma, from source to sink in order.
+ public static final String POLICY_BASED_DATA_MOVEMENT_PATH = GOBBLIN_SERVICE_PREFIX + "fullDataPath";
+
+ public static final String ATTRS_PATH_IN_CONFIG = "executorAttrs";
+
+ // Gobblin Service Graph Representation Topology related Keys
+ public static final String NODE_SECURITY_KEY = "node.secured";
+ // True means node is by default secure.
+ public static final String DEFAULT_NODE_SECURITY = "true";
+
+
+ // Policy related configuration Keys
+ public static final String DEFAULT_SERVICE_POLICY = "static";
+ public static final String SERVICE_POLICY_NAME = GOBBLIN_SERVICE_PREFIX + "servicePolicy";
+ // Logging
+ public static final String GOBBLIN_SERVICE_LOG4J_CONFIGURATION_FILE = "log4j-service.properties";
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
index e8dfb1d..ea75dc3 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
@@ -68,11 +68,11 @@ public class GobblinClusterConfigurationKeys {
public static final String JOB_CONFIGURATION_MANAGER_KEY = GOBBLIN_CLUSTER_PREFIX + "job.configuration.manager";
public static final String JOB_SPEC_REFRESH_INTERVAL = GOBBLIN_CLUSTER_PREFIX + "job.spec.refresh.interval";
- public static final String SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS_KEY = GOBBLIN_CLUSTER_PREFIX + "specConsumer.class";
- public static final String DEFAULT_SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS =
- "org.apache.gobblin.service.SimpleKafkaSpecExecutorInstanceConsumer";
- public static final String DEFAULT_STREAMING_SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS =
- "org.apache.gobblin.service.StreamingKafkaSpecExecutorInstanceConsumer";
+ public static final String SPEC_CONSUMER_CLASS_KEY = GOBBLIN_CLUSTER_PREFIX + "specConsumer.class";
+ public static final String DEFAULT_SPEC_CONSUMER_CLASS =
+ "org.apache.gobblin.service.SimpleKafkaSpecConsumer";
+ public static final String DEFAULT_STREAMING_SPEC_CONSUMER_CLASS =
+ "org.apache.gobblin.service.StreamingKafkaSpecConsumer";
public static final String JOB_CATALOG_KEY = GOBBLIN_CLUSTER_PREFIX + "job.catalog";
public static final String DEFAULT_JOB_CATALOG =
"org.apache.gobblin.runtime.job_catalog.NonObservingFSJobCatalog";
@@ -80,4 +80,4 @@ public class GobblinClusterConfigurationKeys {
public static final String STOP_TIMEOUT_SECONDS = GOBBLIN_CLUSTER_PREFIX + "stopTimeoutSeconds";
public static final long DEFAULT_STOP_TIMEOUT_SECONDS = 60;
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/ScheduledJobConfigurationManager.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/ScheduledJobConfigurationManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/ScheduledJobConfigurationManager.java
index 9290b5a..0f2d356 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/ScheduledJobConfigurationManager.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/ScheduledJobConfigurationManager.java
@@ -27,24 +27,23 @@ import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
-import org.apache.commons.lang3.reflect.ConstructorUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import com.google.common.base.Optional;
import com.google.common.collect.Maps;
import com.google.common.eventbus.EventBus;
import com.typesafe.config.Config;
+import org.apache.commons.lang3.reflect.ConstructorUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.apache.gobblin.annotation.Alpha;
import org.apache.gobblin.runtime.api.JobSpec;
import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceConsumer;
import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ConfigUtils;
import org.apache.gobblin.util.ExecutorsUtils;
+import org.apache.gobblin.runtime.api.SpecConsumer;
+import org.apache.gobblin.runtime.api.SpecExecutor;
@Alpha
@@ -59,9 +58,9 @@ public class ScheduledJobConfigurationManager extends JobConfigurationManager {
private final ScheduledExecutorService fetchJobSpecExecutor;
- private final SpecExecutorInstanceConsumer specExecutorInstanceConsumer;
+ private final SpecConsumer _specConsumer;
- private final ClassAliasResolver<SpecExecutorInstanceConsumer> aliasResolver;
+ private final ClassAliasResolver<SpecConsumer> aliasResolver;
public ScheduledJobConfigurationManager(EventBus eventBus, Config config) {
super(eventBus, config);
@@ -73,17 +72,17 @@ public class ScheduledJobConfigurationManager extends JobConfigurationManager {
this.fetchJobSpecExecutor = Executors.newSingleThreadScheduledExecutor(
ExecutorsUtils.newThreadFactory(Optional.of(LOGGER), Optional.of("FetchJobSpecExecutor")));
- this.aliasResolver = new ClassAliasResolver<>(SpecExecutorInstanceConsumer.class);
+ this.aliasResolver = new ClassAliasResolver<>(SpecConsumer.class);
try {
- String specExecutorInstanceConsumerClassName = GobblinClusterConfigurationKeys.DEFAULT_SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS;
- if (config.hasPath(GobblinClusterConfigurationKeys.SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS_KEY)) {
- specExecutorInstanceConsumerClassName = config.getString(GobblinClusterConfigurationKeys.SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS_KEY);
+ String specConsumerClassName = GobblinClusterConfigurationKeys.DEFAULT_SPEC_CONSUMER_CLASS;
+ if (config.hasPath(GobblinClusterConfigurationKeys.SPEC_CONSUMER_CLASS_KEY)) {
+ specConsumerClassName = config.getString(GobblinClusterConfigurationKeys.SPEC_CONSUMER_CLASS_KEY);
}
- LOGGER.info("Using SpecExecutorInstanceConsumer ClassNameclass name/alias " + specExecutorInstanceConsumerClassName);
- this.specExecutorInstanceConsumer = (SpecExecutorInstanceConsumer) ConstructorUtils
- .invokeConstructor(Class.forName(this.aliasResolver.resolve( specExecutorInstanceConsumerClassName)), config);
+ LOGGER.info("Using SpecConsumer ClassNameclass name/alias " + specConsumerClassName);
+ this._specConsumer = (SpecConsumer) ConstructorUtils
+ .invokeConstructor(Class.forName(this.aliasResolver.resolve(specConsumerClassName)), config);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException
- | ClassNotFoundException e) {
+ | ClassNotFoundException e) {
throw new RuntimeException(e);
}
}
@@ -116,25 +115,25 @@ public class ScheduledJobConfigurationManager extends JobConfigurationManager {
* @throws InterruptedException
*/
private void fetchJobSpecs() throws ExecutionException, InterruptedException {
- List<Pair<SpecExecutorInstance.Verb, Spec>> changesSpecs =
- (List<Pair<SpecExecutorInstance.Verb, Spec>>) this.specExecutorInstanceConsumer.changedSpecs().get();
+ List<Pair<SpecExecutor.Verb, Spec>> changesSpecs =
+ (List<Pair<SpecExecutor.Verb, Spec>>) this._specConsumer.changedSpecs().get();
- for (Pair<SpecExecutorInstance.Verb, Spec> entry : changesSpecs) {
+ for (Pair<SpecExecutor.Verb, Spec> entry : changesSpecs) {
- SpecExecutorInstance.Verb verb = entry.getKey();
- if (verb.equals(SpecExecutorInstance.Verb.ADD)) {
+ SpecExecutor.Verb verb = entry.getKey();
+ if (verb.equals(SpecExecutor.Verb.ADD)) {
// Handle addition
JobSpec jobSpec = (JobSpec) entry.getValue();
postNewJobConfigArrival(jobSpec.getUri().toString(), jobSpec.getConfigAsProperties());
jobSpecs.put(entry.getValue().getUri(), (JobSpec) entry.getValue());
- } else if (verb.equals(SpecExecutorInstanceConsumer.Verb.UPDATE)) {
+ } else if (verb.equals(SpecExecutor.Verb.UPDATE)) {
// Handle update
JobSpec jobSpec = (JobSpec) entry.getValue();
postUpdateJobConfigArrival(jobSpec.getUri().toString(), jobSpec.getConfigAsProperties());
jobSpecs.put(entry.getValue().getUri(), (JobSpec) entry.getValue());
- } else if (verb.equals(SpecExecutorInstanceConsumer.Verb.DELETE)) {
+ } else if (verb.equals(SpecExecutor.Verb.DELETE)) {
// Handle delete
Spec anonymousSpec = (Spec) entry.getValue();
@@ -148,4 +147,4 @@ public class ScheduledJobConfigurationManager extends JobConfigurationManager {
protected void shutDown() throws Exception {
ExecutorsUtils.shutdownExecutorService(this.fetchJobSpecExecutor, Optional.of(LOGGER));
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
index e660710..7370dc6 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
@@ -25,30 +25,29 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableList;
import com.google.common.eventbus.EventBus;
import com.google.common.util.concurrent.Service;
import com.typesafe.config.Config;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.apache.gobblin.annotation.Alpha;
import org.apache.gobblin.runtime.api.JobSpec;
import org.apache.gobblin.runtime.api.MutableJobCatalog;
import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceConsumer;
+import org.apache.gobblin.runtime.api.SpecExecutor;
import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ConfigUtils;
import org.apache.gobblin.util.ExecutorsUtils;
import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
+import org.apache.gobblin.runtime.api.SpecConsumer;
/**
- * A {@link JobConfigurationManager} that fetches job specs from a {@link SpecExecutorInstanceConsumer} in a loop
+ * A {@link JobConfigurationManager} that fetches job specs from a {@link SpecConsumer} in a loop
* without
*/
@Alpha
@@ -57,7 +56,7 @@ public class StreamingJobConfigurationManager extends JobConfigurationManager {
private final ExecutorService fetchJobSpecExecutor;
- private final SpecExecutorInstanceConsumer specExecutorInstanceConsumer;
+ private final SpecConsumer specConsumer;
private final long stopTimeoutSeconds;
@@ -71,23 +70,23 @@ public class StreamingJobConfigurationManager extends JobConfigurationManager {
ExecutorsUtils.newThreadFactory(Optional.of(LOGGER), Optional.of("FetchJobSpecExecutor")));
String specExecutorInstanceConsumerClassName =
- ConfigUtils.getString(config, GobblinClusterConfigurationKeys.SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS_KEY,
- GobblinClusterConfigurationKeys.DEFAULT_STREAMING_SPEC_EXECUTOR_INSTANCE_CONSUMER_CLASS);
+ ConfigUtils.getString(config, GobblinClusterConfigurationKeys.SPEC_CONSUMER_CLASS_KEY,
+ GobblinClusterConfigurationKeys.DEFAULT_STREAMING_SPEC_CONSUMER_CLASS);
- LOGGER.info("Using SpecExecutorInstanceConsumer ClassNameclass name/alias " +
+ LOGGER.info("Using SpecConsumer ClassNameclass name/alias " +
specExecutorInstanceConsumerClassName);
try {
- ClassAliasResolver<SpecExecutorInstanceConsumer> aliasResolver =
- new ClassAliasResolver<>(SpecExecutorInstanceConsumer.class);
+ ClassAliasResolver<SpecConsumer> aliasResolver =
+ new ClassAliasResolver<>(SpecConsumer.class);
- this.specExecutorInstanceConsumer = (SpecExecutorInstanceConsumer) GobblinConstructorUtils.invokeFirstConstructor(
+ this.specConsumer = (SpecConsumer) GobblinConstructorUtils.invokeFirstConstructor(
Class.forName(aliasResolver.resolve(specExecutorInstanceConsumerClassName)),
ImmutableList.<Object>of(config, jobCatalog),
ImmutableList.<Object>of(config));
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException
- | ClassNotFoundException e) {
- throw new RuntimeException("Could not construct SpecExecutorInstanceConsumer " +
+ | ClassNotFoundException e) {
+ throw new RuntimeException("Could not construct SpecConsumer " +
specExecutorInstanceConsumerClassName, e);
}
}
@@ -97,8 +96,8 @@ public class StreamingJobConfigurationManager extends JobConfigurationManager {
LOGGER.info("Starting the " + StreamingJobConfigurationManager.class.getSimpleName());
// if the instance consumer is a service then need to start it to consume job specs
- if (this.specExecutorInstanceConsumer instanceof Service) {
- ((Service) this.specExecutorInstanceConsumer).startAsync().awaitRunning();
+ if (this.specConsumer instanceof Service) {
+ ((Service) this.specConsumer).startAsync().awaitRunning();
}
// submit command to fetch job specs
@@ -120,25 +119,25 @@ public class StreamingJobConfigurationManager extends JobConfigurationManager {
}
private void fetchJobSpecs() throws ExecutionException, InterruptedException {
- List<Pair<SpecExecutorInstance.Verb, Spec>> changesSpecs =
- (List<Pair<SpecExecutorInstance.Verb, Spec>>) this.specExecutorInstanceConsumer.changedSpecs().get();
+ List<Pair<SpecExecutor.Verb, Spec>> changesSpecs =
+ (List<Pair<SpecExecutor.Verb, Spec>>) this.specConsumer.changedSpecs().get();
// propagate thread interruption so that caller will exit from loop
if (Thread.interrupted()) {
throw new InterruptedException();
}
- for (Pair<SpecExecutorInstance.Verb, Spec> entry : changesSpecs) {
- SpecExecutorInstance.Verb verb = entry.getKey();
- if (verb.equals(SpecExecutorInstance.Verb.ADD)) {
+ for (Pair<SpecExecutor.Verb, Spec> entry : changesSpecs) {
+ SpecExecutor.Verb verb = entry.getKey();
+ if (verb.equals(SpecExecutor.Verb.ADD)) {
// Handle addition
JobSpec jobSpec = (JobSpec) entry.getValue();
postNewJobConfigArrival(jobSpec.getUri().toString(), jobSpec.getConfigAsProperties());
- } else if (verb.equals(SpecExecutorInstanceConsumer.Verb.UPDATE)) {
+ } else if (verb.equals(SpecExecutor.Verb.UPDATE)) {
// Handle update
JobSpec jobSpec = (JobSpec) entry.getValue();
postUpdateJobConfigArrival(jobSpec.getUri().toString(), jobSpec.getConfigAsProperties());
- } else if (verb.equals(SpecExecutorInstanceConsumer.Verb.DELETE)) {
+ } else if (verb.equals(SpecExecutor.Verb.DELETE)) {
// Handle delete
Spec anonymousSpec = (Spec) entry.getValue();
postDeleteJobConfigArrival(anonymousSpec.getUri().toString(), new Properties());
@@ -148,11 +147,11 @@ public class StreamingJobConfigurationManager extends JobConfigurationManager {
@Override
protected void shutDown() throws Exception {
- if (this.specExecutorInstanceConsumer instanceof Service) {
- ((Service) this.specExecutorInstanceConsumer).stopAsync().awaitTerminated(this.stopTimeoutSeconds,
+ if (this.specConsumer instanceof Service) {
+ ((Service) this.specConsumer).stopAsync().awaitTerminated(this.stopTimeoutSeconds,
TimeUnit.SECONDS);
}
ExecutorsUtils.shutdownExecutorService(this.fetchJobSpecExecutor, Optional.of(LOGGER));
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutor.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutor.java b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutor.java
new file mode 100644
index 0000000..2e06817
--- /dev/null
+++ b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutor.java
@@ -0,0 +1,76 @@
+/*
+ * 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.Future;
+
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.runtime.spec_executorInstance.AbstractSpecExecutor;
+import org.apache.gobblin.util.CompletedFuture;
+import org.slf4j.Logger;
+
+import com.google.common.base.Optional;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+
+public class AzkabanSpecExecutor extends AbstractSpecExecutor {
+
+ // Executor Instance
+ protected final Config _config;
+
+ private SpecProducer<Spec> azkabanSpecProducer;
+
+ public AzkabanSpecExecutor(Config config, Optional<Logger> log) {
+ super(config, log);
+ Config defaultConfig = ConfigFactory.load(ServiceAzkabanConfigKeys.DEFAULT_AZKABAN_PROJECT_CONFIG_FILE);
+ _config = config.withFallback(defaultConfig);
+ azkabanSpecProducer = new AzkabanSpecProducer(_config, log);
+ }
+
+ @Override
+ public Future<String> getDescription() {
+ return new CompletedFuture<>("SimpleSpecExecutorInstance with URI: " + specExecutorInstanceUri, null);
+ }
+
+
+ @Override
+ public Future<? extends SpecProducer> getProducer() {
+ return new CompletedFuture<>(this.azkabanSpecProducer, null);
+ }
+
+ @Override
+ public Future<Config> getConfig() {
+ return new CompletedFuture<>(_config, null);
+ }
+
+ @Override
+ public Future<String> getHealth() {
+ return new CompletedFuture<>("Healthy", null);
+ }
+
+ @Override
+ protected void startUp() throws Exception {
+ // nothing to do in default implementation
+ }
+
+ @Override
+ protected void shutDown() throws Exception {
+ // nothing to do in default implementation
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstance.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstance.java b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstance.java
deleted file mode 100644
index dcc89cc..0000000
--- a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstance.java
+++ /dev/null
@@ -1,108 +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.net.URI;
-import java.net.URISyntaxException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Future;
-
-import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.util.CompletedFuture;
-import org.apache.gobblin.util.ConfigUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Splitter;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.AbstractIdleService;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
-
-
-public class AzkabanSpecExecutorInstance extends AbstractIdleService implements SpecExecutorInstance {
- protected static final Splitter SPLIT_BY_COMMA = Splitter.on(",").omitEmptyStrings().trimResults();
- protected static final Splitter SPLIT_BY_COLON = Splitter.on(":").omitEmptyStrings().trimResults();
-
- // Executor Instance
- protected final Config _config;
- protected final Logger _log;
- protected final URI _specExecutorInstanceUri;
- protected final Map<String, String> _capabilities;
-
- public AzkabanSpecExecutorInstance(Config config, Optional<Logger> log) {
- Config defaultConfig = ConfigFactory.load(ServiceAzkabanConfigKeys.DEFAULT_AZKABAN_PROJECT_CONFIG_FILE);
- _config = config.withFallback(defaultConfig);
- _log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
- try {
- _specExecutorInstanceUri = new URI(ConfigUtils.getString(config, ConfigurationKeys.SPECEXECUTOR_INSTANCE_URI_KEY,
- "NA"));
- } catch (URISyntaxException e) {
- throw new RuntimeException(e);
- }
- _capabilities = Maps.newHashMap();
- if (config.hasPath(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY)) {
- String capabilitiesStr = config.getString(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY);
- List<String> capabilities = SPLIT_BY_COMMA.splitToList(capabilitiesStr);
- for (String capability : capabilities) {
- List<String> currentCapability = SPLIT_BY_COLON.splitToList(capability);
- Preconditions.checkArgument(currentCapability.size() == 2, "Only one source:destination pair is supported "
- + "per capability, found: " + currentCapability);
- _capabilities.put(currentCapability.get(0), currentCapability.get(1));
- }
- }
- }
-
- @Override
- public URI getUri() {
- return _specExecutorInstanceUri;
- }
-
- @Override
- public Future<String> getDescription() {
- return new CompletedFuture<>("SimpleSpecExecutorInstance with URI: " + _specExecutorInstanceUri, null);
- }
-
- @Override
- public Future<Config> getConfig() {
- return new CompletedFuture<>(_config, null);
- }
-
- @Override
- public Future<String> getHealth() {
- return new CompletedFuture<>("Healthy", null);
- }
-
- @Override
- public Future<? extends Map<String, String>> getCapabilities() {
- return new CompletedFuture<>(_capabilities, null);
- }
-
- @Override
- protected void startUp() throws Exception {
- // nothing to do in default implementation
- }
-
- @Override
- protected void shutDown() throws Exception {
- // nothing to do in default implementation
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstanceProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstanceProducer.java b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstanceProducer.java
deleted file mode 100644
index 47df250..0000000
--- a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecExecutorInstanceProducer.java
+++ /dev/null
@@ -1,176 +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.io.Closeable;
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-import java.util.concurrent.Future;
-
-import org.apache.commons.codec.EncoderException;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.gobblin.runtime.api.JobSpec;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
-import org.apache.gobblin.util.CompletedFuture;
-import org.apache.gobblin.util.ConfigUtils;
-import org.slf4j.Logger;
-
-import com.google.common.base.Optional;
-import com.typesafe.config.Config;
-
-
-public class AzkabanSpecExecutorInstanceProducer extends AzkabanSpecExecutorInstance
- implements SpecExecutorInstanceProducer<Spec>, Closeable {
-
- // Session Id for GaaS User
- private String _sessionId;
-
-
- public AzkabanSpecExecutorInstanceProducer(Config config, Optional<Logger> log) {
- super(config, log);
-
- try {
- // Initialize Azkaban client / producer and cache credentials
- String azkabanUsername = _config.getString(ServiceAzkabanConfigKeys.AZKABAN_USERNAME_KEY);
- String azkabanPassword = getAzkabanPassword(_config);
- String azkabanServerUrl = _config.getString(ServiceAzkabanConfigKeys.AZKABAN_SERVER_URL_KEY);
-
- _sessionId = AzkabanAjaxAPIClient.authenticateAndGetSessionId(azkabanUsername, azkabanPassword, azkabanServerUrl);
- } catch (IOException | EncoderException e) {
- throw new RuntimeException("Could not authenticate with Azkaban", e);
- }
- }
-
- private String getAzkabanPassword(Config config) {
- if (StringUtils.isNotBlank(System.getProperty(ServiceAzkabanConfigKeys.AZKABAN_PASSWORD_SYSTEM_KEY))) {
- return System.getProperty(ServiceAzkabanConfigKeys.AZKABAN_PASSWORD_SYSTEM_KEY);
- }
-
- return ConfigUtils.getString(config, ServiceAzkabanConfigKeys.AZKABAN_PASSWORD_KEY, StringUtils.EMPTY);
- }
-
- public AzkabanSpecExecutorInstanceProducer(Config config, Logger log) {
- this(config, Optional.of(log));
- }
-
- /** Constructor with no logging */
- public AzkabanSpecExecutorInstanceProducer(Config config) {
- this(config, Optional.<Logger>absent());
- }
-
- @Override
- public void close() throws IOException {
-
- }
-
- @Override
- public Future<?> addSpec(Spec addedSpec) {
- // If project already exists, execute it
- try {
- AzkabanProjectConfig azkabanProjectConfig = new AzkabanProjectConfig((JobSpec) addedSpec);
- boolean azkabanProjectExists = AzkabanJobHelper.isAzkabanJobPresent(_sessionId, azkabanProjectConfig);
-
- // If project does not already exists, create and execute it
- if (azkabanProjectExists) {
- _log.info("Executing Azkaban Project: " + azkabanProjectConfig.getAzkabanProjectName());
- AzkabanJobHelper.executeJob(_sessionId, AzkabanJobHelper.getProjectId(_sessionId, azkabanProjectConfig),
- azkabanProjectConfig);
- } else {
- _log.info("Setting up Azkaban Project: " + azkabanProjectConfig.getAzkabanProjectName());
-
- // Deleted project also returns true if-project-exists check, so optimistically first create the project
- // .. (it will create project if it was never created or deleted), if project exists it will fail with
- // .. appropriate exception message, catch that and run in replace project mode if force overwrite is
- // .. specified
- try {
- createNewAzkabanProject(_sessionId, azkabanProjectConfig);
- } catch (IOException e) {
- if ("Project already exists.".equalsIgnoreCase(e.getMessage())) {
- if (ConfigUtils.getBoolean(((JobSpec) addedSpec).getConfig(),
- ServiceAzkabanConfigKeys.AZKABAN_PROJECT_OVERWRITE_IF_EXISTS_KEY, false)) {
- _log.info("Project already exists for this Spec, but force overwrite specified");
- updateExistingAzkabanProject(_sessionId, azkabanProjectConfig);
- } else {
- _log.info(String.format("Azkaban project already exists: " + "%smanager?project=%s",
- azkabanProjectConfig.getAzkabanServerUrl(), azkabanProjectConfig.getAzkabanProjectName()));
- }
- } else {
- throw e;
- }
- }
- }
-
-
- } catch (IOException e) {
- throw new RuntimeException("Issue in setting up Azkaban project.", e);
- }
-
- return new CompletedFuture<>(_config, null);
- }
-
- @Override
- public Future<?> updateSpec(Spec updatedSpec) {
- // Re-create project
- AzkabanProjectConfig azkabanProjectConfig = new AzkabanProjectConfig((JobSpec) updatedSpec);
-
- try {
- updateExistingAzkabanProject(_sessionId, azkabanProjectConfig);
- } catch (IOException e) {
- throw new RuntimeException("Issue in setting up Azkaban project.", e);
- }
-
- return new CompletedFuture<>(_config, null);
- }
-
- @Override
- public Future<?> deleteSpec(URI deletedSpecURI) {
- // Delete project
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Future<? extends List<Spec>> listSpecs() {
- throw new UnsupportedOperationException();
- }
-
- private void createNewAzkabanProject(String sessionId, AzkabanProjectConfig azkabanProjectConfig) throws IOException {
- // Create Azkaban Job
- String azkabanProjectId = AzkabanJobHelper.createAzkabanJob(sessionId, azkabanProjectConfig);
-
- // Schedule Azkaban Job
- AzkabanJobHelper.scheduleJob(sessionId, azkabanProjectId, azkabanProjectConfig);
-
- _log.info(String.format("Azkaban project created: %smanager?project=%s",
- azkabanProjectConfig.getAzkabanServerUrl(), azkabanProjectConfig.getAzkabanProjectName()));
- }
-
- private void updateExistingAzkabanProject(String sessionId, AzkabanProjectConfig azkabanProjectConfig) throws IOException {
- _log.info(String.format("Updating project: %smanager?project=%s", azkabanProjectConfig.getAzkabanServerUrl(),
- azkabanProjectConfig.getAzkabanProjectName()));
-
- // Get project Id
- String azkabanProjectId = AzkabanJobHelper.getProjectId(sessionId, azkabanProjectConfig);
-
- // Replace Azkaban Job
- AzkabanJobHelper.replaceAzkabanJob(sessionId, azkabanProjectId, azkabanProjectConfig);
-
- // Change schedule
- AzkabanJobHelper.changeJobSchedule(sessionId, azkabanProjectId, azkabanProjectConfig);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecProducer.java b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecProducer.java
new file mode 100644
index 0000000..5a491ab
--- /dev/null
+++ b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/service/modules/orchestration/AzkabanSpecProducer.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gobblin.service.modules.orchestration;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.concurrent.Future;
+
+import org.apache.commons.codec.EncoderException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.util.CompletedFuture;
+import org.apache.gobblin.util.ConfigUtils;
+import org.slf4j.Logger;
+
+import com.google.common.base.Optional;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class AzkabanSpecProducer implements SpecProducer<Spec>, Closeable {
+
+ // Session Id for GaaS User
+ private String _sessionId;
+ private Config _config;
+
+ public AzkabanSpecProducer(Config config, Optional<Logger> log) {
+ this._config = config;
+ try {
+ // Initialize Azkaban client / producer and cache credentials
+ String azkabanUsername = _config.getString(ServiceAzkabanConfigKeys.AZKABAN_USERNAME_KEY);
+ String azkabanPassword = getAzkabanPassword(_config);
+ String azkabanServerUrl = _config.getString(ServiceAzkabanConfigKeys.AZKABAN_SERVER_URL_KEY);
+
+ _sessionId = AzkabanAjaxAPIClient.authenticateAndGetSessionId(azkabanUsername, azkabanPassword, azkabanServerUrl);
+ } catch (IOException | EncoderException e) {
+ throw new RuntimeException("Could not authenticate with Azkaban", e);
+ }
+ }
+
+ private String getAzkabanPassword(Config config) {
+ if (StringUtils.isNotBlank(System.getProperty(ServiceAzkabanConfigKeys.AZKABAN_PASSWORD_SYSTEM_KEY))) {
+ return System.getProperty(ServiceAzkabanConfigKeys.AZKABAN_PASSWORD_SYSTEM_KEY);
+ }
+
+ return ConfigUtils.getString(config, ServiceAzkabanConfigKeys.AZKABAN_PASSWORD_KEY, StringUtils.EMPTY);
+ }
+
+ public AzkabanSpecProducer(Config config, Logger log) {
+ this(config, Optional.of(log));
+ }
+
+ /** Constructor with no logging */
+ public AzkabanSpecProducer(Config config) {
+ this(config, Optional.<Logger>absent());
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+
+ @Override
+ public Future<?> addSpec(Spec addedSpec) {
+ // If project already exists, execute it
+ try {
+ AzkabanProjectConfig azkabanProjectConfig = new AzkabanProjectConfig((JobSpec) addedSpec);
+ boolean azkabanProjectExists = AzkabanJobHelper.isAzkabanJobPresent(_sessionId, azkabanProjectConfig);
+
+ // If project does not already exists, create and execute it
+ if (azkabanProjectExists) {
+ log.info("Executing Azkaban Project: " + azkabanProjectConfig.getAzkabanProjectName());
+ AzkabanJobHelper.executeJob(_sessionId, AzkabanJobHelper.getProjectId(_sessionId, azkabanProjectConfig),
+ azkabanProjectConfig);
+ } else {
+ log.info("Setting up Azkaban Project: " + azkabanProjectConfig.getAzkabanProjectName());
+
+ // Deleted project also returns true if-project-exists check, so optimistically first create the project
+ // .. (it will create project if it was never created or deleted), if project exists it will fail with
+ // .. appropriate exception message, catch that and run in replace project mode if force overwrite is
+ // .. specified
+ try {
+ createNewAzkabanProject(_sessionId, azkabanProjectConfig);
+ } catch (IOException e) {
+ if ("Project already exists.".equalsIgnoreCase(e.getMessage())) {
+ if (ConfigUtils.getBoolean(((JobSpec) addedSpec).getConfig(),
+ ServiceAzkabanConfigKeys.AZKABAN_PROJECT_OVERWRITE_IF_EXISTS_KEY, false)) {
+ log.info("Project already exists for this Spec, but force overwrite specified");
+ updateExistingAzkabanProject(_sessionId, azkabanProjectConfig);
+ } else {
+ log.info(String.format("Azkaban project already exists: " + "%smanager?project=%s",
+ azkabanProjectConfig.getAzkabanServerUrl(), azkabanProjectConfig.getAzkabanProjectName()));
+ }
+ } else {
+ throw e;
+ }
+ }
+ }
+
+
+ } catch (IOException e) {
+ throw new RuntimeException("Issue in setting up Azkaban project.", e);
+ }
+
+ return new CompletedFuture<>(_config, null);
+ }
+
+ @Override
+ public Future<?> updateSpec(Spec updatedSpec) {
+ // Re-create project
+ AzkabanProjectConfig azkabanProjectConfig = new AzkabanProjectConfig((JobSpec) updatedSpec);
+
+ try {
+ updateExistingAzkabanProject(_sessionId, azkabanProjectConfig);
+ } catch (IOException e) {
+ throw new RuntimeException("Issue in setting up Azkaban project.", e);
+ }
+
+ return new CompletedFuture<>(_config, null);
+ }
+
+ @Override
+ public Future<?> deleteSpec(URI deletedSpecURI) {
+ // Delete project
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Future<? extends List<Spec>> listSpecs() {
+ throw new UnsupportedOperationException();
+ }
+
+ private void createNewAzkabanProject(String sessionId, AzkabanProjectConfig azkabanProjectConfig) throws IOException {
+ // Create Azkaban Job
+ String azkabanProjectId = AzkabanJobHelper.createAzkabanJob(sessionId, azkabanProjectConfig);
+
+ // Schedule Azkaban Job
+ AzkabanJobHelper.scheduleJob(sessionId, azkabanProjectId, azkabanProjectConfig);
+
+ log.info(String.format("Azkaban project created: %smanager?project=%s",
+ azkabanProjectConfig.getAzkabanServerUrl(), azkabanProjectConfig.getAzkabanProjectName()));
+ }
+
+ private void updateExistingAzkabanProject(String sessionId, AzkabanProjectConfig azkabanProjectConfig) throws IOException {
+ log.info(String.format("Updating project: %smanager?project=%s", azkabanProjectConfig.getAzkabanServerUrl(),
+ azkabanProjectConfig.getAzkabanProjectName()));
+
+ // Get project Id
+ String azkabanProjectId = AzkabanJobHelper.getProjectId(sessionId, azkabanProjectConfig);
+
+ // Replace Azkaban Job
+ AzkabanJobHelper.replaceAzkabanJob(sessionId, azkabanProjectId, azkabanProjectConfig);
+
+ // Change schedule
+ AzkabanJobHelper.changeJobSchedule(sessionId, azkabanProjectId, azkabanProjectConfig);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecConsumer.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecConsumer.java
new file mode 100644
index 0000000..083ccf3
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecConsumer.java
@@ -0,0 +1,264 @@
+/*
+ * 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.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Future;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.typesafe.config.Config;
+
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.kafka.client.ByteArrayBasedKafkaRecord;
+import org.apache.gobblin.kafka.client.DecodeableKafkaRecord;
+import org.apache.gobblin.kafka.client.GobblinKafkaConsumerClient;
+import org.apache.gobblin.kafka.client.Kafka08ConsumerClient;
+import org.apache.gobblin.kafka.client.KafkaConsumerRecord;
+import org.apache.gobblin.metrics.reporter.util.FixedSchemaVersionWriter;
+import org.apache.gobblin.metrics.reporter.util.SchemaVersionWriter;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecConsumer;
+import org.apache.gobblin.runtime.job_spec.AvroJobSpec;
+import org.apache.gobblin.source.extractor.extract.kafka.KafkaOffsetRetrievalFailureException;
+import org.apache.gobblin.source.extractor.extract.kafka.KafkaPartition;
+import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic;
+import org.apache.gobblin.util.CompletedFuture;
+import static org.apache.gobblin.service.SimpleKafkaSpecExecutor.*;
+
+import lombok.extern.slf4j.Slf4j;
+
+
+@Slf4j
+public class SimpleKafkaSpecConsumer implements SpecConsumer<Spec>, Closeable {
+
+ // Consumer
+ protected final GobblinKafkaConsumerClient _kafka08Consumer;
+ protected final List<KafkaPartition> _partitions;
+ protected final List<Long> _lowWatermark;
+ protected final List<Long> _nextWatermark;
+ protected final List<Long> _highWatermark;
+
+ private Iterator<KafkaConsumerRecord> messageIterator = null;
+ private int currentPartitionIdx = -1;
+ private boolean isFirstRun = true;
+
+ private final BinaryDecoder _decoder;
+ private final SpecificDatumReader<AvroJobSpec> _reader;
+ private final SchemaVersionWriter<?> _versionWriter;
+
+ public SimpleKafkaSpecConsumer(Config config, Optional<Logger> log) {
+
+ // Consumer
+ _kafka08Consumer = new Kafka08ConsumerClient.Factory().create(config);
+ List<KafkaTopic> kafkaTopics = _kafka08Consumer.getFilteredTopics(Collections.EMPTY_LIST,
+ Lists.newArrayList(Pattern.compile(config.getString(SimpleKafkaSpecExecutor.SPEC_KAFKA_TOPICS_KEY))));
+ _partitions = kafkaTopics.get(0).getPartitions();
+ _lowWatermark = Lists.newArrayList(Collections.nCopies(_partitions.size(), 0L));
+ _nextWatermark = Lists.newArrayList(Collections.nCopies(_partitions.size(), 0L));
+ _highWatermark = Lists.newArrayList(Collections.nCopies(_partitions.size(), 0L));
+
+ InputStream dummyInputStream = new ByteArrayInputStream(new byte[0]);
+ _decoder = DecoderFactory.get().binaryDecoder(dummyInputStream, null);
+ _reader = new SpecificDatumReader<AvroJobSpec>(AvroJobSpec.SCHEMA$);
+ _versionWriter = new FixedSchemaVersionWriter();
+ }
+
+ public SimpleKafkaSpecConsumer(Config config, Logger log) {
+ this(config, Optional.of(log));
+ }
+
+ /** Constructor with no logging */
+ public SimpleKafkaSpecConsumer(Config config) {
+ this(config, Optional.<Logger>absent());
+ }
+
+ @Override
+ public Future<? extends List<Pair<SpecExecutor.Verb, Spec>>> changedSpecs() {
+ List<Pair<SpecExecutor.Verb, Spec>> changesSpecs = new ArrayList<>();
+ initializeWatermarks();
+ this.currentPartitionIdx = -1;
+ while (!allPartitionsFinished()) {
+ if (currentPartitionFinished()) {
+ moveToNextPartition();
+ continue;
+ }
+ if (this.messageIterator == null || !this.messageIterator.hasNext()) {
+ try {
+ this.messageIterator = fetchNextMessageBuffer();
+ } catch (Exception e) {
+ log.error(String.format("Failed to fetch next message buffer for partition %s. Will skip this partition.",
+ getCurrentPartition()), e);
+ moveToNextPartition();
+ continue;
+ }
+ if (this.messageIterator == null || !this.messageIterator.hasNext()) {
+ moveToNextPartition();
+ continue;
+ }
+ }
+ while (!currentPartitionFinished()) {
+ if (!this.messageIterator.hasNext()) {
+ break;
+ }
+
+ KafkaConsumerRecord nextValidMessage = this.messageIterator.next();
+
+ // Even though we ask Kafka to give us a message buffer starting from offset x, it may
+ // return a buffer that starts from offset smaller than x, so we need to skip messages
+ // until we get to x.
+ if (nextValidMessage.getOffset() < _nextWatermark.get(this.currentPartitionIdx)) {
+ continue;
+ }
+
+ _nextWatermark.set(this.currentPartitionIdx, nextValidMessage.getNextOffset());
+ try {
+ final AvroJobSpec record;
+
+ if (nextValidMessage instanceof ByteArrayBasedKafkaRecord) {
+ record = decodeRecord((ByteArrayBasedKafkaRecord)nextValidMessage);
+ } else if (nextValidMessage instanceof DecodeableKafkaRecord){
+ record = ((DecodeableKafkaRecord<?, AvroJobSpec>) nextValidMessage).getValue();
+ } else {
+ throw new IllegalStateException(
+ "Unsupported KafkaConsumerRecord type. The returned record can either be ByteArrayBasedKafkaRecord"
+ + " or DecodeableKafkaRecord");
+ }
+
+ JobSpec.Builder jobSpecBuilder = JobSpec.builder(record.getUri());
+
+ Properties props = new Properties();
+ props.putAll(record.getProperties());
+ jobSpecBuilder.withJobCatalogURI(record.getUri()).withVersion(record.getVersion())
+ .withDescription(record.getDescription()).withConfigAsProperties(props);
+
+ if (!record.getTemplateUri().isEmpty()) {
+ jobSpecBuilder.withTemplate(new URI(record.getTemplateUri()));
+ }
+
+ String verbName = record.getMetadata().get(VERB_KEY);
+ Verb verb = Verb.valueOf(verbName);
+
+ changesSpecs.add(new ImmutablePair<Verb, Spec>(verb, jobSpecBuilder.build()));
+ } catch (Throwable t) {
+ log.error("Could not decode record at partition " + this.currentPartitionIdx +
+ " offset " + nextValidMessage.getOffset());
+ }
+ }
+ }
+
+ return new CompletedFuture(changesSpecs, null);
+ }
+
+ private void initializeWatermarks() {
+ initializeLowWatermarks();
+ initializeHighWatermarks();
+ }
+
+ private void initializeLowWatermarks() {
+ try {
+ int i=0;
+ for (KafkaPartition kafkaPartition : _partitions) {
+ if (isFirstRun) {
+ long earliestOffset = _kafka08Consumer.getEarliestOffset(kafkaPartition);
+ _lowWatermark.set(i, earliestOffset);
+ } else {
+ _lowWatermark.set(i, _highWatermark.get(i));
+ }
+ i++;
+ }
+ isFirstRun = false;
+ } catch (KafkaOffsetRetrievalFailureException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void initializeHighWatermarks() {
+ try {
+ int i=0;
+ for (KafkaPartition kafkaPartition : _partitions) {
+ long latestOffset = _kafka08Consumer.getLatestOffset(kafkaPartition);
+ _highWatermark.set(i, latestOffset);
+ i++;
+ }
+ } catch (KafkaOffsetRetrievalFailureException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private boolean allPartitionsFinished() {
+ return this.currentPartitionIdx >= _nextWatermark.size();
+ }
+
+ private boolean currentPartitionFinished() {
+ if (this.currentPartitionIdx == -1) {
+ return true;
+ } else if (_nextWatermark.get(this.currentPartitionIdx) >= _highWatermark.get(this.currentPartitionIdx)) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ private int moveToNextPartition() {
+ this.messageIterator = null;
+ return this.currentPartitionIdx ++;
+ }
+
+ private KafkaPartition getCurrentPartition() {
+ return _partitions.get(this.currentPartitionIdx);
+ }
+
+ private Iterator<KafkaConsumerRecord> fetchNextMessageBuffer() {
+ return _kafka08Consumer.consume(_partitions.get(this.currentPartitionIdx),
+ _nextWatermark.get(this.currentPartitionIdx), _highWatermark.get(this.currentPartitionIdx));
+ }
+
+ private AvroJobSpec decodeRecord(ByteArrayBasedKafkaRecord kafkaConsumerRecord) throws IOException {
+ InputStream is = new ByteArrayInputStream(kafkaConsumerRecord.getMessageBytes());
+ _versionWriter.readSchemaVersioningInformation(new DataInputStream(is));
+
+ Decoder decoder = DecoderFactory.get().binaryDecoder(is, _decoder);
+
+ return _reader.read(null, decoder);
+ }
+
+ @Override
+ public void close() throws IOException {
+ _kafka08Consumer.close();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutor.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutor.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutor.java
new file mode 100644
index 0000000..8545bf6
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutor.java
@@ -0,0 +1,105 @@
+/*
+ * 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.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.concurrent.Future;
+
+import com.google.common.base.Optional;
+import com.typesafe.config.Config;
+import com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.util.CompletedFuture;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecConsumer;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.runtime.spec_executorInstance.AbstractSpecExecutor;
+
+/**
+ * An {@link SpecExecutor} that use Kafka as the communication mechanism.
+ */
+public class SimpleKafkaSpecExecutor extends AbstractSpecExecutor {
+ public static final String SPEC_KAFKA_TOPICS_KEY = "spec.kafka.topics";
+
+
+ protected static final String VERB_KEY = "Verb";
+
+ private SpecProducer<Spec> specProducer;
+
+ public SimpleKafkaSpecExecutor(Config config, Optional<Logger> log) {
+ super(config, log);
+ specProducer = new SimpleKafkaSpecProducer(config, log);
+ }
+
+ /**
+ * Constructor with no logging, necessary for simple use case.
+ * @param config
+ */
+ public SimpleKafkaSpecExecutor(Config config) {
+ this(config, Optional.absent());
+ }
+
+ @Override
+ public Future<? extends SpecProducer> getProducer() {
+ return new CompletedFuture<>(this.specProducer, null);
+ }
+
+ @Override
+ public Future<String> getDescription() {
+ return new CompletedFuture<>("SimpleSpecExecutorInstance with URI: " + specExecutorInstanceUri, null);
+ }
+
+ @Override
+ protected void startUp() throws Exception {
+ optionalCloser = Optional.of(Closer.create());
+ specProducer = optionalCloser.get().register((SimpleKafkaSpecProducer) specProducer);
+ }
+
+ @Override
+ protected void shutDown() throws Exception {
+ if (optionalCloser.isPresent()) {
+ optionalCloser.get().close();
+ } else {
+ log.warn("There's no Closer existed in " + this.getClass().getName());
+ }
+ }
+
+ public static class SpecExecutorInstanceDataPacket implements Serializable {
+
+ protected Verb _verb;
+ protected URI _uri;
+ protected Spec _spec;
+
+ public SpecExecutorInstanceDataPacket(Verb verb, URI uri, Spec spec) {
+ _verb = verb;
+ _uri = uri;
+ _spec = spec;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("Verb: %s, URI: %s, Spec: %s", _verb, _uri, _spec);
+ }
+ }
+}
\ No newline at end of file
[2/4] incubator-gobblin git commit: [GOBBLIN-3] Multi-hop flow
compiler implementation
Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/AbstractSpecExecutor.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/AbstractSpecExecutor.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/AbstractSpecExecutor.java
new file mode 100644
index 0000000..e0a235c
--- /dev/null
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/AbstractSpecExecutor.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.spec_executorInstance;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Maps;
+import com.google.common.io.Closer;
+import com.google.common.util.concurrent.AbstractIdleService;
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.runtime.api.SpecConsumer;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.GobblinInstanceEnvironment;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.util.CompletedFuture;
+
+import edu.umd.cs.findbugs.annotations.SuppressWarnings;
+
+
+/**
+ * An abstract implementation of SpecExecutor without specifying communication mechanism.
+ *
+ * Normally in the implementation of {@link AbstractSpecExecutor}, it is necessary to specify:
+ * {@link SpecProducer}
+ * {@link SpecConsumer}
+ * {@link Closer}
+ */
+public abstract class AbstractSpecExecutor extends AbstractIdleService implements SpecExecutor {
+
+ private static final Splitter SPLIT_BY_COMMA = Splitter.on(",").omitEmptyStrings().trimResults();
+ private static final Splitter SPLIT_BY_COLON = Splitter.on(":").omitEmptyStrings().trimResults();
+
+ protected final transient Logger log;
+
+ // Executor Instance identifier
+ protected final URI specExecutorInstanceUri;
+
+ @SuppressWarnings(justification = "No bug", value = "SE_BAD_FIELD")
+ protected final Config config;
+
+ protected final Map<ServiceNode, ServiceNode> capabilities;
+
+ /**
+ * While AbstractSpecExecutor is up, for most producer implementations (like SimpleKafkaSpecProducer),
+ * they implements {@link java.io.Closeable} which requires registration and close methods.
+ * {@link Closer} is mainly used for managing {@link SpecProducer} and {@link SpecConsumer}.
+ */
+ protected Optional<Closer> optionalCloser;
+
+ public AbstractSpecExecutor(Config config) {
+ this(config, Optional.<Logger>absent());
+ }
+
+ public AbstractSpecExecutor(Config config, GobblinInstanceEnvironment env) {
+ this(config, Optional.of(env.getLog()));
+ }
+
+ public AbstractSpecExecutor(Config config, Optional<Logger> log) {
+
+ /**
+ * Since URI is regarded as the unique identifier for {@link SpecExecutor}(Used in equals method)
+ * it is dangerous to use default URI.
+ */
+ if (!config.hasPath(ConfigurationKeys.SPECEXECUTOR_INSTANCE_URI_KEY)) {
+ if (log.isPresent()) {
+ log.get().warn("The SpecExecutor doesn't specify URI, using the default one.");
+ }
+ }
+
+ try {
+ specExecutorInstanceUri =
+ new URI(ConfigUtils.getString(config, ConfigurationKeys.SPECEXECUTOR_INSTANCE_URI_KEY, "NA"));
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+ this.log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
+ this.config = config;
+ this.capabilities = Maps.newHashMap();
+ if (config.hasPath(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY)) {
+ String capabilitiesStr = config.getString(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY);
+ List<String> capabilities = SPLIT_BY_COMMA.splitToList(capabilitiesStr);
+ for (String capability : capabilities) {
+ List<String> currentCapability = SPLIT_BY_COLON.splitToList(capability);
+ Preconditions.checkArgument(currentCapability.size() == 2,
+ "Only one source:destination pair is supported " + "per capability, found: " + currentCapability);
+ this.capabilities.put(new BaseServiceNodeImpl(currentCapability.get(0)),
+ new BaseServiceNodeImpl(currentCapability.get(1)));
+ }
+ }
+ optionalCloser = Optional.absent();
+ }
+
+ @Override
+ public URI getUri() {
+ return specExecutorInstanceUri;
+ }
+
+ /**
+ * The definition of attributes are the technology that a {@link SpecExecutor} is using and
+ * the physical location that it runs on.
+ *
+ * These attributes are supposed to be static and read-only.
+ */
+ @Override
+ public Config getAttrs() {
+ Preconditions.checkArgument(this.config.hasPath(ServiceConfigKeys.ATTRS_PATH_IN_CONFIG),
+ "Input configuration doesn't contains SpecExecutor Attributes path.");
+ return this.config.getConfig(ServiceConfigKeys.ATTRS_PATH_IN_CONFIG);
+ }
+
+ @Override
+ public Future<Config> getConfig() {
+ return new CompletedFuture(this.config, null);
+ }
+
+ @Override
+ public Future<? extends Map<ServiceNode, ServiceNode>> getCapabilities() {
+ return new CompletedFuture(this.capabilities, null);
+ }
+
+ /**
+ * Two {@link SpecExecutor}s with the same {@link #specExecutorInstanceUri}
+ * should be considered as the same {@link SpecExecutor}.
+ */
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ AbstractSpecExecutor that = (AbstractSpecExecutor) o;
+
+ return specExecutorInstanceUri.equals(that.specExecutorInstanceUri);
+ }
+
+ @Override
+ public int hashCode() {
+ return specExecutorInstanceUri.hashCode();
+ }
+
+ /**
+ * @return In default implementation we just return 'Healthy'.
+ */
+ @Override
+ public Future<String> getHealth() {
+ return new CompletedFuture("Healthy", null);
+ }
+
+ abstract protected void startUp() throws Exception;
+
+ abstract protected void shutDown() throws Exception;
+
+ abstract public Future<? extends SpecProducer> getProducer();
+
+ abstract public Future<String> getDescription();
+}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/BaseServiceNodeImpl.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/BaseServiceNodeImpl.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/BaseServiceNodeImpl.java
new file mode 100644
index 0000000..dcc0c3b
--- /dev/null
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/BaseServiceNodeImpl.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gobblin.runtime.spec_executorInstance;
+
+import java.util.Properties;
+
+import com.google.common.base.Preconditions;
+import com.typesafe.config.Config;
+
+import lombok.Setter;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+import lombok.Getter;
+
+/**
+ * A base implementation for {@link ServiceNode} with default secured setting.
+ */
+public class BaseServiceNodeImpl implements ServiceNode {
+
+ @Getter
+ public String nodeName;
+
+ /**
+ * Contains read-only properties of an {@link ServiceNode}
+ */
+ @Getter
+ public Config nodeProps;
+
+ /**
+ * One of mutable properties of Node.
+ * Initialization: Obtained from {@link ServiceConfigKeys}.
+ * Getter/Setter: Simply thur. {@link BaseServiceNodeImpl}.
+ */
+ @Getter
+ @Setter
+ private boolean isNodeSecure;
+
+ /**
+ * For nodes missing configuration
+ * @param nodeName
+ */
+ public BaseServiceNodeImpl(String nodeName) {
+ this(nodeName, new Properties());
+ }
+
+ public BaseServiceNodeImpl(String nodeName, Properties props) {
+ Preconditions.checkNotNull(nodeName);
+ this.nodeName = nodeName;
+ isNodeSecure = Boolean.parseBoolean
+ (props.getProperty(ServiceConfigKeys.NODE_SECURITY_KEY, ServiceConfigKeys.DEFAULT_NODE_SECURITY));
+ nodeProps = ConfigUtils.propertiesToConfig(props);
+ }
+
+ /**
+ * By default each node is acceptable to use in path-finding.
+ */
+ @Override
+ public boolean isNodeEnabled() {
+ return true;
+ }
+
+ /**
+ * The comparison between two nodes should involve the configuration.
+ * Node name is the identifier for the node.
+ * */
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ BaseServiceNodeImpl that = (BaseServiceNodeImpl) o;
+
+ return nodeName.equals(that.nodeName);
+ }
+
+ @Override
+ public int hashCode() {
+ return nodeName.hashCode();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutor.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutor.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutor.java
new file mode 100644
index 0000000..e0be4e9
--- /dev/null
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutor.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.spec_executorInstance;
+
+import com.typesafe.config.ConfigFactory;
+import com.typesafe.config.ConfigValue;
+import java.net.URI;
+import java.util.Properties;
+import java.util.concurrent.Future;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.slf4j.Logger;
+
+import com.google.common.base.Optional;
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.util.CompletedFuture;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecConsumer;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.runtime.api.GobblinInstanceEnvironment;
+
+
+
+/**
+ * An {@link SpecExecutor} implementation that keep provisioned {@link Spec} in memory.
+ * Therefore there's no necessity to install {@link SpecConsumer} in this case.
+ */
+public class InMemorySpecExecutor extends AbstractSpecExecutor {
+ // Communication mechanism components.
+ // Not specifying final for further extension based on this implementation.
+ private SpecProducer<Spec> inMemorySpecProducer;
+
+ public InMemorySpecExecutor(Config config){
+ this(config, Optional.absent());
+ }
+
+ public InMemorySpecExecutor(Config config, GobblinInstanceEnvironment env){
+ this(config, Optional.of(env.getLog()));
+ }
+
+ public InMemorySpecExecutor(Config config, Optional<Logger> log) {
+ super(config, log);
+ inMemorySpecProducer = new InMemorySpecProducer(config);
+ }
+
+ /**
+ * A creator that create a SpecExecutor only specifying URI for uniqueness.
+ * @param uri
+ */
+ public static SpecExecutor createDummySpecExecutor(URI uri) {
+ Properties properties = new Properties();
+ properties.setProperty(ConfigurationKeys.SPECEXECUTOR_INSTANCE_URI_KEY, uri.toString());
+ return new InMemorySpecExecutor(ConfigFactory.parseProperties(properties));
+ }
+
+ @Override
+ public Future<String> getDescription() {
+ return new CompletedFuture("InMemory SpecExecutor", null);
+ }
+
+ @Override
+ public Future<? extends SpecProducer> getProducer(){
+ return new CompletedFuture(this.inMemorySpecProducer, null);
+ }
+
+ @Override
+ protected void startUp() throws Exception {
+ // Nothing to do in the abstract implementation.
+ }
+
+ @Override
+ protected void shutDown() throws Exception {
+ // Nothing to do in the abstract implementation.
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutorInstanceProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutorInstanceProducer.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutorInstanceProducer.java
deleted file mode 100644
index 77faaa7..0000000
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecExecutorInstanceProducer.java
+++ /dev/null
@@ -1,147 +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.runtime.spec_executorInstance;
-
-import java.io.Serializable;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Future;
-
-import edu.umd.cs.findbugs.annotations.SuppressWarnings;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Splitter;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.typesafe.config.Config;
-
-import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.runtime.api.GobblinInstanceEnvironment;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
-import org.apache.gobblin.util.CompletedFuture;
-
-
-public class InMemorySpecExecutorInstanceProducer implements SpecExecutorInstanceProducer<Spec>, Serializable {
-
- private static final Splitter SPLIT_BY_COMMA = Splitter.on(",").omitEmptyStrings().trimResults();
- private static final Splitter SPLIT_BY_COLON = Splitter.on(":").omitEmptyStrings().trimResults();
-
- private static final long serialVersionUID = 6106269076155338045L;
-
- protected final transient Logger log;
- protected final Map<URI, Spec> provisionedSpecs;
- @SuppressWarnings (justification="No bug", value="SE_BAD_FIELD")
- protected final Config config;
- protected final Map<String, String> capabilities;
-
- public InMemorySpecExecutorInstanceProducer(Config config) {
- this(config, Optional.<Logger>absent());
- }
-
- public InMemorySpecExecutorInstanceProducer(Config config, GobblinInstanceEnvironment env) {
- this(config, Optional.of(env.getLog()));
- }
-
- public InMemorySpecExecutorInstanceProducer(Config config, Optional<Logger> log) {
- this.log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
- this.config = config;
- this.provisionedSpecs = Maps.newHashMap();
- this.capabilities = Maps.newHashMap();
- if (config.hasPath(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY)) {
- String capabilitiesStr = config.getString(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY);
- List<String> capabilities = SPLIT_BY_COMMA.splitToList(capabilitiesStr);
- for (String capability : capabilities) {
- List<String> currentCapability = SPLIT_BY_COLON.splitToList(capability);
- Preconditions.checkArgument(currentCapability.size() == 2, "Only one source:destination pair is supported "
- + "per capability, found: " + currentCapability);
- this.capabilities.put(currentCapability.get(0), currentCapability.get(1));
- }
- }
- }
-
- @Override
- public URI getUri() {
- try {
- return new URI("InMemorySpecExecutorInstanceProducer");
- } catch (URISyntaxException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public Future<String> getDescription() {
- return new CompletedFuture("InMemory SpecExecutorInstanceProducer", null);
- }
-
- @Override
- public Future<Config> getConfig() {
- return new CompletedFuture(this.config, null);
- }
-
- @Override
- public Future<String> getHealth() {
- return new CompletedFuture("Healthy", null);
- }
-
- @Override
- public Future<? extends Map<String, String>> getCapabilities() {
- return new CompletedFuture(this.capabilities, null);
- }
-
- @Override
- public Future<?> addSpec(Spec addedSpec) {
- provisionedSpecs.put(addedSpec.getUri(), addedSpec);
- log.info(String.format("Added Spec: %s with Uri: %s for execution on this executor.", addedSpec, addedSpec.getUri()));
-
- return new CompletedFuture(Boolean.TRUE, null);
- }
-
- @Override
- public Future<?> updateSpec(Spec updatedSpec) {
- if (!provisionedSpecs.containsKey(updatedSpec.getUri())) {
- throw new RuntimeException("Spec not found: " + updatedSpec.getUri());
- }
- provisionedSpecs.put(updatedSpec.getUri(), updatedSpec);
- log.info(String.format("Updated Spec: %s with Uri: %s for execution on this executor.", updatedSpec, updatedSpec.getUri()));
-
- return new CompletedFuture(Boolean.TRUE, null);
- }
-
- @Override
- public Future<?> deleteSpec(URI deletedSpecURI) {
- if (!provisionedSpecs.containsKey(deletedSpecURI)) {
- throw new RuntimeException("Spec not found: " + deletedSpecURI);
- }
- provisionedSpecs.remove(deletedSpecURI);
- log.info(String.format("Deleted Spec with Uri: %s from this executor.", deletedSpecURI));
-
- return new CompletedFuture(Boolean.TRUE, null);
- }
-
- @Override
- public Future<? extends List<Spec>> listSpecs() {
- return new CompletedFuture<>(Lists.newArrayList(provisionedSpecs.values()), null);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecProducer.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecProducer.java
new file mode 100644
index 0000000..80f64ec
--- /dev/null
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_executorInstance/InMemorySpecProducer.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.spec_executorInstance;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.util.CompletedFuture;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class InMemorySpecProducer implements SpecProducer<Spec>, Serializable {
+ private final Map<URI, Spec> provisionedSpecs;
+ private transient Config config;
+
+ private static final long serialVersionUID = 6106269076155338045L;
+
+ public InMemorySpecProducer(Config config) {
+ this.config = config;
+ this.provisionedSpecs = Maps.newHashMap();
+ }
+
+ @Override
+ public Future<?> addSpec(Spec addedSpec) {
+ provisionedSpecs.put(addedSpec.getUri(), addedSpec);
+ log.info(String.format("Added Spec: %s with Uri: %s for execution on this executor.", addedSpec, addedSpec.getUri()));
+
+ return new CompletedFuture(Boolean.TRUE, null);
+ }
+
+ @Override
+ public Future<?> updateSpec(Spec updatedSpec) {
+ if (!provisionedSpecs.containsKey(updatedSpec.getUri())) {
+ throw new RuntimeException("Spec not found: " + updatedSpec.getUri());
+ }
+ provisionedSpecs.put(updatedSpec.getUri(), updatedSpec);
+ log.info(String.format("Updated Spec: %s with Uri: %s for execution on this executor.", updatedSpec, updatedSpec.getUri()));
+
+ return new CompletedFuture(Boolean.TRUE, null);
+ }
+
+ @Override
+ public Future<?> deleteSpec(URI deletedSpecURI) {
+ if (!provisionedSpecs.containsKey(deletedSpecURI)) {
+ throw new RuntimeException("Spec not found: " + deletedSpecURI);
+ }
+ provisionedSpecs.remove(deletedSpecURI);
+ log.info(String.format("Deleted Spec with Uri: %s from this executor.", deletedSpecURI));
+
+ return new CompletedFuture(Boolean.TRUE, null);
+ }
+
+ @Override
+ public Future<? extends List<Spec>> listSpecs() {
+ return new CompletedFuture<>(Lists.newArrayList(provisionedSpecs.values()), null);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/FlowCatalogTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/FlowCatalogTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/FlowCatalogTest.java
index c42e605..73c1f46 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/FlowCatalogTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/FlowCatalogTest.java
@@ -38,13 +38,13 @@ import com.google.gson.GsonBuilder;
import com.typesafe.config.Config;
import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
import org.apache.gobblin.runtime.api.FlowSpec;
import org.apache.gobblin.runtime.app.ServiceBasedAppLauncher;
import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
-import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutorInstanceProducer;
import org.apache.gobblin.util.ConfigUtils;
import org.apache.gobblin.util.PathUtils;
+import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor;
+import org.apache.gobblin.runtime.api.SpecExecutor;
public class FlowCatalogTest {
@@ -89,7 +89,7 @@ public class FlowCatalogTest {
properties.put("specExecInstance.capabilities", "source:destination");
Config config = ConfigUtils.propertiesToConfig(properties);
- SpecExecutorInstanceProducer specExecutorInstanceProducer = new InMemorySpecExecutorInstanceProducer(config);
+ SpecExecutor specExecutorInstanceProducer = new InMemorySpecExecutor(config);
FlowSpec.Builder flowSpecBuilder = null;
try {
@@ -171,4 +171,4 @@ public class FlowCatalogTest {
URI uri = PathUtils.relativizePath(new Path(SPEC_STORE_DIR), new Path(SPEC_STORE_PARENT_DIR)).toUri();
return uri;
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/TopologyCatalogTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/TopologyCatalogTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/TopologyCatalogTest.java
index 594c755..48fba40 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/TopologyCatalogTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/spec_catalog/TopologyCatalogTest.java
@@ -23,6 +23,7 @@ import java.util.Collection;
import java.util.Properties;
import org.apache.commons.io.FileUtils;
+import org.apache.gobblin.runtime.api.SpecExecutor;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -37,11 +38,10 @@ import com.google.gson.GsonBuilder;
import com.typesafe.config.Config;
import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
import org.apache.gobblin.runtime.api.TopologySpec;
import org.apache.gobblin.runtime.app.ServiceBasedAppLauncher;
import org.apache.gobblin.runtime.spec_catalog.TopologyCatalog;
-import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutorInstanceProducer;
+import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor;
import org.apache.gobblin.util.ConfigUtils;
import org.apache.gobblin.util.PathUtils;
@@ -89,13 +89,13 @@ public class TopologyCatalogTest {
properties.put("specExecInstance.capabilities", "source:destination");
Config config = ConfigUtils.propertiesToConfig(properties);
- SpecExecutorInstanceProducer specExecutorInstanceProducer = new InMemorySpecExecutorInstanceProducer(config);
+ SpecExecutor specExecutorInstanceProducer = new InMemorySpecExecutor(config);
TopologySpec.Builder topologySpecBuilder = TopologySpec.builder(computeTopologySpecURI())
.withConfig(config)
.withDescription(SPEC_DESCRIPTION)
.withVersion(SPEC_VERSION)
- .withSpecExecutorInstanceProducer(specExecutorInstanceProducer);
+ .withSpecExecutor(specExecutorInstanceProducer);
return topologySpecBuilder.build();
}
@@ -166,4 +166,4 @@ public class TopologyCatalogTest {
URI uri = PathUtils.relativizePath(new Path(SPEC_STORE_DIR), new Path(SPEC_STORE_PARENT_DIR)).toUri();
return uri;
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/build.gradle
----------------------------------------------------------------------
diff --git a/gobblin-service/build.gradle b/gobblin-service/build.gradle
index 642d818..0b461bd 100644
--- a/gobblin-service/build.gradle
+++ b/gobblin-service/build.gradle
@@ -58,6 +58,7 @@ dependencies {
compile externalDependency.javaxInject
compile externalDependency.jgit
compile externalDependency.jodaTime
+ compile externalDependency.jgrapht
compile externalDependency.kafka08
compile externalDependency.log4j
compile externalDependency.lombok
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/HelixUtils.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/HelixUtils.java b/gobblin-service/src/main/java/org/apache/gobblin/service/HelixUtils.java
deleted file mode 100644
index 4f01623..0000000
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/HelixUtils.java
+++ /dev/null
@@ -1,110 +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 com.google.common.annotations.VisibleForTesting;
-import java.util.UUID;
-import org.apache.helix.Criteria;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
-import org.apache.helix.InstanceType;
-import org.apache.helix.manager.zk.ZKHelixManager;
-import org.apache.helix.model.HelixConfigScope;
-import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
-
-import org.apache.gobblin.annotation.Alpha;
-import org.slf4j.Logger;
-
-
-@Alpha
-public class HelixUtils {
-
- /***
- * 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));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java b/gobblin-service/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java
deleted file mode 100644
index 8ea19c4..0000000
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/ServiceConfigKeys.java
+++ /dev/null
@@ -1,80 +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 org.apache.gobblin.annotation.Alpha;
-import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutorInstanceProducer;
-import org.apache.gobblin.service.modules.flow.IdentityFlowToJobSpecCompiler;
-import org.apache.gobblin.service.modules.topology.ConfigBasedTopologySpecFactory;
-
-@Alpha
-public class ServiceConfigKeys {
-
- private static final String GOBBLIN_SERVICE_PREFIX = "gobblin.service.";
-
- // Gobblin Service Manager Keys
- public static final String GOBBLIN_SERVICE_TOPOLOGY_CATALOG_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "topologyCatalog.enabled";
- public static final String GOBBLIN_SERVICE_FLOW_CATALOG_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "flowCatalog.enabled";
- public static final String GOBBLIN_SERVICE_SCHEDULER_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "scheduler.enabled";
- public static final String GOBBLIN_SERVICE_ORCHESTRATOR_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "orchestrator.enabled";
- public static final String GOBBLIN_SERVICE_RESTLI_SERVER_ENABLED_KEY = GOBBLIN_SERVICE_PREFIX + "restliServer.enabled";
- 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";
-
- // 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";
-
- // 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 DEFAULT_GOBBLIN_SERVICE_FLOWCOMPILER_CLASS = IdentityFlowToJobSpecCompiler.class.getCanonicalName();
-
- // Flow specific Keys
- public static final String FLOW_SOURCE_IDENTIFIER_KEY = "gobblin.flow.sourceIdentifier";
- public static final String FLOW_DESTINATION_IDENTIFIER_KEY = "gobblin.flow.destinationIdentifier";
-
- // Command line options
- public static final String SERVICE_NAME_OPTION_NAME = "service_name";
-
- // Topology Factory Keys (for overall factory)
- public static final String TOPOLOGY_FACTORY_PREFIX = "topologySpecFactory.";
- public static final String DEFAULT_TOPOLOGY_SPEC_FACTORY = ConfigBasedTopologySpecFactory.class.getCanonicalName();
- public static final String TOPOLOGYSPEC_FACTORY_KEY = TOPOLOGY_FACTORY_PREFIX + "class";
- public static final String TOPOLOGY_FACTORY_TOPOLOGY_NAMES_KEY = TOPOLOGY_FACTORY_PREFIX + "topologyNames";
-
- // Topology Factory Keys (for individual topologies)
- public static final String TOPOLOGYSPEC_DESCRIPTION_KEY = "description";
- public static final String TOPOLOGYSPEC_VERSION_KEY = "version";
- public static final String TOPOLOGYSPEC_URI_KEY = "uri";
- public static final String DEFAULT_SPEC_EXECUTOR_INSTANCE_PRODUCER = InMemorySpecExecutorInstanceProducer.class.getCanonicalName();
- public static final String SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY = "specExecutorInstanceProducer.class";
-
- // Template Catalog Keys
- public static final String TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY = GOBBLIN_SERVICE_PREFIX + "templateCatalogs.fullyQualifiedPath";
-
- // Logging
- public static final String GOBBLIN_SERVICE_LOG4J_CONFIGURATION_FILE = "log4j-service.properties";
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java
----------------------------------------------------------------------
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 c2591e1..598371d 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
@@ -83,7 +83,7 @@ import org.apache.gobblin.scheduler.SchedulerService;
import org.apache.gobblin.service.FlowConfig;
import org.apache.gobblin.service.FlowConfigClient;
import org.apache.gobblin.service.FlowConfigsResource;
-import org.apache.gobblin.service.HelixUtils;
+import org.apache.gobblin.service.modules.utils.HelixUtils;
import org.apache.gobblin.service.ServiceConfigKeys;
import org.apache.gobblin.service.modules.orchestration.Orchestrator;
import org.apache.gobblin.service.modules.scheduler.GobblinServiceJobScheduler;
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java
new file mode 100644
index 0000000..26f4463
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java
@@ -0,0 +1,259 @@
+/*
+ * 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.flow;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Timer;
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.gobblin.configuration.State;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.Tag;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecCompiler;
+import org.apache.gobblin.runtime.api.TopologySpec;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.runtime.job_catalog.FSJobCatalog;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.ServiceMetricNames;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.FlowSpec;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.JobTemplate;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.runtime.api.SpecNotFoundException;
+import org.apache.gobblin.runtime.job_spec.ResolvedJobSpec;
+
+import lombok.Getter;
+import lombok.Setter;
+
+// Provide base implementation for constructing multi-hops route.
+@Alpha
+public abstract class BaseFlowToJobSpecCompiler implements SpecCompiler {
+
+ // Since {@link SpecCompiler} is an {@link SpecCatalogListener}, it is expected that any Spec change should be reflected
+ // to these data structures.
+ @Getter
+ @Setter
+ protected final Map<URI, TopologySpec> topologySpecMap;
+
+
+ /**
+ * Mapping between each FlowEdge and a list of applicable Templates.
+ * Compiler should obtain this Map info from higher level component.
+ * since {@link TopologySpec} doesn't contain Templates.
+ * Key: EdgeIdentifier from {@link org.apache.gobblin.runtime.api.FlowEdge#getEdgeIdentity()}
+ * Value: List of template URI.
+ */
+ // TODO: Define how template info are instantiated. ETL-6217
+ @Getter
+ @Setter
+ protected final Map<String, List<URI>> edgeTemplateMap;
+
+
+ protected final Config config;
+ protected final Logger log;
+ protected final Optional<FSJobCatalog> templateCatalog;
+
+ protected final MetricContext metricContext;
+ @Getter
+ protected Optional<Meter> flowCompilationSuccessFulMeter;
+ @Getter
+ protected Optional<Meter> flowCompilationFailedMeter;
+ @Getter
+ protected Optional<Timer> flowCompilationTimer;
+
+ public BaseFlowToJobSpecCompiler(Config config){
+ this(config,true);
+ }
+
+ public BaseFlowToJobSpecCompiler(Config config, boolean instrumentationEnabled){
+ this(config, Optional.<Logger>absent(), true);
+ }
+
+ public BaseFlowToJobSpecCompiler(Config config, Optional<Logger> log){
+ this(config, log,true);
+ }
+
+ public BaseFlowToJobSpecCompiler(Config config, Optional<Logger> log, boolean instrumentationEnabled){
+ this.log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
+ if (instrumentationEnabled) {
+ this.metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(config), IdentityFlowToJobSpecCompiler.class);
+ this.flowCompilationSuccessFulMeter = Optional.of(this.metricContext.meter(ServiceMetricNames.FLOW_COMPILATION_SUCCESSFUL_METER));
+ this.flowCompilationFailedMeter = Optional.of(this.metricContext.meter(ServiceMetricNames.FLOW_COMPILATION_FAILED_METER));
+ this.flowCompilationTimer = Optional.<Timer>of(this.metricContext.timer(ServiceMetricNames.FLOW_COMPILATION_TIMER));
+ }
+ else {
+ this.metricContext = null;
+ this.flowCompilationSuccessFulMeter = Optional.absent();
+ this.flowCompilationFailedMeter = Optional.absent();
+ this.flowCompilationTimer = Optional.absent();
+ }
+
+ this.topologySpecMap = Maps.newConcurrentMap();
+ this.edgeTemplateMap = Maps.newConcurrentMap();
+ this.config = config;
+
+ /***
+ * ETL-5996
+ * For multi-tenancy, the following needs to be added:
+ * 1. Change singular templateCatalog to Map<URI, JobCatalogWithTemplates> to support multiple templateCatalogs
+ * 2. Pick templateCatalog from JobCatalogWithTemplates based on URI, and try to resolve JobSpec using that
+ */
+ try {
+ if (this.config.hasPath(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY)
+ && StringUtils.isNotBlank(this.config.getString(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY))) {
+ Config templateCatalogCfg = config
+ .withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY,
+ this.config.getValue(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY));
+ this.templateCatalog = Optional.of(new FSJobCatalog(templateCatalogCfg));
+ } else {
+ this.templateCatalog = Optional.absent();
+ }
+ } catch (IOException e) {
+ throw new RuntimeException("Could not initialize FlowCompiler because of "
+ + "TemplateCatalog initialization failure", e);
+ }
+ }
+
+ @Override
+ public synchronized void onAddSpec(Spec addedSpec) {
+ topologySpecMap.put(addedSpec.getUri(), (TopologySpec) addedSpec);
+ }
+
+ @Override
+ public synchronized void onDeleteSpec(URI deletedSpecURI, String deletedSpecVersion) {
+ if (topologySpecMap.containsKey(deletedSpecURI)) {
+ topologySpecMap.remove(deletedSpecURI);
+ }
+ }
+
+ @Override
+ public synchronized void onUpdateSpec(Spec updatedSpec) {
+ topologySpecMap.put(updatedSpec.getUri(), (TopologySpec) updatedSpec);
+ }
+
+ @Nonnull
+ @Override
+ public MetricContext getMetricContext() {
+ return this.metricContext;
+ }
+
+ @Override
+ public boolean isInstrumentationEnabled() {
+ return null != this.metricContext;
+ }
+
+ @Override
+ public List<Tag<?>> generateTags(State state){
+ return Collections.emptyList();
+ }
+
+ @Override
+ public void switchMetricContext(List<Tag<?>> tags) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void switchMetricContext(MetricContext context) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Map<URI, TopologySpec> getTopologySpecMap() {
+ return this.topologySpecMap;
+ }
+
+ public abstract Map<Spec, SpecExecutor> compileFlow(Spec spec);
+
+ /**
+ * Naive implementation of generating jobSpec, which fetch the first available template,
+ * in an exemplified single-hop FlowCompiler implementation.
+ * @param flowSpec
+ * @return
+ */
+ protected JobSpec jobSpecGenerator(FlowSpec flowSpec) {
+ JobSpec jobSpec;
+ JobSpec.Builder jobSpecBuilder = JobSpec.builder(flowSpec.getUri())
+ .withConfig(flowSpec.getConfig())
+ .withDescription(flowSpec.getDescription())
+ .withVersion(flowSpec.getVersion());
+
+ if (flowSpec.getTemplateURIs().isPresent() && templateCatalog.isPresent()) {
+ // Only first template uri will be honored for Identity
+ jobSpecBuilder = jobSpecBuilder.withTemplate(flowSpec.getTemplateURIs().get().iterator().next());
+ try {
+ jobSpec = new ResolvedJobSpec(jobSpecBuilder.build(), templateCatalog.get());
+ log.info("Resolved JobSpec properties are: " + jobSpec.getConfigAsProperties());
+ } catch (SpecNotFoundException | JobTemplate.TemplateException e) {
+ throw new RuntimeException("Could not resolve template in JobSpec from TemplateCatalog", e);
+ }
+ } else {
+ jobSpec = jobSpecBuilder.build();
+ log.info("Unresolved JobSpec properties are: " + jobSpec.getConfigAsProperties());
+ }
+
+ // Remove schedule
+ jobSpec.setConfig(jobSpec.getConfig().withoutPath(ConfigurationKeys.JOB_SCHEDULE_KEY));
+
+ // Add job.name and job.group
+ if (flowSpec.getConfig().hasPath(ConfigurationKeys.FLOW_NAME_KEY)) {
+ jobSpec.setConfig(jobSpec.getConfig()
+ .withValue(ConfigurationKeys.JOB_NAME_KEY, flowSpec.getConfig().getValue(ConfigurationKeys.FLOW_NAME_KEY)));
+ }
+ if (flowSpec.getConfig().hasPath(ConfigurationKeys.FLOW_GROUP_KEY)) {
+ jobSpec.setConfig(jobSpec.getConfig()
+ .withValue(ConfigurationKeys.JOB_GROUP_KEY, flowSpec.getConfig().getValue(ConfigurationKeys.FLOW_GROUP_KEY)));
+ }
+
+ // Add flow execution id for this compilation
+ long flowExecutionId = System.currentTimeMillis();
+ jobSpec.setConfig(jobSpec.getConfig().withValue(ConfigurationKeys.FLOW_EXECUTION_ID_KEY,
+ ConfigValueFactory.fromAnyRef(flowExecutionId)));
+
+ // Reset properties in Spec from Config
+ jobSpec.setConfigAsProperties(ConfigUtils.configToProperties(jobSpec.getConfig()));
+ return jobSpec;
+ }
+
+ /**
+ * Ideally each edge has its own eligible template repository(Based on {@link SpecExecutor})
+ * to pick templates from.
+ *
+ * This function is to transform from all mixed templates ({@link #templateCatalog})
+ * into categorized {@link #edgeTemplateMap}.
+ *
+ */
+ abstract protected void populateEdgeTemplateMap();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/FlowEdgeProps.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/FlowEdgeProps.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/FlowEdgeProps.java
new file mode 100644
index 0000000..751bb09
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/FlowEdgeProps.java
@@ -0,0 +1,67 @@
+/*
+ * 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.flow;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import com.typesafe.config.ConfigObject;
+import java.io.Serializable;
+import java.util.Properties;
+
+import static org.apache.gobblin.service.ServiceConfigKeys.*;
+
+import lombok.Data;
+import lombok.Getter;
+import lombok.Setter;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class FlowEdgeProps {
+ protected static final boolean DEFAULT_EDGE_SAFETY = true;
+
+ /**
+ * Contains read-only properties that users want to package in.
+ */
+ @Getter
+ protected Config config;
+
+ /**
+ * One of the mutable properties of an edge.
+ */
+ @Getter
+ @Setter
+ private boolean isEdgeSecure;
+
+ public FlowEdgeProps(Config config) {
+ this.config = config;
+ isEdgeSecure = getInitialEdgeSafety();
+ }
+
+ public FlowEdgeProps() {
+ this(ConfigFactory.empty());
+ }
+
+ /**
+ * When initializing an edge, load and security value from properties will be used
+ * but could be overriden afterwards.
+ */
+ private boolean getInitialEdgeSafety() {
+ return
+ config.hasPath(EDGE_SECURITY_KEY) ? config.getBoolean(EDGE_SECURITY_KEY) : DEFAULT_EDGE_SAFETY;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/IdentityFlowToJobSpecCompiler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/IdentityFlowToJobSpecCompiler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/IdentityFlowToJobSpecCompiler.java
index 9e5fd11..3fb20a2 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/IdentityFlowToJobSpecCompiler.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/IdentityFlowToJobSpecCompiler.java
@@ -17,188 +17,82 @@
package org.apache.gobblin.service.modules.flow;
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collections;
-import java.util.List;
+
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
-import javax.annotation.Nonnull;
-import lombok.Getter;
-import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import com.codahale.metrics.Meter;
-import com.codahale.metrics.Timer;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
import com.typesafe.config.Config;
-import com.typesafe.config.ConfigValueFactory;
import org.apache.gobblin.annotation.Alpha;
-import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.instrumented.Instrumented;
-import org.apache.gobblin.metrics.MetricContext;
-import org.apache.gobblin.metrics.Tag;
import org.apache.gobblin.runtime.api.FlowSpec;
import org.apache.gobblin.runtime.api.JobSpec;
-import org.apache.gobblin.runtime.api.JobTemplate;
import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecCompiler;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
-import org.apache.gobblin.runtime.api.SpecNotFoundException;
import org.apache.gobblin.runtime.api.TopologySpec;
-import org.apache.gobblin.runtime.job_catalog.FSJobCatalog;
-import org.apache.gobblin.runtime.job_spec.ResolvedJobSpec;
import org.apache.gobblin.service.ServiceConfigKeys;
-import org.apache.gobblin.service.ServiceMetricNames;
-import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.apache.gobblin.runtime.api.SpecExecutor;
/***
* Take in a logical {@link Spec} ie flow and compile corresponding materialized job {@link Spec}
- * and its mapping to {@link SpecExecutorInstance}.
+ * and its mapping to {@link SpecExecutor}.
*/
@Alpha
-public class IdentityFlowToJobSpecCompiler implements SpecCompiler {
-
- private final Map<URI, TopologySpec> topologySpecMap;
- private final Config config;
- private final Logger log;
- private final Optional<FSJobCatalog> templateCatalog;
-
- protected final MetricContext metricContext;
- @Getter
- private Optional<Meter> flowCompilationSuccessFulMeter;
- @Getter
- private Optional<Meter> flowCompilationFailedMeter;
- @Getter
- private Optional<Timer> flowCompilationTimer;
+public class IdentityFlowToJobSpecCompiler extends BaseFlowToJobSpecCompiler {
public IdentityFlowToJobSpecCompiler(Config config) {
- this(config, true);
+ super(config, true);
}
public IdentityFlowToJobSpecCompiler(Config config, boolean instrumentationEnabled) {
- this(config, Optional.<Logger>absent(), instrumentationEnabled);
+ super(config, Optional.<Logger>absent(), instrumentationEnabled);
}
public IdentityFlowToJobSpecCompiler(Config config, Optional<Logger> log) {
- this(config, log, true);
+ super(config, log, true);
}
public IdentityFlowToJobSpecCompiler(Config config, Optional<Logger> log, boolean instrumentationEnabled) {
- this.log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
- if (instrumentationEnabled) {
- this.metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(config), IdentityFlowToJobSpecCompiler.class);
- this.flowCompilationSuccessFulMeter = Optional.of(this.metricContext.meter(ServiceMetricNames.FLOW_COMPILATION_SUCCESSFUL_METER));
- this.flowCompilationFailedMeter = Optional.of(this.metricContext.meter(ServiceMetricNames.FLOW_COMPILATION_FAILED_METER));
- this.flowCompilationTimer = Optional.<Timer>of(this.metricContext.timer(ServiceMetricNames.FLOW_COMPILATION_TIMER));
- }
- else {
- this.metricContext = null;
- this.flowCompilationSuccessFulMeter = Optional.absent();
- this.flowCompilationFailedMeter = Optional.absent();
- this.flowCompilationTimer = Optional.absent();
- }
-
- this.topologySpecMap = Maps.newConcurrentMap();
- this.config = config;
- /***
- * For multi-tenancy, the following needs to be added:
- * 1. Change singular templateCatalog to Map<URI, JobCatalogWithTemplates> to support multiple templateCatalogs
- * 2. Pick templateCatalog from JobCatalogWithTemplates based on URI, and try to resolve JobSpec using that
- */
- try {
- if (this.config.hasPath(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY)
- && StringUtils.isNotBlank(this.config.getString(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY))) {
- Config templateCatalogCfg = config
- .withValue(ConfigurationKeys.JOB_CONFIG_FILE_GENERAL_PATH_KEY,
- this.config.getValue(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY));
- this.templateCatalog = Optional.of(new FSJobCatalog(templateCatalogCfg));
- } else {
- this.templateCatalog = Optional.absent();
- }
- } catch (IOException e) {
- throw new RuntimeException("Could not initialize IdentityFlowToJobSpecCompiler because of "
- + "TemplateCatalog initialization failure", e);
- }
+ super(config, log, instrumentationEnabled);
}
@Override
- public Map<Spec, SpecExecutorInstanceProducer> compileFlow(Spec spec) {
+ public Map<Spec, SpecExecutor> compileFlow(Spec spec) {
Preconditions.checkNotNull(spec);
Preconditions.checkArgument(spec instanceof FlowSpec, "IdentityFlowToJobSpecCompiler only converts FlowSpec to JobSpec");
long startTime = System.nanoTime();
- Map<Spec, SpecExecutorInstanceProducer> specExecutorInstanceMap = Maps.newLinkedHashMap();
+ Map<Spec, SpecExecutor> specExecutorMap = Maps.newLinkedHashMap();
FlowSpec flowSpec = (FlowSpec) spec;
String source = flowSpec.getConfig().getString(ServiceConfigKeys.FLOW_SOURCE_IDENTIFIER_KEY);
String destination = flowSpec.getConfig().getString(ServiceConfigKeys.FLOW_DESTINATION_IDENTIFIER_KEY);
log.info(String.format("Compiling flow for source: %s and destination: %s", source, destination));
- JobSpec jobSpec;
- JobSpec.Builder jobSpecBuilder = JobSpec.builder(flowSpec.getUri())
- .withConfig(flowSpec.getConfig())
- .withDescription(flowSpec.getDescription())
- .withVersion(flowSpec.getVersion());
-
- if (flowSpec.getTemplateURIs().isPresent() && templateCatalog.isPresent()) {
- // Only first template uri will be honored for Identity
- jobSpecBuilder = jobSpecBuilder.withTemplate(flowSpec.getTemplateURIs().get().iterator().next());
- try {
- jobSpec = new ResolvedJobSpec(jobSpecBuilder.build(), templateCatalog.get());
- log.info("Resolved JobSpec properties are: " + jobSpec.getConfigAsProperties());
- } catch (SpecNotFoundException | JobTemplate.TemplateException e) {
- throw new RuntimeException("Could not resolve template in JobSpec from TemplateCatalog", e);
- }
- } else {
- jobSpec = jobSpecBuilder.build();
- log.info("Unresolved JobSpec properties are: " + jobSpec.getConfigAsProperties());
- }
-
- // Remove schedule
- jobSpec.setConfig(jobSpec.getConfig().withoutPath(ConfigurationKeys.JOB_SCHEDULE_KEY));
-
- // Add job.name and job.group
- if (flowSpec.getConfig().hasPath(ConfigurationKeys.FLOW_NAME_KEY)) {
- jobSpec.setConfig(jobSpec.getConfig()
- .withValue(ConfigurationKeys.JOB_NAME_KEY, flowSpec.getConfig().getValue(ConfigurationKeys.FLOW_NAME_KEY)));
- }
- if (flowSpec.getConfig().hasPath(ConfigurationKeys.FLOW_GROUP_KEY)) {
- jobSpec.setConfig(jobSpec.getConfig()
- .withValue(ConfigurationKeys.JOB_GROUP_KEY, flowSpec.getConfig().getValue(ConfigurationKeys.FLOW_GROUP_KEY)));
- }
-
- // Add flow execution id for this compilation
- long flowExecutionId = System.currentTimeMillis();
- jobSpec.setConfig(jobSpec.getConfig().withValue(ConfigurationKeys.FLOW_EXECUTION_ID_KEY,
- ConfigValueFactory.fromAnyRef(flowExecutionId)));
-
- // Reset properties in Spec from Config
- jobSpec.setConfigAsProperties(ConfigUtils.configToProperties(jobSpec.getConfig()));
+ JobSpec jobSpec = jobSpecGenerator(flowSpec);
for (TopologySpec topologySpec : topologySpecMap.values()) {
try {
- Map<String, String> capabilities = (Map<String, String>) topologySpec.getSpecExecutorInstanceProducer().getCapabilities().get();
- for (Map.Entry<String, String> capability : capabilities.entrySet()) {
+ Map<ServiceNode, ServiceNode> capabilities = (Map<ServiceNode, ServiceNode>) topologySpec.getSpecExecutor().getCapabilities().get();
+ for (Map.Entry<ServiceNode, ServiceNode> capability : capabilities.entrySet()) {
log.info(String.format("Evaluating current JobSpec: %s against TopologySpec: %s with "
- + "capability of source: %s and destination: %s ", jobSpec.getUri(),
+ + "capability of source: %s and destination: %s ", jobSpec.getUri(),
topologySpec.getUri(), capability.getKey(), capability.getValue()));
- if (source.equals(capability.getKey()) && destination.equals(capability.getValue())) {
- specExecutorInstanceMap.put(jobSpec, topologySpec.getSpecExecutorInstanceProducer());
+ if (source.equals(capability.getKey().getNodeName()) && destination.equals(capability.getValue().getNodeName())) {
+ specExecutorMap.put(jobSpec, topologySpec.getSpecExecutor());
log.info(String.format("Current JobSpec: %s is executable on TopologySpec: %s. Added TopologySpec as candidate.",
jobSpec.getUri(), topologySpec.getUri()));
log.info("Since we found a candidate executor, we will not try to compute more. "
+ "(Intended limitation for IdentityFlowToJobSpecCompiler)");
- return specExecutorInstanceMap;
+ return specExecutorMap;
}
}
} catch (InterruptedException | ExecutionException e) {
@@ -209,52 +103,12 @@ public class IdentityFlowToJobSpecCompiler implements SpecCompiler {
Instrumented.markMeter(this.flowCompilationSuccessFulMeter);
Instrumented.updateTimer(this.flowCompilationTimer, System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
- return specExecutorInstanceMap;
- }
-
- @Override
- public Map<URI, TopologySpec> getTopologySpecMap() {
- return this.topologySpecMap;
- }
-
- @Override
- public void onAddSpec(Spec addedSpec) {
- topologySpecMap.put(addedSpec.getUri(), (TopologySpec) addedSpec);
- }
-
- @Override
- public void onDeleteSpec(URI deletedSpecURI, String deletedSpecVersion) {
- topologySpecMap.remove(deletedSpecURI);
- }
-
- @Override
- public void onUpdateSpec(Spec updatedSpec) {
- topologySpecMap.put(updatedSpec.getUri(), (TopologySpec) updatedSpec);
- }
-
- @Nonnull
- @Override
- public MetricContext getMetricContext() {
- return this.metricContext;
- }
-
- @Override
- public boolean isInstrumentationEnabled() {
- return null != this.metricContext;
- }
-
- @Override
- public List<Tag<?>> generateTags(org.apache.gobblin.configuration.State state) {
- return Collections.emptyList();
- }
-
- @Override
- public void switchMetricContext(List<Tag<?>> tags) {
- throw new UnsupportedOperationException();
+ return specExecutorMap;
}
@Override
- public void switchMetricContext(MetricContext context) {
- throw new UnsupportedOperationException();
+ protected void populateEdgeTemplateMap() {
+ log.warn("No population of templates based on edge happen in this implementation");
+ return;
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/LoadBasedFlowEdgeImpl.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/LoadBasedFlowEdgeImpl.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/LoadBasedFlowEdgeImpl.java
new file mode 100644
index 0000000..83b94e3
--- /dev/null
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/LoadBasedFlowEdgeImpl.java
@@ -0,0 +1,180 @@
+/*
+ * 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.flow;
+
+import com.typesafe.config.Config;
+
+import org.jgrapht.graph.DefaultWeightedEdge;
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.api.FlowEdge;
+import org.apache.gobblin.runtime.api.ServiceNode;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+
+import lombok.Getter;
+
+/**
+ * A base implementation of a flowEdge in the weight multi-edge graph.
+ * For a weightedMultiGraph there could be multiple edges between two vertices.
+ * Recall that a triplet of <SourceNode, targetNode, specExecutor> determines one edge.
+ * It is expected that {@link org.jgrapht.graph.DirectedWeightedMultigraph#getAllEdges(Object, Object)}
+ * can return multiple edges with the same pair of source and destination but different SpecExecutor.
+ *
+ * Each edge has a {@FlowEdgeProp} which contains mutable and immutable properties.
+ * The {@link LoadBasedFlowEdgeImpl} exposes two mutable properties: Load and Security.
+ *
+ * Load of an edge is equivalent to weight defined in {@link DefaultWeightedEdge}.
+ * Since {@link #getWeight()} method is protected, {@link #getEdgeLoad()} will return the load.
+ * There's no setLoad, which is logically supposed to happen by invoking
+ * {@link org.jgrapht.graph.DirectedWeightedMultigraph#setEdgeWeight(Object, double)}.
+ *
+ * Security of an edge describes if an edge is secure to be part of data movement path at current stage.
+ *
+ */
+@Alpha
+public class LoadBasedFlowEdgeImpl extends DefaultWeightedEdge implements FlowEdge {
+
+ /**
+ * In our cases {@link LoadBasedFlowEdgeImpl} is not likely to be serialized.
+ * While as it extends {@link DefaultWeightedEdge} for best practice we made all fields transient,
+ * and specify serialVersionUID.
+ */
+ private static final long serialVersionUID = 1L;
+
+ @Getter
+ private transient ServiceNode sourceNode;
+ @Getter
+ private transient ServiceNode targetNode;
+ @Getter
+ private transient SpecExecutor specExecutorInstance;
+
+ /**
+ * Contains both read-only and mutable attributes of properties of an edge.
+ * Mutable properties in{@link FlowEdgeProps} expose their Setter & Getter
+ * thru. either the {@link FlowEdgeProps}
+ * or graph-level api, e.g. {@link org.jgrapht.graph.DirectedWeightedMultigraph#setEdgeWeight(Object, double)}
+ *
+ * Typical mutable properties of an edge includes:
+ * Load(Weight), Security.
+ */
+ private final transient FlowEdgeProps flowEdgeProps;
+
+ public LoadBasedFlowEdgeImpl(ServiceNode sourceNode, ServiceNode targetNode,
+ FlowEdgeProps flowEdgeProps, SpecExecutor specExecutorInstance) {
+ this.sourceNode = sourceNode;
+ this.targetNode = targetNode;
+ this.flowEdgeProps = flowEdgeProps;
+ this.specExecutorInstance = specExecutorInstance;
+ }
+
+ public LoadBasedFlowEdgeImpl(ServiceNode sourceNode, ServiceNode targetNode,
+ SpecExecutor specExecutor) {
+ this(sourceNode, targetNode, new FlowEdgeProps(specExecutor.getAttrs()),
+ specExecutor);
+ }
+
+ // Load: Directly using {@link DefaultWeightedEdge}'s weight field.
+ /**
+ * Load:
+ * Initialization: super's default constructor
+ * Getter: {@link #getEdgeLoad()}} thru. {@link DefaultWeightedEdge}'s {@link #getWeight()}.
+ * Setter:Thru. {@link org.jgrapht.graph.DirectedWeightedMultigraph#setEdgeWeight(Object, double)}
+ */
+ public double getEdgeLoad() {
+ return getWeight();
+ }
+
+ // Security: Get/Set thru. FlowEdgeProps
+ /**
+ * Initialization\Getter\Setter: By {@link FlowEdgeProps}
+ */
+ public boolean getIsEdgeSecure() {
+ return flowEdgeProps.isEdgeSecure();
+ }
+ public void setIsEdgeSecure(boolean isEdgeSecure) {
+ this.flowEdgeProps.setEdgeSecure(isEdgeSecure);
+ }
+
+
+ @Override
+ public String getEdgeIdentity() {
+ return this.calculateEdgeIdentity(this.sourceNode, this.targetNode, this.specExecutorInstance);
+ }
+
+ @Override
+ public Config getEdgeProperties() {
+ return this.flowEdgeProps.getConfig();
+ }
+
+ @Override
+ /**
+ * Naive rule: If edge is secure, then it is qualified to be considered in path-finding.
+ */
+ public boolean isEdgeEnabled() {
+ return this.flowEdgeProps.isEdgeSecure();
+ }
+
+
+ /**
+ * A naive implementation of edge identity calculation.
+ * @return
+ */
+ public static String calculateEdgeIdentity(ServiceNode sourceNode, ServiceNode targetNode, SpecExecutor specExecutorInstance){
+ return sourceNode.getNodeName() + "-" + specExecutorInstance.getUri() + "-" + targetNode.getNodeName();
+ }
+
+ /**
+ * Recall that we need a triplet to uniquely define a {@link FlowEdge}:
+ * - {@link ServiceNode} sourceNode
+ * - {@link ServiceNode} targetNode
+ * - {@link SpecExecutor} SpecExecutor
+ *
+ * We DO NOT distinguish between two edges by other props like weight,
+ * as the load should be an attribute of an edge.
+ * These are IntelliJ-generated methods for equals and hashCode().
+ *
+ * @param o The object that being compared
+ * @return If two {@link LoadBasedFlowEdgeImpl} are equivalent.
+ */
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ LoadBasedFlowEdgeImpl that = (LoadBasedFlowEdgeImpl) o;
+
+ if (!sourceNode.equals(that.sourceNode)) {
+ return false;
+ }
+ if (!targetNode.equals(that.targetNode)) {
+ return false;
+ }
+ return specExecutorInstance.equals(that.specExecutorInstance);
+ }
+
+ @Override
+ public int hashCode() {
+ int result = sourceNode.hashCode();
+ result = 31 * result + targetNode.hashCode();
+ result = 31 * result + specExecutorInstance.hashCode();
+ return result;
+ }
+}
\ No newline at end of file
[3/4] incubator-gobblin git commit: [GOBBLIN-3] Multi-hop flow
compiler implementation
Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstance.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstance.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstance.java
deleted file mode 100644
index a60d8e2..0000000
--- a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstance.java
+++ /dev/null
@@ -1,131 +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 com.google.common.util.concurrent.AbstractIdleService;
-import java.io.Serializable;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Future;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Splitter;
-import com.google.common.collect.Maps;
-import com.typesafe.config.Config;
-
-import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.util.CompletedFuture;
-import org.apache.gobblin.util.ConfigUtils;
-
-
-public class SimpleKafkaSpecExecutorInstance extends AbstractIdleService implements SpecExecutorInstance {
- public static final String SPEC_KAFKA_TOPICS_KEY = "spec.kafka.topics";
- protected static final Splitter SPLIT_BY_COMMA = Splitter.on(",").omitEmptyStrings().trimResults();
- protected static final Splitter SPLIT_BY_COLON = Splitter.on(":").omitEmptyStrings().trimResults();
-
- // Executor Instance
- protected final Config _config;
- protected final Logger _log;
- protected final URI _specExecutorInstanceUri;
- protected final Map<String, String> _capabilities;
-
- protected static final String VERB_KEY = "Verb";
-
- public SimpleKafkaSpecExecutorInstance(Config config, Optional<Logger> log) {
- _config = config;
- _log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
- try {
- _specExecutorInstanceUri = new URI(ConfigUtils.getString(config, ConfigurationKeys.SPECEXECUTOR_INSTANCE_URI_KEY,
- "NA"));
- } catch (URISyntaxException e) {
- throw new RuntimeException(e);
- }
- _capabilities = Maps.newHashMap();
- if (config.hasPath(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY)) {
- String capabilitiesStr = config.getString(ConfigurationKeys.SPECEXECUTOR_INSTANCE_CAPABILITIES_KEY);
- List<String> capabilities = SPLIT_BY_COMMA.splitToList(capabilitiesStr);
- for (String capability : capabilities) {
- List<String> currentCapability = SPLIT_BY_COLON.splitToList(capability);
- Preconditions.checkArgument(currentCapability.size() == 2, "Only one source:destination pair is supported "
- + "per capability, found: " + currentCapability);
- _capabilities.put(currentCapability.get(0), currentCapability.get(1));
- }
- }
- }
-
- @Override
- public URI getUri() {
- return _specExecutorInstanceUri;
- }
-
- @Override
- public Future<String> getDescription() {
- return new CompletedFuture<>("SimpleSpecExecutorInstance with URI: " + _specExecutorInstanceUri, null);
- }
-
- @Override
- public Future<Config> getConfig() {
- return new CompletedFuture<>(_config, null);
- }
-
- @Override
- public Future<String> getHealth() {
- return new CompletedFuture<>("Healthy", null);
- }
-
- @Override
- public Future<? extends Map<String, String>> getCapabilities() {
- return new CompletedFuture<>(_capabilities, null);
- }
-
- @Override
- protected void startUp() throws Exception {
- // nothing to do in default implementation
- }
-
- @Override
- protected void shutDown() throws Exception {
- // nothing to do in default implementation
- }
-
- public static class SpecExecutorInstanceDataPacket implements Serializable {
-
- protected Verb _verb;
- protected URI _uri;
- protected Spec _spec;
-
- public SpecExecutorInstanceDataPacket(Verb verb, URI uri, Spec spec) {
- _verb = verb;
- _uri = uri;
- _spec = spec;
- }
-
- @Override
- public String toString() {
- return String.format("Verb: %s, URI: %s, Spec: %s", _verb, _uri, _spec);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceConsumer.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceConsumer.java
deleted file mode 100644
index 90960e7..0000000
--- a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceConsumer.java
+++ /dev/null
@@ -1,261 +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.ByteArrayInputStream;
-import java.io.Closeable;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.Future;
-import java.util.regex.Pattern;
-
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.Decoder;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.typesafe.config.Config;
-
-import org.apache.gobblin.kafka.client.ByteArrayBasedKafkaRecord;
-import org.apache.gobblin.kafka.client.DecodeableKafkaRecord;
-import org.apache.gobblin.kafka.client.GobblinKafkaConsumerClient;
-import org.apache.gobblin.kafka.client.Kafka08ConsumerClient;
-import org.apache.gobblin.kafka.client.KafkaConsumerRecord;
-import org.apache.gobblin.metrics.reporter.util.FixedSchemaVersionWriter;
-import org.apache.gobblin.metrics.reporter.util.SchemaVersionWriter;
-import org.apache.gobblin.runtime.api.JobSpec;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceConsumer;
-import org.apache.gobblin.runtime.job_spec.AvroJobSpec;
-import org.apache.gobblin.source.extractor.extract.kafka.KafkaOffsetRetrievalFailureException;
-import org.apache.gobblin.source.extractor.extract.kafka.KafkaPartition;
-import org.apache.gobblin.source.extractor.extract.kafka.KafkaTopic;
-import org.apache.gobblin.util.CompletedFuture;
-
-public class SimpleKafkaSpecExecutorInstanceConsumer extends SimpleKafkaSpecExecutorInstance
- implements SpecExecutorInstanceConsumer<Spec>, Closeable {
-
- // Consumer
- protected final GobblinKafkaConsumerClient _kafka08Consumer;
- protected final List<KafkaPartition> _partitions;
- protected final List<Long> _lowWatermark;
- protected final List<Long> _nextWatermark;
- protected final List<Long> _highWatermark;
-
- private Iterator<KafkaConsumerRecord> messageIterator = null;
- private int currentPartitionIdx = -1;
- private boolean isFirstRun = true;
-
- private final BinaryDecoder _decoder;
- private final SpecificDatumReader<AvroJobSpec> _reader;
- private final SchemaVersionWriter<?> _versionWriter;
-
- public SimpleKafkaSpecExecutorInstanceConsumer(Config config, Optional<Logger> log) {
- super(config, log);
-
- // Consumer
- _kafka08Consumer = new Kafka08ConsumerClient.Factory().create(config);
- List<KafkaTopic> kafkaTopics = _kafka08Consumer.getFilteredTopics(Collections.EMPTY_LIST,
- Lists.newArrayList(Pattern.compile(config.getString(SPEC_KAFKA_TOPICS_KEY))));
- _partitions = kafkaTopics.get(0).getPartitions();
- _lowWatermark = Lists.newArrayList(Collections.nCopies(_partitions.size(), 0L));
- _nextWatermark = Lists.newArrayList(Collections.nCopies(_partitions.size(), 0L));
- _highWatermark = Lists.newArrayList(Collections.nCopies(_partitions.size(), 0L));
-
- InputStream dummyInputStream = new ByteArrayInputStream(new byte[0]);
- _decoder = DecoderFactory.get().binaryDecoder(dummyInputStream, null);
- _reader = new SpecificDatumReader<AvroJobSpec>(AvroJobSpec.SCHEMA$);
- _versionWriter = new FixedSchemaVersionWriter();
- }
-
- public SimpleKafkaSpecExecutorInstanceConsumer(Config config, Logger log) {
- this(config, Optional.of(log));
- }
-
- /** Constructor with no logging */
- public SimpleKafkaSpecExecutorInstanceConsumer(Config config) {
- this(config, Optional.<Logger>absent());
- }
-
- @Override
- public Future<? extends List<Pair<Verb, Spec>>> changedSpecs() {
- List<Pair<Verb, Spec>> changesSpecs = new ArrayList<>();
- initializeWatermarks();
- this.currentPartitionIdx = -1;
- while (!allPartitionsFinished()) {
- if (currentPartitionFinished()) {
- moveToNextPartition();
- continue;
- }
- if (this.messageIterator == null || !this.messageIterator.hasNext()) {
- try {
- this.messageIterator = fetchNextMessageBuffer();
- } catch (Exception e) {
- _log.error(String.format("Failed to fetch next message buffer for partition %s. Will skip this partition.",
- getCurrentPartition()), e);
- moveToNextPartition();
- continue;
- }
- if (this.messageIterator == null || !this.messageIterator.hasNext()) {
- moveToNextPartition();
- continue;
- }
- }
- while (!currentPartitionFinished()) {
- if (!this.messageIterator.hasNext()) {
- break;
- }
-
- KafkaConsumerRecord nextValidMessage = this.messageIterator.next();
-
- // Even though we ask Kafka to give us a message buffer starting from offset x, it may
- // return a buffer that starts from offset smaller than x, so we need to skip messages
- // until we get to x.
- if (nextValidMessage.getOffset() < _nextWatermark.get(this.currentPartitionIdx)) {
- continue;
- }
-
- _nextWatermark.set(this.currentPartitionIdx, nextValidMessage.getNextOffset());
- try {
- final AvroJobSpec record;
-
- if (nextValidMessage instanceof ByteArrayBasedKafkaRecord) {
- record = decodeRecord((ByteArrayBasedKafkaRecord)nextValidMessage);
- } else if (nextValidMessage instanceof DecodeableKafkaRecord){
- record = ((DecodeableKafkaRecord<?, AvroJobSpec>) nextValidMessage).getValue();
- } else {
- throw new IllegalStateException(
- "Unsupported KafkaConsumerRecord type. The returned record can either be ByteArrayBasedKafkaRecord"
- + " or DecodeableKafkaRecord");
- }
-
- JobSpec.Builder jobSpecBuilder = JobSpec.builder(record.getUri());
-
- Properties props = new Properties();
- props.putAll(record.getProperties());
- jobSpecBuilder.withJobCatalogURI(record.getUri()).withVersion(record.getVersion())
- .withDescription(record.getDescription()).withConfigAsProperties(props);
-
- if (!record.getTemplateUri().isEmpty()) {
- jobSpecBuilder.withTemplate(new URI(record.getTemplateUri()));
- }
-
- String verbName = record.getMetadata().get(VERB_KEY);
- Verb verb = Verb.valueOf(verbName);
-
- changesSpecs.add(new ImmutablePair<Verb, Spec>(verb, jobSpecBuilder.build()));
- } catch (Throwable t) {
- _log.error("Could not decode record at partition " + this.currentPartitionIdx +
- " offset " + nextValidMessage.getOffset());
- }
- }
- }
-
- return new CompletedFuture(changesSpecs, null);
- }
-
- private void initializeWatermarks() {
- initializeLowWatermarks();
- initializeHighWatermarks();
- }
-
- private void initializeLowWatermarks() {
- try {
- int i=0;
- for (KafkaPartition kafkaPartition : _partitions) {
- if (isFirstRun) {
- long earliestOffset = _kafka08Consumer.getEarliestOffset(kafkaPartition);
- _lowWatermark.set(i, earliestOffset);
- } else {
- _lowWatermark.set(i, _highWatermark.get(i));
- }
- i++;
- }
- isFirstRun = false;
- } catch (KafkaOffsetRetrievalFailureException e) {
- throw new RuntimeException(e);
- }
- }
-
- private void initializeHighWatermarks() {
- try {
- int i=0;
- for (KafkaPartition kafkaPartition : _partitions) {
- long latestOffset = _kafka08Consumer.getLatestOffset(kafkaPartition);
- _highWatermark.set(i, latestOffset);
- i++;
- }
- } catch (KafkaOffsetRetrievalFailureException e) {
- throw new RuntimeException(e);
- }
- }
-
- private boolean allPartitionsFinished() {
- return this.currentPartitionIdx >= _nextWatermark.size();
- }
-
- private boolean currentPartitionFinished() {
- if (this.currentPartitionIdx == -1) {
- return true;
- } else if (_nextWatermark.get(this.currentPartitionIdx) >= _highWatermark.get(this.currentPartitionIdx)) {
- return true;
- } else {
- return false;
- }
- }
-
- private int moveToNextPartition() {
- this.messageIterator = null;
- return this.currentPartitionIdx ++;
- }
-
- private KafkaPartition getCurrentPartition() {
- return _partitions.get(this.currentPartitionIdx);
- }
-
- private Iterator<KafkaConsumerRecord> fetchNextMessageBuffer() {
- return _kafka08Consumer.consume(_partitions.get(this.currentPartitionIdx),
- _nextWatermark.get(this.currentPartitionIdx), _highWatermark.get(this.currentPartitionIdx));
- }
-
- private AvroJobSpec decodeRecord(ByteArrayBasedKafkaRecord kafkaConsumerRecord) throws IOException {
- InputStream is = new ByteArrayInputStream(kafkaConsumerRecord.getMessageBytes());
- _versionWriter.readSchemaVersioningInformation(new DataInputStream(is));
-
- Decoder decoder = DecoderFactory.get().binaryDecoder(is, _decoder);
-
- return _reader.read(null, decoder);
- }
-
- @Override
- public void close() throws IOException {
- _kafka08Consumer.close();
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceProducer.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceProducer.java
deleted file mode 100644
index cdafe06..0000000
--- a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceProducer.java
+++ /dev/null
@@ -1,139 +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.net.URI;
-import java.util.List;
-import java.util.concurrent.Future;
-
-import javax.annotation.concurrent.NotThreadSafe;
-
-import org.apache.avro.mapred.AvroJob;
-import org.slf4j.Logger;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
-import com.typesafe.config.Config;
-
-import org.apache.gobblin.kafka.writer.Kafka08DataWriter;
-import org.apache.gobblin.metrics.reporter.util.AvroBinarySerializer;
-import org.apache.gobblin.metrics.reporter.util.AvroSerializer;
-import org.apache.gobblin.metrics.reporter.util.FixedSchemaVersionWriter;
-import org.apache.gobblin.runtime.api.JobSpec;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceProducer;
-import org.apache.gobblin.runtime.job_spec.AvroJobSpec;
-import org.apache.gobblin.util.ConfigUtils;
-import org.apache.gobblin.writer.WriteCallback;
-
-
-@NotThreadSafe
-public class SimpleKafkaSpecExecutorInstanceProducer extends SimpleKafkaSpecExecutorInstance
- implements SpecExecutorInstanceProducer<Spec>, Closeable {
-
- // Producer
- protected Kafka08DataWriter<byte[]> _kafka08Producer;
- private final AvroSerializer<AvroJobSpec> _serializer;
-
- public SimpleKafkaSpecExecutorInstanceProducer(Config config, Optional<Logger> log) {
- super(config, log);
-
- try {
- _serializer = new AvroBinarySerializer<>(AvroJobSpec.SCHEMA$, new FixedSchemaVersionWriter());
- } catch (IOException e) {
- throw new RuntimeException("Could not create AvroBinarySerializer", e);
- }
- }
-
- public SimpleKafkaSpecExecutorInstanceProducer(Config config, Logger log) {
- this(config, Optional.of(log));
- }
-
- /** Constructor with no logging */
- public SimpleKafkaSpecExecutorInstanceProducer(Config config) {
- this(config, Optional.<Logger>absent());
- }
-
- @Override
- public Future<?> addSpec(Spec addedSpec) {
- AvroJobSpec avroJobSpec = convertToAvroJobSpec(addedSpec, Verb.ADD);
-
- _log.info("Adding Spec: " + addedSpec + " using Kafka.");
-
- return getKafka08Producer().write(_serializer.serializeRecord(avroJobSpec), WriteCallback.EMPTY);
- }
-
- @Override
- public Future<?> updateSpec(Spec updatedSpec) {
- AvroJobSpec avroJobSpec = convertToAvroJobSpec(updatedSpec, Verb.UPDATE);
-
- _log.info("Updating Spec: " + updatedSpec + " using Kafka.");
-
- return getKafka08Producer().write(_serializer.serializeRecord(avroJobSpec), WriteCallback.EMPTY);
- }
-
- @Override
- public Future<?> deleteSpec(URI deletedSpecURI) {
-
- AvroJobSpec avroJobSpec = AvroJobSpec.newBuilder().setUri(deletedSpecURI.toString())
- .setMetadata(ImmutableMap.of(VERB_KEY, Verb.DELETE.name())).build();
-
- _log.info("Deleting Spec: " + deletedSpecURI + " using Kafka.");
-
- return getKafka08Producer().write(_serializer.serializeRecord(avroJobSpec), WriteCallback.EMPTY);
- }
-
- @Override
- public Future<? extends List<Spec>> listSpecs() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void close() throws IOException {
- _kafka08Producer.close();
- }
-
- private Kafka08DataWriter<byte[]> getKafka08Producer() {
- if (null == _kafka08Producer) {
- _kafka08Producer = new Kafka08DataWriter<byte[]>(ConfigUtils.configToProperties(_config));
- }
- return _kafka08Producer;
- }
-
- private AvroJobSpec convertToAvroJobSpec(Spec spec, Verb verb) {
- if (spec instanceof JobSpec) {
- JobSpec jobSpec = (JobSpec) spec;
- AvroJobSpec.Builder avroJobSpecBuilder = AvroJobSpec.newBuilder();
-
- avroJobSpecBuilder.setUri(jobSpec.getUri().toString()).setVersion(jobSpec.getVersion())
- .setDescription(jobSpec.getDescription()).setProperties(Maps.fromProperties(jobSpec.getConfigAsProperties()))
- .setMetadata(ImmutableMap.of(VERB_KEY, verb.name()));
-
- if (jobSpec.getTemplateURI().isPresent()) {
- avroJobSpecBuilder.setTemplateUri(jobSpec.getTemplateURI().get().toString());
- }
-
- return avroJobSpecBuilder.build();
- } else {
- throw new RuntimeException("Unsupported spec type " + spec.getClass());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecProducer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecProducer.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecProducer.java
new file mode 100644
index 0000000..13aae6f
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/SimpleKafkaSpecProducer.java
@@ -0,0 +1,140 @@
+/*
+ * 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.net.URI;
+import java.util.List;
+import java.util.concurrent.Future;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.typesafe.config.Config;
+
+import org.slf4j.Logger;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.kafka.writer.Kafka08DataWriter;
+import org.apache.gobblin.metrics.reporter.util.AvroBinarySerializer;
+import org.apache.gobblin.metrics.reporter.util.AvroSerializer;
+import org.apache.gobblin.metrics.reporter.util.FixedSchemaVersionWriter;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.job_spec.AvroJobSpec;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.writer.WriteCallback;
+import static org.apache.gobblin.service.SimpleKafkaSpecExecutor.*;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@NotThreadSafe
+public class SimpleKafkaSpecProducer implements SpecProducer<Spec>, Closeable {
+
+ // Producer
+ protected Kafka08DataWriter<byte[]> _kafka08Producer;
+ private final AvroSerializer<AvroJobSpec> _serializer;
+ private Config _config;
+
+ public SimpleKafkaSpecProducer(Config config, Optional<Logger> log) {
+
+ try {
+ _serializer = new AvroBinarySerializer<>(AvroJobSpec.SCHEMA$, new FixedSchemaVersionWriter());
+ _config = config;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not create AvroBinarySerializer", e);
+ }
+ }
+
+ public SimpleKafkaSpecProducer(Config config, Logger log) {
+ this(config, Optional.of(log));
+ }
+
+ /** Constructor with no logging */
+ public SimpleKafkaSpecProducer(Config config) {
+ this(config, Optional.<Logger>absent());
+ }
+
+ @Override
+ public Future<?> addSpec(Spec addedSpec) {
+ AvroJobSpec avroJobSpec = convertToAvroJobSpec(addedSpec, SpecExecutor.Verb.ADD);
+
+ log.info("Adding Spec: " + addedSpec + " using Kafka.");
+
+ return getKafka08Producer().write(_serializer.serializeRecord(avroJobSpec), WriteCallback.EMPTY);
+ }
+
+ @Override
+ public Future<?> updateSpec(Spec updatedSpec) {
+ AvroJobSpec avroJobSpec = convertToAvroJobSpec(updatedSpec, SpecExecutor.Verb.UPDATE);
+
+ log.info("Updating Spec: " + updatedSpec + " using Kafka.");
+
+ return getKafka08Producer().write(_serializer.serializeRecord(avroJobSpec), WriteCallback.EMPTY);
+ }
+
+ @Override
+ public Future<?> deleteSpec(URI deletedSpecURI) {
+
+ AvroJobSpec avroJobSpec = AvroJobSpec.newBuilder().setUri(deletedSpecURI.toString())
+ .setMetadata(ImmutableMap.of(VERB_KEY, Verb.DELETE.name())).build();
+
+ log.info("Deleting Spec: " + deletedSpecURI + " using Kafka.");
+
+ return getKafka08Producer().write(_serializer.serializeRecord(avroJobSpec), WriteCallback.EMPTY);
+ }
+
+ @Override
+ public Future<? extends List<Spec>> listSpecs() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void close() throws IOException {
+ _kafka08Producer.close();
+ }
+
+ private Kafka08DataWriter<byte[]> getKafka08Producer() {
+ if (null == _kafka08Producer) {
+ _kafka08Producer = new Kafka08DataWriter<byte[]>(ConfigUtils.configToProperties(_config));
+ }
+ return _kafka08Producer;
+ }
+
+ private AvroJobSpec convertToAvroJobSpec(Spec spec, Verb verb) {
+ if (spec instanceof JobSpec) {
+ JobSpec jobSpec = (JobSpec) spec;
+ AvroJobSpec.Builder avroJobSpecBuilder = AvroJobSpec.newBuilder();
+
+ avroJobSpecBuilder.setUri(jobSpec.getUri().toString()).setVersion(jobSpec.getVersion())
+ .setDescription(jobSpec.getDescription()).setProperties(Maps.fromProperties(jobSpec.getConfigAsProperties()))
+ .setMetadata(ImmutableMap.of(VERB_KEY, verb.name()));
+
+ if (jobSpec.getTemplateURI().isPresent()) {
+ avroJobSpecBuilder.setTemplateUri(jobSpec.getTemplateURI().get().toString());
+ }
+
+ return avroJobSpecBuilder.build();
+ } else {
+ throw new RuntimeException("Unsupported spec type " + spec.getClass());
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
new file mode 100644
index 0000000..fd42211
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
@@ -0,0 +1,173 @@
+/*
+ * 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.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import com.google.common.util.concurrent.AbstractIdleService;
+
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.MutableJobCatalog;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecConsumer;
+import org.apache.gobblin.runtime.job_monitor.AvroJobSpecKafkaJobMonitor;
+import org.apache.gobblin.runtime.job_monitor.KafkaJobMonitor;
+import org.apache.gobblin.runtime.std.DefaultJobCatalogListenerImpl;
+import org.apache.gobblin.util.CompletedFuture;
+import org.apache.gobblin.util.ConfigUtils;
+import static org.apache.gobblin.service.SimpleKafkaSpecExecutor.*;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+/**
+ * SpecConsumer that consumes from kafka in a streaming manner
+ * Implemented {@link AbstractIdleService} for starting up and shutting down.
+ */
+public class StreamingKafkaSpecConsumer extends AbstractIdleService implements SpecConsumer<Spec>, Closeable {
+ public static final String SPEC_STREAMING_BLOCKING_QUEUE_SIZE = "spec.StreamingBlockingQueueSize";
+ private static final int DEFAULT_SPEC_STREAMING_BLOCKING_QUEUE_SIZE = 100;
+ private final AvroJobSpecKafkaJobMonitor _jobMonitor;
+ private final BlockingQueue<ImmutablePair<Verb, Spec>> _jobSpecQueue;
+
+ public StreamingKafkaSpecConsumer(Config config, MutableJobCatalog jobCatalog, Optional<Logger> log) {
+ String topic = config.getString(SPEC_KAFKA_TOPICS_KEY);
+ Config defaults = ConfigFactory.parseMap(ImmutableMap.of(AvroJobSpecKafkaJobMonitor.TOPIC_KEY, topic,
+ KafkaJobMonitor.KAFKA_AUTO_OFFSET_RESET_KEY, KafkaJobMonitor.KAFKA_AUTO_OFFSET_RESET_SMALLEST));
+
+ try {
+ _jobMonitor = (AvroJobSpecKafkaJobMonitor)(new AvroJobSpecKafkaJobMonitor.Factory())
+ .forConfig(config.withFallback(defaults), jobCatalog);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not create job monitor", e);
+ }
+
+ _jobSpecQueue = new LinkedBlockingQueue<>(ConfigUtils.getInt(config, "SPEC_STREAMING_BLOCKING_QUEUE_SIZE",
+ DEFAULT_SPEC_STREAMING_BLOCKING_QUEUE_SIZE));
+
+ // listener will add job specs to a blocking queue to send to callers of changedSpecs()
+ jobCatalog.addListener(new JobSpecListener());
+ }
+
+ public StreamingKafkaSpecConsumer(Config config, MutableJobCatalog jobCatalog, Logger log) {
+ this(config, jobCatalog, Optional.of(log));
+ }
+
+ /** Constructor with no logging */
+ public StreamingKafkaSpecConsumer(Config config, MutableJobCatalog jobCatalog) {
+ this(config, jobCatalog, Optional.<Logger>absent());
+ }
+
+ /**
+ * This method returns job specs receive from Kafka. It will block if there are no job specs.
+ * @return list of (verb, jobspecs) pairs.
+ */
+ @Override
+ public Future<? extends List<Pair<Verb, Spec>>> changedSpecs() {
+ List<Pair<Verb, Spec>> changesSpecs = new ArrayList<>();
+
+ try {
+ Pair<Verb, Spec> specPair = _jobSpecQueue.take();
+
+ do {
+ changesSpecs.add(specPair);
+
+ // if there are more elements then pass them along in this call
+ specPair = _jobSpecQueue.poll();
+ } while (specPair != null);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+
+ return new CompletedFuture(changesSpecs, null);
+ }
+
+ @Override
+ protected void startUp() {
+ _jobMonitor.startAsync().awaitRunning();
+ }
+
+ @Override
+ protected void shutDown() {
+ _jobMonitor.stopAsync().awaitTerminated();
+ }
+
+ @Override
+ public void close() throws IOException {
+ shutDown();
+ }
+
+ /**
+ * JobCatalog listener that puts messages into a blocking queue for consumption by changedSpecs method of
+ * {@link StreamingKafkaSpecConsumer}
+ */
+ protected class JobSpecListener extends DefaultJobCatalogListenerImpl {
+ public JobSpecListener() {
+ super(StreamingKafkaSpecConsumer.this.log);
+ }
+
+ @Override public void onAddJob(JobSpec addedJob) {
+ super.onAddJob(addedJob);
+
+ try {
+ _jobSpecQueue.put(new ImmutablePair<Verb, Spec>(Verb.ADD, addedJob));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ @Override public void onDeleteJob(URI deletedJobURI, String deletedJobVersion) {
+ super.onDeleteJob(deletedJobURI, deletedJobVersion);
+ try {
+ JobSpec.Builder jobSpecBuilder = JobSpec.builder(deletedJobURI);
+
+ Properties props = new Properties();
+ jobSpecBuilder.withVersion(deletedJobVersion).withConfigAsProperties(props);
+
+ _jobSpecQueue.put(new ImmutablePair<Verb, Spec>(Verb.DELETE, jobSpecBuilder.build()));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ @Override public void onUpdateJob(JobSpec updatedJob) {
+ super.onUpdateJob(updatedJob);
+
+ try {
+ _jobSpecQueue.put(new ImmutablePair<Verb, Spec>(Verb.UPDATE, updatedJob));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceConsumer.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceConsumer.java
deleted file mode 100644
index ac7fe03..0000000
--- a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceConsumer.java
+++ /dev/null
@@ -1,171 +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 org.apache.gobblin.util.ConfigUtils;
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
-
-import org.apache.gobblin.runtime.api.JobSpec;
-import org.apache.gobblin.runtime.api.MutableJobCatalog;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstanceConsumer;
-import org.apache.gobblin.runtime.job_monitor.AvroJobSpecKafkaJobMonitor;
-import org.apache.gobblin.runtime.job_monitor.KafkaJobMonitor;
-import org.apache.gobblin.runtime.std.DefaultJobCatalogListenerImpl;
-import org.apache.gobblin.util.CompletedFuture;
-
-
-/**
- * SpecExecutorInstanceConsumer that consumes from kafka in a streaming manner
- */
-public class StreamingKafkaSpecExecutorInstanceConsumer extends SimpleKafkaSpecExecutorInstance
- implements SpecExecutorInstanceConsumer<Spec>, Closeable {
- public static final String SPEC_STREAMING_BLOCKING_QUEUE_SIZE = "spec.StreamingBlockingQueueSize";
- private static final int DEFAULT_SPEC_STREAMING_BLOCKING_QUEUE_SIZE = 100;
- private final AvroJobSpecKafkaJobMonitor _jobMonitor;
- private final BlockingQueue<ImmutablePair<Verb, Spec>> _jobSpecQueue;
-
- public StreamingKafkaSpecExecutorInstanceConsumer(Config config, MutableJobCatalog jobCatalog, Optional<Logger> log) {
- super(config, log);
- String topic = config.getString(SPEC_KAFKA_TOPICS_KEY);
- Config defaults = ConfigFactory.parseMap(ImmutableMap.of(AvroJobSpecKafkaJobMonitor.TOPIC_KEY, topic,
- KafkaJobMonitor.KAFKA_AUTO_OFFSET_RESET_KEY, KafkaJobMonitor.KAFKA_AUTO_OFFSET_RESET_SMALLEST));
-
- try {
- _jobMonitor = (AvroJobSpecKafkaJobMonitor)(new AvroJobSpecKafkaJobMonitor.Factory())
- .forConfig(config.withFallback(defaults), jobCatalog);
- } catch (IOException e) {
- throw new RuntimeException("Could not create job monitor", e);
- }
-
- _jobSpecQueue = new LinkedBlockingQueue<>(ConfigUtils.getInt(config, "SPEC_STREAMING_BLOCKING_QUEUE_SIZE",
- DEFAULT_SPEC_STREAMING_BLOCKING_QUEUE_SIZE));
-
- // listener will add job specs to a blocking queue to send to callers of changedSpecs()
- jobCatalog.addListener(new JobSpecListener());
- }
-
- public StreamingKafkaSpecExecutorInstanceConsumer(Config config, MutableJobCatalog jobCatalog, Logger log) {
- this(config, jobCatalog, Optional.of(log));
- }
-
- /** Constructor with no logging */
- public StreamingKafkaSpecExecutorInstanceConsumer(Config config, MutableJobCatalog jobCatalog) {
- this(config, jobCatalog, Optional.<Logger>absent());
- }
-
- /**
- * This method returns job specs receive from Kafka. It will block if there are no job specs.
- * @return list of (verb, jobspecs) pairs.
- */
- @Override
- public Future<? extends List<Pair<Verb, Spec>>> changedSpecs() {
- List<Pair<Verb, Spec>> changesSpecs = new ArrayList<>();
-
- try {
- Pair<Verb, Spec> specPair = _jobSpecQueue.take();
-
- do {
- changesSpecs.add(specPair);
-
- // if there are more elements then pass them along in this call
- specPair = _jobSpecQueue.poll();
- } while (specPair != null);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
-
- return new CompletedFuture(changesSpecs, null);
- }
-
- @Override
- protected void startUp() {
- _jobMonitor.startAsync().awaitRunning();
- }
-
- @Override
- protected void shutDown() {
- _jobMonitor.stopAsync().awaitTerminated();
- }
-
- @Override
- public void close() throws IOException {
- shutDown();
- }
-
- /**
- * JobCatalog listener that puts messages into a blocking queue for consumption by changedSpecs method of
- * {@link StreamingKafkaSpecExecutorInstanceConsumer}
- */
- protected class JobSpecListener extends DefaultJobCatalogListenerImpl {
- public JobSpecListener() {
- super(StreamingKafkaSpecExecutorInstanceConsumer.this._log);
- }
-
- @Override public void onAddJob(JobSpec addedJob) {
- super.onAddJob(addedJob);
-
- try {
- _jobSpecQueue.put(new ImmutablePair<Verb, Spec>(Verb.ADD, addedJob));
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
-
- @Override public void onDeleteJob(URI deletedJobURI, String deletedJobVersion) {
- super.onDeleteJob(deletedJobURI, deletedJobVersion);
- try {
- JobSpec.Builder jobSpecBuilder = JobSpec.builder(deletedJobURI);
-
- Properties props = new Properties();
- jobSpecBuilder.withVersion(deletedJobVersion).withConfigAsProperties(props);
-
- _jobSpecQueue.put(new ImmutablePair<Verb, Spec>(Verb.DELETE, jobSpecBuilder.build()));
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
-
- @Override public void onUpdateJob(JobSpec updatedJob) {
- super.onUpdateJob(updatedJob);
-
- try {
- _jobSpecQueue.put(new ImmutablePair<Verb, Spec>(Verb.UPDATE, updatedJob));
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceTest.java b/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceTest.java
deleted file mode 100644
index 58c5d72..0000000
--- a/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorInstanceTest.java
+++ /dev/null
@@ -1,180 +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.net.URI;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.apache.commons.lang3.tuple.Pair;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.AfterSuite;
-import org.testng.annotations.Test;
-
-import com.google.common.io.Closer;
-import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.kafka.writer.KafkaWriterConfigurationKeys;
-import org.apache.gobblin.metrics.reporter.KafkaTestBase;
-import org.apache.gobblin.runtime.api.JobSpec;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.util.ConfigUtils;
-import org.apache.gobblin.writer.WriteResponse;
-
-
-@Slf4j
-public class SimpleKafkaSpecExecutorInstanceTest extends KafkaTestBase {
-
- public static final String TOPIC = SimpleKafkaSpecExecutorInstanceTest.class.getSimpleName();
-
- private Closer _closer;
- private Properties _properties;
- private SimpleKafkaSpecExecutorInstanceProducer _seip;
- private SimpleKafkaSpecExecutorInstanceConsumer _seic;
- private String _kafkaBrokers;
-
- public SimpleKafkaSpecExecutorInstanceTest()
- throws InterruptedException, RuntimeException {
- super(TOPIC);
- _kafkaBrokers = "localhost:" + kafkaPort;
- log.info("Going to use Kakfa broker: " + _kafkaBrokers);
- }
-
- @Test
- public void testAddSpec() throws Exception {
- _closer = Closer.create();
- _properties = new Properties();
-
- // Properties for Producer
- _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_TOPIC, TOPIC);
- _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX + "bootstrap.servers", _kafkaBrokers);
- _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX+"value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
-
- // Properties for Consumer
- _properties.setProperty(ConfigurationKeys.KAFKA_BROKERS, _kafkaBrokers);
- _properties.setProperty(SimpleKafkaSpecExecutorInstanceProducer.SPEC_KAFKA_TOPICS_KEY, TOPIC);
-
- // SEI Producer
- _seip = _closer.register(new SimpleKafkaSpecExecutorInstanceProducer(ConfigUtils.propertiesToConfig(_properties)));
-
- String addedSpecUriString = "/foo/bar/addedSpec";
- Spec spec = initJobSpec(addedSpecUriString);
- WriteResponse writeResponse = (WriteResponse) _seip.addSpec(spec).get();
- log.info("WriteResponse: " + writeResponse);
-
- try {
- Thread.sleep(1000);
- } catch(InterruptedException ex) {
- Thread.currentThread().interrupt();
- }
-
- _seic = _closer.register(new SimpleKafkaSpecExecutorInstanceConsumer(ConfigUtils.propertiesToConfig(_properties)));
-
- List<Pair<SpecExecutorInstance.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
- Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
-
- Map.Entry<SpecExecutorInstance.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
- Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutorInstance.Verb.ADD), "Verb did not match");
- Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(addedSpecUriString), "Expected URI did not match");
- Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
- }
-
- @Test (dependsOnMethods = "testAddSpec")
- public void testUpdateSpec() throws Exception {
- String updatedSpecUriString = "/foo/bar/updatedSpec";
- Spec spec = initJobSpec(updatedSpecUriString);
- WriteResponse writeResponse = (WriteResponse) _seip.updateSpec(spec).get();
- log.info("WriteResponse: " + writeResponse);
-
- try {
- Thread.sleep(1000);
- } catch(InterruptedException ex) {
- Thread.currentThread().interrupt();
- }
-
- List<Pair<SpecExecutorInstance.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
- Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
-
- Map.Entry<SpecExecutorInstance.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
- Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutorInstance.Verb.UPDATE), "Verb did not match");
- Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(updatedSpecUriString), "Expected URI did not match");
- Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
- }
-
- @Test (dependsOnMethods = "testUpdateSpec")
- public void testDeleteSpec() throws Exception {
- String deletedSpecUriString = "/foo/bar/deletedSpec";
- WriteResponse writeResponse = (WriteResponse) _seip.deleteSpec(new URI(deletedSpecUriString)).get();
- log.info("WriteResponse: " + writeResponse);
-
- try {
- Thread.sleep(1000);
- } catch(InterruptedException ex) {
- Thread.currentThread().interrupt();
- }
-
- List<Pair<SpecExecutorInstance.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
- Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
-
- Map.Entry<SpecExecutorInstance.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
- Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutorInstance.Verb.DELETE), "Verb did not match");
- Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(deletedSpecUriString), "Expected URI did not match");
- Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
- }
-
- @Test (dependsOnMethods = "testDeleteSpec")
- public void testResetConsumption() throws Exception {
- SimpleKafkaSpecExecutorInstanceConsumer seic = _closer
- .register(new SimpleKafkaSpecExecutorInstanceConsumer(ConfigUtils.propertiesToConfig(_properties)));
-
- List<Pair<SpecExecutorInstance.Verb, Spec>> consumedEvent = seic.changedSpecs().get();
- Assert.assertTrue(consumedEvent.size() == 3, "Consumption was reset, we should see all events");
- }
-
- private JobSpec initJobSpec(String specUri) {
- Properties properties = new Properties();
- return JobSpec.builder(specUri)
- .withConfig(ConfigUtils.propertiesToConfig(properties))
- .withVersion("1")
- .withDescription("Spec Description")
- .build();
- }
-
- @AfterClass
- public void after() {
- try {
- _closer.close();
- } catch(Exception e) {
- log.error("Failed to close SEIC and SEIP.", e);
- }
- try {
- close();
- } catch(Exception e) {
- log.error("Failed to close Kafka server.", e);
- }
- }
-
- @AfterSuite
- public void afterSuite() {
- closeServer();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorTest.java b/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorTest.java
new file mode 100644
index 0000000..5567c18
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/SimpleKafkaSpecExecutorTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import com.google.common.io.Closer;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.Test;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.kafka.writer.KafkaWriterConfigurationKeys;
+import org.apache.gobblin.metrics.reporter.KafkaTestBase;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.writer.WriteResponse;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class SimpleKafkaSpecExecutorTest extends KafkaTestBase {
+
+ public static final String TOPIC = SimpleKafkaSpecExecutorTest.class.getSimpleName();
+
+ private Closer _closer;
+ private Properties _properties;
+ private SimpleKafkaSpecProducer _seip;
+ private SimpleKafkaSpecConsumer _seic;
+ private String _kafkaBrokers;
+
+ public SimpleKafkaSpecExecutorTest()
+ throws InterruptedException, RuntimeException {
+ super(TOPIC);
+ _kafkaBrokers = "localhost:" + kafkaPort;
+ log.info("Going to use Kakfa broker: " + _kafkaBrokers);
+ }
+
+ @Test
+ public void testAddSpec() throws Exception {
+ _closer = Closer.create();
+ _properties = new Properties();
+
+ // Properties for Producer
+ _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_TOPIC, TOPIC);
+ _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX + "bootstrap.servers", _kafkaBrokers);
+ _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX+"value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
+
+ // Properties for Consumer
+ _properties.setProperty(ConfigurationKeys.KAFKA_BROKERS, _kafkaBrokers);
+ _properties.setProperty(SimpleKafkaSpecExecutor.SPEC_KAFKA_TOPICS_KEY, TOPIC);
+
+ // SEI Producer
+ _seip = _closer.register(new SimpleKafkaSpecProducer(ConfigUtils.propertiesToConfig(_properties)));
+
+ String addedSpecUriString = "/foo/bar/addedSpec";
+ Spec spec = initJobSpec(addedSpecUriString);
+ WriteResponse writeResponse = (WriteResponse) _seip.addSpec(spec).get();
+ log.info("WriteResponse: " + writeResponse);
+
+ try {
+ Thread.sleep(1000);
+ } catch(InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ }
+
+ _seic = _closer.register(new SimpleKafkaSpecConsumer(ConfigUtils.propertiesToConfig(_properties)));
+
+ List<Pair<SpecExecutor.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
+ Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
+
+ Map.Entry<SpecExecutor.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
+ Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutor.Verb.ADD), "Verb did not match");
+ Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(addedSpecUriString), "Expected URI did not match");
+ Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
+ }
+
+ @Test (dependsOnMethods = "testAddSpec")
+ public void testUpdateSpec() throws Exception {
+ String updatedSpecUriString = "/foo/bar/updatedSpec";
+ Spec spec = initJobSpec(updatedSpecUriString);
+ WriteResponse writeResponse = (WriteResponse) _seip.updateSpec(spec).get();
+ log.info("WriteResponse: " + writeResponse);
+
+ try {
+ Thread.sleep(1000);
+ } catch(InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ }
+
+ List<Pair<SpecExecutor.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
+ Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
+
+ Map.Entry<SpecExecutor.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
+ Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutor.Verb.UPDATE), "Verb did not match");
+ Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(updatedSpecUriString), "Expected URI did not match");
+ Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
+ }
+
+ @Test (dependsOnMethods = "testUpdateSpec")
+ public void testDeleteSpec() throws Exception {
+ String deletedSpecUriString = "/foo/bar/deletedSpec";
+ WriteResponse writeResponse = (WriteResponse) _seip.deleteSpec(new URI(deletedSpecUriString)).get();
+ log.info("WriteResponse: " + writeResponse);
+
+ try {
+ Thread.sleep(1000);
+ } catch(InterruptedException ex) {
+ Thread.currentThread().interrupt();
+ }
+
+ List<Pair<SpecExecutor.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
+ Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
+
+ Map.Entry<SpecExecutor.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
+ Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutor.Verb.DELETE), "Verb did not match");
+ Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(deletedSpecUriString), "Expected URI did not match");
+ Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
+ }
+
+ @Test (dependsOnMethods = "testDeleteSpec")
+ public void testResetConsumption() throws Exception {
+ SimpleKafkaSpecConsumer seic = _closer
+ .register(new SimpleKafkaSpecConsumer(ConfigUtils.propertiesToConfig(_properties)));
+
+ List<Pair<SpecExecutor.Verb, Spec>> consumedEvent = seic.changedSpecs().get();
+ Assert.assertTrue(consumedEvent.size() == 3, "Consumption was reset, we should see all events");
+ }
+
+ private JobSpec initJobSpec(String specUri) {
+ Properties properties = new Properties();
+ return JobSpec.builder(specUri)
+ .withConfig(ConfigUtils.propertiesToConfig(properties))
+ .withVersion("1")
+ .withDescription("Spec Description")
+ .build();
+ }
+
+ @AfterClass
+ public void after() {
+ try {
+ _closer.close();
+ } catch(Exception e) {
+ log.error("Failed to close SEIC and SEIP.", e);
+ }
+ try {
+ close();
+ } catch(Exception e) {
+ log.error("Failed to close Kafka server.", e);
+ }
+ }
+
+ @AfterSuite
+ public void afterSuite() {
+ closeServer();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceTest.java b/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceTest.java
deleted file mode 100644
index 939aafa..0000000
--- a/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorInstanceTest.java
+++ /dev/null
@@ -1,192 +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.io.IOException;
-import java.net.URI;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.AfterSuite;
-import org.testng.annotations.Test;
-
-import com.google.common.io.Closer;
-import com.typesafe.config.Config;
-
-import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.kafka.writer.KafkaWriterConfigurationKeys;
-import org.apache.gobblin.metrics.reporter.KafkaTestBase;
-import org.apache.gobblin.runtime.api.JobSpec;
-import org.apache.gobblin.runtime.api.Spec;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.runtime.job_catalog.NonObservingFSJobCatalog;
-import org.apache.gobblin.util.ConfigUtils;
-import org.apache.gobblin.writer.WriteResponse;
-import lombok.extern.slf4j.Slf4j;
-
-
-@Slf4j
-public class StreamingKafkaSpecExecutorInstanceTest extends KafkaTestBase {
-
- public static final String TOPIC = StreamingKafkaSpecExecutorInstanceTest.class.getSimpleName();
-
- private Closer _closer;
- private Properties _properties;
- private SimpleKafkaSpecExecutorInstanceProducer _seip;
- private StreamingKafkaSpecExecutorInstanceConsumer _seic;
- private NonObservingFSJobCatalog _jobCatalog;
- private String _kafkaBrokers;
- private static final String _TEST_DIR_PATH = "/tmp/StreamingKafkaSpecExecutorInstanceTest";
- private static final String _JOBS_DIR_PATH = _TEST_DIR_PATH + "/jobs";
-
- public StreamingKafkaSpecExecutorInstanceTest()
- throws InterruptedException, RuntimeException {
- super(TOPIC);
- _kafkaBrokers = "localhost:" + kafkaPort;
- log.info("Going to use Kakfa broker: " + _kafkaBrokers);
-
- cleanupTestDir();
- }
-
- private void cleanupTestDir() {
- File testDir = new File(_TEST_DIR_PATH);
-
- if (testDir.exists()) {
- try {
- FileUtils.deleteDirectory(testDir);
- } catch (IOException e) {
- throw new RuntimeException("Could not delete test directory", e);
- }
- }
- }
-
- @Test
- public void testAddSpec() throws Exception {
- _closer = Closer.create();
- _properties = new Properties();
-
- // Properties for Producer
- _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_TOPIC, TOPIC);
- _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX + "bootstrap.servers", _kafkaBrokers);
- _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX+"value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
-
- // Properties for Consumer
- _properties.setProperty("jobSpecMonitor.kafka.zookeeper.connect", zkConnect);
- _properties.setProperty(SimpleKafkaSpecExecutorInstanceProducer.SPEC_KAFKA_TOPICS_KEY, TOPIC);
- _properties.setProperty("gobblin.cluster.jobconf.fullyQualifiedPath", _JOBS_DIR_PATH);
-
- Config config = ConfigUtils.propertiesToConfig(_properties);
-
- // SEI Producer
- _seip = _closer.register(new SimpleKafkaSpecExecutorInstanceProducer(config));
-
- String addedSpecUriString = "/foo/bar/addedSpec";
- Spec spec = initJobSpec(addedSpecUriString);
- WriteResponse writeResponse = (WriteResponse) _seip.addSpec(spec).get();
- log.info("WriteResponse: " + writeResponse);
-
- _jobCatalog = new NonObservingFSJobCatalog(config.getConfig("gobblin.cluster"));
- _jobCatalog.startAsync().awaitRunning();
-
- // SEI Consumer
- _seic = _closer.register(new StreamingKafkaSpecExecutorInstanceConsumer(config, _jobCatalog));
- _seic.startAsync().awaitRunning();
-
- List<Pair<SpecExecutorInstance.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
- Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
-
- Map.Entry<SpecExecutorInstance.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
- Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutorInstance.Verb.ADD), "Verb did not match");
- Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(addedSpecUriString), "Expected URI did not match");
- Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
- }
-
- @Test (dependsOnMethods = "testAddSpec")
- public void testUpdateSpec() throws Exception {
- // update is only treated as an update for existing job specs
- String updatedSpecUriString = "/foo/bar/addedSpec";
- Spec spec = initJobSpec(updatedSpecUriString);
- WriteResponse writeResponse = (WriteResponse) _seip.updateSpec(spec).get();
- log.info("WriteResponse: " + writeResponse);
-
- List<Pair<SpecExecutorInstance.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
- Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
-
- Map.Entry<SpecExecutorInstance.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
- Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutorInstance.Verb.UPDATE), "Verb did not match");
- Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(updatedSpecUriString), "Expected URI did not match");
- Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
- }
-
- @Test (dependsOnMethods = "testUpdateSpec")
- public void testDeleteSpec() throws Exception {
- // delete needs to be on a job spec that exists to get notification
- String deletedSpecUriString = "/foo/bar/addedSpec";
- WriteResponse writeResponse = (WriteResponse) _seip.deleteSpec(new URI(deletedSpecUriString)).get();
- log.info("WriteResponse: " + writeResponse);
-
- List<Pair<SpecExecutorInstance.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
- Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
-
- Map.Entry<SpecExecutorInstance.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
- Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutorInstance.Verb.DELETE), "Verb did not match");
- Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(deletedSpecUriString), "Expected URI did not match");
- Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
- }
-
- private JobSpec initJobSpec(String specUri) {
- Properties properties = new Properties();
- return JobSpec.builder(specUri)
- .withConfig(ConfigUtils.propertiesToConfig(properties))
- .withVersion("1")
- .withDescription("Spec Description")
- .build();
- }
-
- @AfterClass
- public void after() {
- try {
- _closer.close();
- } catch(Exception e) {
- log.error("Failed to close SEIC and SEIP.", e);
- }
- try {
- close();
- } catch(Exception e) {
- log.error("Failed to close Kafka server.", e);
- }
-
- if (_jobCatalog != null) {
- _jobCatalog.stopAsync().awaitTerminated();
- }
-
- cleanupTestDir();
- }
-
- @AfterSuite
- public void afterSuite() {
- closeServer();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorTest.java b/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorTest.java
new file mode 100644
index 0000000..e9c7ee6
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-08/src/test/java/org/apache/gobblin/service/StreamingKafkaSpecExecutorTest.java
@@ -0,0 +1,191 @@
+/*
+ * 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.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.Test;
+
+import com.google.common.io.Closer;
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.kafka.writer.KafkaWriterConfigurationKeys;
+import org.apache.gobblin.metrics.reporter.KafkaTestBase;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.job_catalog.NonObservingFSJobCatalog;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.writer.WriteResponse;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+
+import lombok.extern.slf4j.Slf4j;
+
+
+@Slf4j
+public class StreamingKafkaSpecExecutorTest extends KafkaTestBase {
+
+ public static final String TOPIC = StreamingKafkaSpecExecutorTest.class.getSimpleName();
+
+ private Closer _closer;
+ private Properties _properties;
+ private SimpleKafkaSpecProducer _seip;
+ private StreamingKafkaSpecConsumer _seic;
+ private NonObservingFSJobCatalog _jobCatalog;
+ private String _kafkaBrokers;
+ private static final String _TEST_DIR_PATH = "/tmp/StreamingKafkaSpecExecutorTest";
+ private static final String _JOBS_DIR_PATH = _TEST_DIR_PATH + "/jobs";
+
+ public StreamingKafkaSpecExecutorTest()
+ throws InterruptedException, RuntimeException {
+ super(TOPIC);
+ _kafkaBrokers = "localhost:" + kafkaPort;
+ log.info("Going to use Kakfa broker: " + _kafkaBrokers);
+
+ cleanupTestDir();
+ }
+
+ private void cleanupTestDir() {
+ File testDir = new File(_TEST_DIR_PATH);
+
+ if (testDir.exists()) {
+ try {
+ FileUtils.deleteDirectory(testDir);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not delete test directory", e);
+ }
+ }
+ }
+
+ @Test
+ public void testAddSpec() throws Exception {
+ _closer = Closer.create();
+ _properties = new Properties();
+
+ // Properties for Producer
+ _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_TOPIC, TOPIC);
+ _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX + "bootstrap.servers", _kafkaBrokers);
+ _properties.setProperty(KafkaWriterConfigurationKeys.KAFKA_PRODUCER_CONFIG_PREFIX+"value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
+
+ // Properties for Consumer
+ _properties.setProperty("jobSpecMonitor.kafka.zookeeper.connect", zkConnect);
+ _properties.setProperty(SimpleKafkaSpecExecutor.SPEC_KAFKA_TOPICS_KEY, TOPIC);
+ _properties.setProperty("gobblin.cluster.jobconf.fullyQualifiedPath", _JOBS_DIR_PATH);
+
+ Config config = ConfigUtils.propertiesToConfig(_properties);
+
+ // SEI Producer
+ _seip = _closer.register(new SimpleKafkaSpecProducer(config));
+
+ String addedSpecUriString = "/foo/bar/addedSpec";
+ Spec spec = initJobSpec(addedSpecUriString);
+ WriteResponse writeResponse = (WriteResponse) _seip.addSpec(spec).get();
+ log.info("WriteResponse: " + writeResponse);
+
+ _jobCatalog = new NonObservingFSJobCatalog(config.getConfig("gobblin.cluster"));
+ _jobCatalog.startAsync().awaitRunning();
+
+ // SEI Consumer
+ _seic = _closer.register(new StreamingKafkaSpecConsumer(config, _jobCatalog));
+ _seic.startAsync().awaitRunning();
+
+ List<Pair<SpecExecutor.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
+ Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
+
+ Map.Entry<SpecExecutor.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
+ Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutor.Verb.ADD), "Verb did not match");
+ Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(addedSpecUriString), "Expected URI did not match");
+ Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
+ }
+
+ @Test (dependsOnMethods = "testAddSpec")
+ public void testUpdateSpec() throws Exception {
+ // update is only treated as an update for existing job specs
+ String updatedSpecUriString = "/foo/bar/addedSpec";
+ Spec spec = initJobSpec(updatedSpecUriString);
+ WriteResponse writeResponse = (WriteResponse) _seip.updateSpec(spec).get();
+ log.info("WriteResponse: " + writeResponse);
+
+ List<Pair<SpecExecutor.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
+ Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
+
+ Map.Entry<SpecExecutor.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
+ Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutor.Verb.UPDATE), "Verb did not match");
+ Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(updatedSpecUriString), "Expected URI did not match");
+ Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
+ }
+
+ @Test (dependsOnMethods = "testUpdateSpec")
+ public void testDeleteSpec() throws Exception {
+ // delete needs to be on a job spec that exists to get notification
+ String deletedSpecUriString = "/foo/bar/addedSpec";
+ WriteResponse writeResponse = (WriteResponse) _seip.deleteSpec(new URI(deletedSpecUriString)).get();
+ log.info("WriteResponse: " + writeResponse);
+
+ List<Pair<SpecExecutor.Verb, Spec>> consumedEvent = _seic.changedSpecs().get();
+ Assert.assertTrue(consumedEvent.size() == 1, "Consumption did not match production");
+
+ Map.Entry<SpecExecutor.Verb, Spec> consumedSpecAction = consumedEvent.get(0);
+ Assert.assertTrue(consumedSpecAction.getKey().equals(SpecExecutor.Verb.DELETE), "Verb did not match");
+ Assert.assertTrue(consumedSpecAction.getValue().getUri().toString().equals(deletedSpecUriString), "Expected URI did not match");
+ Assert.assertTrue(consumedSpecAction.getValue() instanceof JobSpec, "Expected JobSpec");
+ }
+
+ private JobSpec initJobSpec(String specUri) {
+ Properties properties = new Properties();
+ return JobSpec.builder(specUri)
+ .withConfig(ConfigUtils.propertiesToConfig(properties))
+ .withVersion("1")
+ .withDescription("Spec Description")
+ .build();
+ }
+
+ @AfterClass
+ public void after() {
+ try {
+ _closer.close();
+ } catch(Exception e) {
+ log.error("Failed to close SEIC and SEIP.", e);
+ }
+ try {
+ close();
+ } catch(Exception e) {
+ log.error("Failed to close Kafka server.", e);
+ }
+
+ if (_jobCatalog != null) {
+ _jobCatalog.stopAsync().awaitTerminated();
+ }
+
+ cleanupTestDir();
+ }
+
+ @AfterSuite
+ public void afterSuite() {
+ closeServer();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCompiler.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCompiler.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCompiler.java
index bb617a7..aceb5dd 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCompiler.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCompiler.java
@@ -24,16 +24,16 @@ import org.apache.gobblin.instrumented.Instrumentable;
/***
* Take in a logical {@link Spec} and compile corresponding materialized {@link Spec}s
- * and the mapping to {@link SpecExecutorInstance} that they can be run on.
+ * and the mapping to {@link SpecExecutor} that they can be run on.
*/
public interface SpecCompiler extends SpecCatalogListener, Instrumentable {
/***
* Take in a logical {@link Spec} and compile corresponding materialized {@link Spec}s
- * and the mapping to {@link SpecExecutorInstance} that they can be run on.
+ * and the mapping to {@link SpecExecutor} that they can be run on.
* @param spec {@link Spec} to compile.
- * @return Map of materialized physical {@link Spec} and {@link SpecExecutorInstance}.
+ * @return Map of materialized physical {@link Spec} and {@link SpecExecutor}.
*/
- Map<Spec, SpecExecutorInstanceProducer> compileFlow(Spec spec);
+ Map<Spec, SpecExecutor> compileFlow(Spec spec);
/***
* Map of {@link Spec} URI and {@link TopologySpec} the {@link SpecCompiler}
@@ -41,4 +41,4 @@ public interface SpecCompiler extends SpecCatalogListener, Instrumentable {
* @return Map of {@link Spec} URI and {@link TopologySpec}
*/
Map<URI, TopologySpec> getTopologySpecMap();
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/TopologySpec.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/TopologySpec.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/TopologySpec.java
index da89b9e..c33f3de 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/TopologySpec.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/TopologySpec.java
@@ -25,26 +25,25 @@ import javax.annotation.concurrent.NotThreadSafe;
import edu.umd.cs.findbugs.annotations.SuppressWarnings;
-import lombok.AllArgsConstructor;
-import lombok.Data;
-
-import org.apache.commons.lang3.reflect.ConstructorUtils;
-
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
+import org.apache.commons.lang3.reflect.ConstructorUtils;
import org.apache.gobblin.annotation.Alpha;
import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutorInstanceProducer;
import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.runtime.spec_executorInstance.InMemorySpecExecutor;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
/**
- * Data model representation that describes a topology ie. a {@link SpecExecutorInstance} and its
+ * Data model representation that describes a topology ie. a {@link SpecExecutor} and its
* capabilities tuple .
*
*/
@@ -53,8 +52,8 @@ import org.apache.gobblin.util.ConfigUtils;
@AllArgsConstructor
@NotThreadSafe
public class TopologySpec implements Configurable, Spec {
- public static final String DEFAULT_SPEC_EXECUTOR_INSTANCE_PRODUCER = InMemorySpecExecutorInstanceProducer.class.getCanonicalName();
- public static final String SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY = "specExecutorInstanceProducer.class";
+ public static final String DEFAULT_SPEC_EXECUTOR_INSTANCE = InMemorySpecExecutor.class.getCanonicalName();
+ public static final String SPEC_EXECUTOR_INSTANCE_KEY = "specExecutorInstance.class";
private static final long serialVersionUID = 6106269076155338046L;
@@ -78,26 +77,29 @@ public class TopologySpec implements Configurable, Spec {
/** Underlying executor instance such as Gobblin cluster or Azkaban */
@SuppressWarnings(justification="Initialization handled by getter", value="SE_TRANSIENT_FIELD_NOT_RESTORED")
- transient SpecExecutorInstanceProducer specExecutorInstanceProducer;
+ transient SpecExecutor specExecutorInstance;
- public SpecExecutorInstanceProducer getSpecExecutorInstanceProducer() {
- if (null == specExecutorInstanceProducer) {
- String specExecutorInstanceProducerClass = DEFAULT_SPEC_EXECUTOR_INSTANCE_PRODUCER;
- if (config.hasPath(SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY)) {
- specExecutorInstanceProducerClass = config.getString(SPEC_EXECUTOR_INSTANCE_PRODUCER_KEY);
+ /**
+ * @return A {@link SpecExecutor}'s instance defined by <Technology, Location, Communication Mechanism>
+ */
+ public synchronized SpecExecutor getSpecExecutor() {
+ if (null == specExecutorInstance) {
+ String specExecutorClass = DEFAULT_SPEC_EXECUTOR_INSTANCE;
+ if (config.hasPath(SPEC_EXECUTOR_INSTANCE_KEY)) {
+ specExecutorClass = config.getString(SPEC_EXECUTOR_INSTANCE_KEY);
}
try {
- ClassAliasResolver<SpecExecutorInstanceProducer> _aliasResolver =
- new ClassAliasResolver<>(SpecExecutorInstanceProducer.class);
- specExecutorInstanceProducer = (SpecExecutorInstanceProducer) ConstructorUtils
+ ClassAliasResolver<SpecExecutor> _aliasResolver =
+ new ClassAliasResolver<>(SpecExecutor.class);
+ specExecutorInstance = (SpecExecutor) ConstructorUtils
.invokeConstructor(Class.forName(_aliasResolver
- .resolve(specExecutorInstanceProducerClass)), config);
+ .resolve(specExecutorClass)), config);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException
| ClassNotFoundException e) {
throw new RuntimeException(e);
}
}
- return specExecutorInstanceProducer;
+ return specExecutorInstance;
}
public static TopologySpec.Builder builder(URI topologySpecUri) {
@@ -164,7 +166,7 @@ public class TopologySpec implements Configurable, Spec {
private String version = "1";
private Optional<String> description = Optional.absent();
private Optional<URI> topologyCatalogURI = Optional.absent();
- private Optional<SpecExecutorInstanceProducer> specExecutorInstanceProducer = Optional.absent();
+ private Optional<SpecExecutor> specExecutorInstance = Optional.absent();
public Builder(URI topologySpecUri) {
Preconditions.checkNotNull(topologySpecUri);
@@ -189,9 +191,9 @@ public class TopologySpec implements Configurable, Spec {
public TopologySpec build() {
Preconditions.checkNotNull(this.uri);
Preconditions.checkNotNull(this.version);
+ return new TopologySpec(getURI(), getVersion(), getDescription(), getConfig(), getConfigAsProperties(),
+ getSpecExceutorInstance());
- return new TopologySpec(getURI(), getVersion(), getDescription(), getConfig(),
- getConfigAsProperties(), getSpecExceutorInstanceProducer());
}
/** The scheme and authority of the topology catalog URI are used to generate TopologySpec URIs from
@@ -314,17 +316,17 @@ public class TopologySpec implements Configurable, Spec {
return this;
}
- public SpecExecutorInstanceProducer getSpecExceutorInstanceProducer() {
- if (!this.specExecutorInstanceProducer.isPresent()) {
- // TODO: Try to init SpecExecutorInstanceProducer from config if not initialized via builder.
- throw new RuntimeException("SpecExecutorInstanceProducer not initialized.");
+ public SpecExecutor getSpecExceutorInstance() {
+ if (!this.specExecutorInstance.isPresent()) {
+ // TODO: Try to init SpecProducer from config if not initialized via builder.
+ throw new RuntimeException("SpecExecutor not initialized.");
}
- return this.specExecutorInstanceProducer.get();
+ return this.specExecutorInstance.get();
}
- public TopologySpec.Builder withSpecExecutorInstanceProducer(SpecExecutorInstanceProducer specExecutorInstanceProducer) {
- Preconditions.checkNotNull(specExecutorInstanceProducer);
- this.specExecutorInstanceProducer = Optional.of(specExecutorInstanceProducer);
+ public TopologySpec.Builder withSpecExecutor(SpecExecutor specExecutor) {
+ Preconditions.checkNotNull(specExecutor);
+ this.specExecutorInstance = Optional.of(specExecutor);
return this;
}
}
@@ -337,4 +339,4 @@ public class TopologySpec implements Configurable, Spec {
return this.uri;
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9402a903/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/AvroJobSpecKafkaJobMonitor.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/AvroJobSpecKafkaJobMonitor.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/AvroJobSpecKafkaJobMonitor.java
index 1f2ce21..59733d3 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/AvroJobSpecKafkaJobMonitor.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/AvroJobSpecKafkaJobMonitor.java
@@ -21,41 +21,25 @@ import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collection;
-import java.util.Map;
import java.util.Properties;
-import java.util.regex.Pattern;
-import org.apache.hadoop.fs.Path;
-
-import com.codahale.metrics.Counter;
-import com.google.common.base.Charsets;
-import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
-import com.typesafe.config.ConfigValue;
-import org.apache.gobblin.metrics.GobblinTrackingEvent;
-import org.apache.gobblin.metrics.event.sla.SlaEventKeys;
import org.apache.gobblin.metrics.reporter.util.FixedSchemaVersionWriter;
-import org.apache.gobblin.metrics.reporter.util.NoopSchemaVersionWriter;
import org.apache.gobblin.metrics.reporter.util.SchemaVersionWriter;
import org.apache.gobblin.runtime.api.GobblinInstanceDriver;
import org.apache.gobblin.runtime.api.JobSpec;
import org.apache.gobblin.runtime.api.JobSpecMonitor;
import org.apache.gobblin.runtime.api.JobSpecMonitorFactory;
import org.apache.gobblin.runtime.api.MutableJobCatalog;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance;
-import org.apache.gobblin.runtime.api.SpecExecutorInstance.Verb;
+import org.apache.gobblin.runtime.api.SpecExecutor.Verb;
import org.apache.gobblin.runtime.job_spec.AvroJobSpec;
-import org.apache.gobblin.runtime.metrics.RuntimeMetrics;
import org.apache.gobblin.util.Either;
-import org.apache.gobblin.util.PathUtils;
import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
-import kafka.message.MessageAndMetadata;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
@@ -153,4 +137,4 @@ public class AvroJobSpecKafkaJobMonitor extends KafkaAvroJobMonitor<AvroJobSpec>
return Lists.newArrayList(Either.<JobSpec, URI>right(jobSpec.getUri()));
}
}
-}
+}
\ No newline at end of file