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