Skip to content

Commit

Permalink
[GOBBLIN-1696] Implement file based flowgraph that detects changes to…
Browse files Browse the repository at this point in the history
… the underlying… (#3548)

* Implement file based flowgraph that detects changes to the underlying filesystem and updates dynamically

* Fix checkstyle

* Address review, use atomic swap instead of mutable shared state for flowgraph

* Fix initialization to only load flowgraph after topology specstore is loaded and for better clarity on exceptions

* Change baseflowgraphlistener to a helper class so that it is extensible to future changes in flowgraph format, and can be leveraged properly by test classes

* Fix checkstyle

* Address review and add metrics for flow update failures

* Fix tests

* Migrate git flowgraph to load fully on change rather than by diff, address review

* Cleanup

* Fix tests
  • Loading branch information
Will-Lo authored Oct 4, 2022
1 parent e642c7c commit a246e23
Show file tree
Hide file tree
Showing 20 changed files with 838 additions and 384 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1022,8 +1022,14 @@ public class ConfigurationKeys {
* Configuration properties related to flowGraphs
*/

public static final String JAVA_PROPS_EXTENSIONS = "javaPropsExtensions";
public static final String HOCON_FILE_EXTENSIONS = "hoconFileExtensions";
public static final String FLOWGRAPH_JAVA_PROPS_EXTENSIONS = "flowGraph.javaPropsExtensions";
public static final String FLOWGRAPH_HOCON_FILE_EXTENSIONS = "flowGraph.hoconFileExtensions";
public static final String DEFAULT_PROPERTIES_EXTENSIONS = "properties";
public static final String DEFAULT_CONF_EXTENSIONS = "conf";
public static final String FLOWGRAPH_POLLING_INTERVAL = "flowGraph.pollingInterval";
public static final String FLOWGRAPH_BASE_DIR = "flowGraph.configBaseDirectory";
public static final String FLOWGRAPH_ABSOLUTE_DIR = "flowGraph.absoluteDirectory";


/***
* Configuration properties related to TopologySpec Store
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,4 +54,6 @@ public class ServiceMetricNames {
public static final String JOBS_SENT_TO_SPEC_EXECUTOR = "JobsSentToSpecExecutor";

public static final String HELIX_LEADER_STATE = "HelixLeaderState";

public static final String FLOWGRAPH_UPDATE_FAILED_METER = GOBBLIN_SERVICE_PREFIX + ".FlowgraphUpdateFailed";
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
Expand All @@ -35,9 +36,10 @@
import org.apache.commons.lang3.reflect.ConstructorUtils;
import org.apache.gobblin.service.modules.flowgraph.FlowGraphMonitor;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;

import com.google.common.annotations.VisibleForTesting;

import org.apache.gobblin.configuration.ConfigurationKeys;

import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
Expand All @@ -52,7 +54,6 @@
import lombok.extern.slf4j.Slf4j;

import org.apache.gobblin.annotation.Alpha;
import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.instrumented.Instrumented;
import org.apache.gobblin.runtime.api.FlowSpec;
import org.apache.gobblin.runtime.api.JobTemplate;
Expand All @@ -72,6 +73,7 @@
import org.apache.gobblin.service.modules.template_catalog.ObservingFSFlowEdgeTemplateCatalog;
import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ConfigUtils;
import org.slf4j.Logger;


/***
Expand All @@ -81,8 +83,7 @@
@Alpha
@Slf4j
public class MultiHopFlowCompiler extends BaseFlowToJobSpecCompiler {
@Getter
private final FlowGraph flowGraph;
private AtomicReference<FlowGraph> flowGraph;
@Getter
private ServiceManager serviceManager;
@Getter
Expand Down Expand Up @@ -111,6 +112,12 @@ public MultiHopFlowCompiler(Config config, Optional<Logger> log) {
this(config, log, true);
}

public MultiHopFlowCompiler(Config config, AtomicReference<FlowGraph> flowGraph) {
super(config, Optional.absent(), true);
this.flowGraph = flowGraph;
this.dataMovementAuthorizer = new NoopDataMovementAuthorizer(config);
}

public MultiHopFlowCompiler(Config config, Optional<Logger> log, boolean instrumentationEnabled) {
super(config, log, instrumentationEnabled);
try {
Expand All @@ -120,8 +127,8 @@ public MultiHopFlowCompiler(Config config, Optional<Logger> log, boolean instrum
} catch (RuntimeException e) {
MultiHopFlowCompiler.log.warn("Exception reading data node alias map, ignoring it.", e);
}

this.flowGraph = new BaseFlowGraph(dataNodeAliasMap);
// Use atomic reference to avoid partial flowgraph upgrades during path compilation.
this.flowGraph = new AtomicReference<>(new BaseFlowGraph(dataNodeAliasMap));

Optional<ObservingFSFlowEdgeTemplateCatalog> flowTemplateCatalog = Optional.absent();
if (config.hasPath(ServiceConfigKeys.TEMPLATE_CATALOGS_FULLY_QUALIFIED_PATH_KEY)
Expand Down Expand Up @@ -152,7 +159,7 @@ public MultiHopFlowCompiler(Config config, Optional<Logger> log, boolean instrum
try {
String flowGraphMonitorClassName = ConfigUtils.getString(this.config, ServiceConfigKeys.GOBBLIN_SERVICE_FLOWGRAPH_CLASS_KEY, GitFlowGraphMonitor.class.getCanonicalName());
this.flowGraphMonitor = (FlowGraphMonitor) ConstructorUtils.invokeConstructor(Class.forName(new ClassAliasResolver<>(FlowGraphMonitor.class).resolve(
flowGraphMonitorClassName)), gitFlowGraphConfig, flowTemplateCatalog, this.flowGraph, this.topologySpecMap, this.getInitComplete());
flowGraphMonitorClassName)), gitFlowGraphConfig, flowTemplateCatalog, this, this.topologySpecMap, this.getInitComplete(), instrumentationEnabled);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException | ClassNotFoundException e) {
throw new RuntimeException(e);
}
Expand All @@ -167,13 +174,6 @@ public MultiHopFlowCompiler(Config config, Optional<Logger> log, boolean instrum
}
}

@VisibleForTesting
MultiHopFlowCompiler(Config config, FlowGraph flowGraph) {
super(config, Optional.absent(), true);
this.flowGraph = flowGraph;
this.dataMovementAuthorizer = new NoopDataMovementAuthorizer(config);
}

/**
* Mark the {@link SpecCompiler} as active. This in turn activates the {@link GitFlowGraphMonitor}, allowing to start polling
* and processing changes
Expand Down Expand Up @@ -207,20 +207,20 @@ public void awaitHealthy() throws InterruptedException {
public Dag<JobExecutionPlan> compileFlow(Spec spec) {
Preconditions.checkNotNull(spec);
Preconditions.checkArgument(spec instanceof FlowSpec, "MultiHopFlowCompiler only accepts FlowSpecs");

FlowGraph graph = this.flowGraph.get();
long startTime = System.nanoTime();

FlowSpec flowSpec = (FlowSpec) spec;
String source = FlowConfigUtils.getDataNode(flowSpec.getConfig(), ServiceConfigKeys.FLOW_SOURCE_IDENTIFIER_KEY, this.dataNodeAliasMap);
String destination = FlowConfigUtils.getDataNode(flowSpec.getConfig(), ServiceConfigKeys.FLOW_DESTINATION_IDENTIFIER_KEY, this.dataNodeAliasMap);

DataNode sourceNode = this.flowGraph.getNode(source);
DataNode sourceNode = graph.getNode(source);
if (sourceNode == null) {
flowSpec.addCompilationError(source, destination, String.format("Flowgraph does not have a node with id %s", source));
return null;
}
List<String> destNodeIds = FlowConfigUtils.getDataNodes(flowSpec.getConfig(), ServiceConfigKeys.FLOW_DESTINATION_IDENTIFIER_KEY, this.dataNodeAliasMap);
List<DataNode> destNodes = destNodeIds.stream().map(this.flowGraph::getNode).collect(Collectors.toList());
List<DataNode> destNodes = destNodeIds.stream().map(graph::getNode).collect(Collectors.toList());
if (destNodes.contains(null)) {
flowSpec.addCompilationError(source, destination, String.format("Flowgraph does not have a node with id %s", destNodeIds.get(destNodes.indexOf(null))));
return null;
Expand Down Expand Up @@ -252,7 +252,7 @@ public Dag<JobExecutionPlan> compileFlow(Spec spec) {
}

//Compute the path from source to destination.
FlowGraphPath flowGraphPath = flowGraph.findPath(datasetFlowSpec);
FlowGraphPath flowGraphPath = graph.findPath(datasetFlowSpec);
if (flowGraphPath != null) {
//Convert the path into a Dag of JobExecutionPlans.
jobExecutionPlanDag = jobExecutionPlanDag.merge(flowGraphPath.asDag(this.config));
Expand Down Expand Up @@ -290,6 +290,10 @@ public Dag<JobExecutionPlan> compileFlow(Spec spec) {
return jobExecutionPlanDag;
}

public void setFlowGraph(FlowGraph flowGraph) {
this.flowGraph.set(flowGraph);
}

/**
* If {@link FlowSpec} has {@link ConfigurationKeys#DATASET_SUBPATHS_KEY}, split it into multiple flowSpecs using a
* provided base input and base output path to generate multiple source/destination paths.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,12 +45,13 @@
@Alpha
@Slf4j
public class BaseFlowGraph implements FlowGraph {
// Synchronize read/write access while the flowgraph is in the middle of an update
private final ReadWriteLock rwLock = new ReentrantReadWriteLock(true);

private Map<DataNode, Set<FlowEdge>> nodesToEdges = new HashMap<>();
private Map<String, DataNode> dataNodeMap = new HashMap<>();
private Map<String, FlowEdge> flowEdgeMap = new HashMap<>();
private Map<String, String> dataNodeAliasMap;
private final Map<DataNode, Set<FlowEdge>> nodesToEdges = new HashMap<>();
private final Map<String, DataNode> dataNodeMap = new HashMap<>();
private final Map<String, FlowEdge> flowEdgeMap = new HashMap<>();
private final Map<String, String> dataNodeAliasMap;

public BaseFlowGraph() {
this(new HashMap<>());
Expand Down Expand Up @@ -239,15 +240,17 @@ public Set<FlowEdge> getEdges(DataNode node) {

/**{@inheritDoc}**/
@Override
public FlowGraphPath findPath(FlowSpec flowSpec) throws PathFinder.PathFinderException, ReflectiveOperationException {
public FlowGraphPath findPath(FlowSpec flowSpec)
throws PathFinder.PathFinderException, ReflectiveOperationException {
try {
rwLock.readLock().lock();
//Instantiate a PathFinder.
Class pathFinderClass = Class.forName(ConfigUtils
.getString(flowSpec.getConfig(), FlowGraphConfigurationKeys.FLOW_GRAPH_PATH_FINDER_CLASS,
Class pathFinderClass = Class.forName(
ConfigUtils.getString(flowSpec.getConfig(), FlowGraphConfigurationKeys.FLOW_GRAPH_PATH_FINDER_CLASS,
FlowGraphConfigurationKeys.DEFAULT_FLOW_GRAPH_PATH_FINDER_CLASS));
PathFinder pathFinder =
(PathFinder) GobblinConstructorUtils.invokeLongestConstructor(pathFinderClass, this, flowSpec, dataNodeAliasMap);
(PathFinder) GobblinConstructorUtils.invokeLongestConstructor(pathFinderClass, this, flowSpec,
dataNodeAliasMap);
return pathFinder.findPath();
} finally {
rwLock.readLock().unlock();
Expand Down
Loading

0 comments on commit a246e23

Please sign in to comment.