[Streaming] Streaming scheduler - part1-1: job graph (#6712)

This commit is contained in:
Tianyi Chen 2020-01-15 13:12:03 +08:00 committed by Hao Chen
parent 4227fd1b60
commit 9a4da1951e
27 changed files with 322 additions and 250 deletions

View file

@ -1,7 +1,6 @@
package org.ray.streaming.api.collector; package org.ray.streaming.api.collector;
import java.util.List; import java.util.List;
import org.ray.streaming.api.collector.Collector;
import org.ray.streaming.message.Record; import org.ray.streaming.message.Record;
/** /**

View file

@ -10,8 +10,8 @@ import java.util.Map;
import java.util.ServiceLoader; import java.util.ServiceLoader;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.ray.streaming.api.stream.StreamSink; import org.ray.streaming.api.stream.StreamSink;
import org.ray.streaming.plan.Plan; import org.ray.streaming.jobgraph.JobGraph;
import org.ray.streaming.plan.PlanBuilder; import org.ray.streaming.jobgraph.JobGraphBuilder;
import org.ray.streaming.schedule.JobScheduler; import org.ray.streaming.schedule.JobScheduler;
/** /**
@ -20,15 +20,21 @@ import org.ray.streaming.schedule.JobScheduler;
public class StreamingContext implements Serializable { public class StreamingContext implements Serializable {
private transient AtomicInteger idGenerator; private transient AtomicInteger idGenerator;
/** /**
* The sinks of this streaming job. * The sinks of this streaming job.
*/ */
private List<StreamSink> streamSinks; private List<StreamSink> streamSinks;
private Map<String, Object> jobConfig;
/**
* The user custom streaming job configuration.
*/
private Map<String, String> jobConfig;
/** /**
* The logic plan. * The logic plan.
*/ */
private Plan plan; private JobGraph jobGraph;
private StreamingContext() { private StreamingContext() {
this.idGenerator = new AtomicInteger(0); this.idGenerator = new AtomicInteger(0);
@ -43,17 +49,17 @@ public class StreamingContext implements Serializable {
/** /**
* Construct job DAG, and execute the job. * Construct job DAG, and execute the job.
*/ */
public void execute() { public void execute(String jobName) {
PlanBuilder planBuilder = new PlanBuilder(this.streamSinks); JobGraphBuilder jobGraphBuilder = new JobGraphBuilder(this.streamSinks, jobName);
this.plan = planBuilder.buildPlan(); this.jobGraph = jobGraphBuilder.build();
plan.printPlan(); jobGraph.printJobGraph();
ServiceLoader<JobScheduler> serviceLoader = ServiceLoader.load(JobScheduler.class); ServiceLoader<JobScheduler> serviceLoader = ServiceLoader.load(JobScheduler.class);
Iterator<JobScheduler> iterator = serviceLoader.iterator(); Iterator<JobScheduler> iterator = serviceLoader.iterator();
Preconditions.checkArgument(iterator.hasNext(), Preconditions.checkArgument(iterator.hasNext(),
"No JobScheduler implementation has been provided."); "No JobScheduler implementation has been provided.");
JobScheduler jobSchedule = iterator.next(); JobScheduler jobSchedule = iterator.next();
jobSchedule.schedule(plan, jobConfig); jobSchedule.schedule(jobGraph, jobConfig);
} }
public int generateId() { public int generateId() {
@ -64,7 +70,7 @@ public class StreamingContext implements Serializable {
streamSinks.add(streamSink); streamSinks.add(streamSink);
} }
public void withConfig(Map<String, Object> jobConfig) { public void withConfig(Map<String, String> jobConfig) {
this.jobConfig = jobConfig; this.jobConfig = jobConfig;
} }
} }

View file

@ -1,7 +1,6 @@
package org.ray.streaming.api.partition.impl; package org.ray.streaming.api.partition.impl;
import java.util.stream.IntStream; import java.util.stream.IntStream;
import org.ray.streaming.api.partition.Partition; import org.ray.streaming.api.partition.Partition;
/** /**

View file

@ -1,18 +1,18 @@
package org.ray.streaming.plan; package org.ray.streaming.jobgraph;
import java.io.Serializable; import java.io.Serializable;
import org.ray.streaming.api.partition.Partition; import org.ray.streaming.api.partition.Partition;
/** /**
* PlanEdge is connection and partition rules of upstream and downstream execution nodes. * Job edge is connection and partition rules of upstream and downstream execution nodes.
*/ */
public class PlanEdge implements Serializable { public class JobEdge implements Serializable {
private int srcVertexId; private int srcVertexId;
private int targetVertexId; private int targetVertexId;
private Partition partition; private Partition partition;
public PlanEdge(int srcVertexId, int targetVertexId, Partition partition) { public JobEdge(int srcVertexId, int targetVertexId, Partition partition) {
this.srcVertexId = srcVertexId; this.srcVertexId = srcVertexId;
this.targetVertexId = targetVertexId; this.targetVertexId = targetVertexId;
this.partition = partition; this.partition = partition;

View file

@ -0,0 +1,103 @@
package org.ray.streaming.jobgraph;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Job graph, the logical plan of streaming job.
*/
public class JobGraph implements Serializable {
private static final Logger LOG = LoggerFactory.getLogger(JobGraph.class);
private final String jobName;
private final Map<String, String> jobConfig;
private List<JobVertex> jobVertexList;
private List<JobEdge> jobEdgeList;
private String digraph;
public JobGraph(String jobName, Map<String, String> jobConfig) {
this.jobName = jobName;
this.jobConfig = jobConfig;
this.jobVertexList = new ArrayList<>();
this.jobEdgeList = new ArrayList<>();
}
/**
* Generate direct-graph(made up of a set of vertices and connected by edges)
* by current job graph for simple log printing.
* @return Digraph in string type.
*
* Notice:
* This is temporarily implemented in hard code.
* May use 'guru.nidi:graphviz-java' as 3rd dependency in the future if needed.
*/
public String generateDigraph() {
StringBuilder digraph = new StringBuilder();
digraph.append("digraph ").append(jobName + " ").append(" {");
for (JobEdge jobEdge: jobEdgeList) {
String srcNode = null;
String targetNode = null;
for (JobVertex jobVertex : jobVertexList) {
if (jobEdge.getSrcVertexId() == jobVertex.getVertexId()) {
srcNode = jobVertex.getVertexId() + "-" + jobVertex.getStreamOperator().getName();
} else if (jobEdge.getTargetVertexId() == jobVertex.getVertexId()) {
targetNode = jobVertex.getVertexId() + "-" + jobVertex.getStreamOperator().getName();
}
}
digraph.append(System.getProperty("line.separator"));
digraph.append(srcNode).append(" -> ").append(targetNode);
}
digraph.append(System.getProperty("line.separator")).append("}");
this.digraph = digraph.toString();
return this.digraph;
}
public void addVertex(JobVertex vertex) {
this.jobVertexList.add(vertex);
}
public void addEdge(JobEdge jobEdge) {
this.jobEdgeList.add(jobEdge);
}
public List<JobVertex> getJobVertexList() {
return jobVertexList;
}
public List<JobEdge> getJobEdgeList() {
return jobEdgeList;
}
public String getDigraph() {
return digraph;
}
public String getJobName() {
return jobName;
}
public Map<String, String> getJobConfig() {
return jobConfig;
}
public void printJobGraph() {
if (!LOG.isInfoEnabled()) {
return;
}
LOG.info("Printing job graph:");
for (JobVertex jobVertex : jobVertexList) {
LOG.info(jobVertex.toString());
}
for (JobEdge jobEdge : jobEdgeList) {
LOG.info(jobEdge.toString());
}
}
}

View file

@ -1,6 +1,8 @@
package org.ray.streaming.plan; package org.ray.streaming.jobgraph;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.ray.streaming.api.stream.DataStream; import org.ray.streaming.api.stream.DataStream;
import org.ray.streaming.api.stream.Stream; import org.ray.streaming.api.stream.Stream;
@ -9,24 +11,33 @@ import org.ray.streaming.api.stream.StreamSource;
import org.ray.streaming.operator.StreamOperator; import org.ray.streaming.operator.StreamOperator;
import org.ray.streaming.python.stream.PythonDataStream; import org.ray.streaming.python.stream.PythonDataStream;
public class PlanBuilder { public class JobGraphBuilder {
private Plan plan; private JobGraph jobGraph;
private AtomicInteger edgeIdGenerator; private AtomicInteger edgeIdGenerator;
private List<StreamSink> streamSinkList; private List<StreamSink> streamSinkList;
public PlanBuilder(List<StreamSink> streamSinkList) { public JobGraphBuilder(List<StreamSink> streamSinkList) {
this.plan = new Plan(); this(streamSinkList, "job-" + System.currentTimeMillis());
}
public JobGraphBuilder(List<StreamSink> streamSinkList, String jobName) {
this(streamSinkList, jobName, new HashMap<>());
}
public JobGraphBuilder(List<StreamSink> streamSinkList, String jobName,
Map<String, String> jobConfig) {
this.jobGraph = new JobGraph(jobName, jobConfig);
this.streamSinkList = streamSinkList; this.streamSinkList = streamSinkList;
this.edgeIdGenerator = new AtomicInteger(0); this.edgeIdGenerator = new AtomicInteger(0);
} }
public Plan buildPlan() { public JobGraph build() {
for (StreamSink streamSink : streamSinkList) { for (StreamSink streamSink : streamSinkList) {
processStream(streamSink); processStream(streamSink);
} }
return this.plan; return this.jobGraph;
} }
private void processStream(Stream stream) { private void processStream(Stream stream) {
@ -34,26 +45,26 @@ public class PlanBuilder {
int parallelism = stream.getParallelism(); int parallelism = stream.getParallelism();
StreamOperator streamOperator = stream.getOperator(); StreamOperator streamOperator = stream.getOperator();
PlanVertex planVertex = null; JobVertex jobVertex = null;
if (stream instanceof StreamSink) { if (stream instanceof StreamSink) {
planVertex = new PlanVertex(vertexId, parallelism, VertexType.SINK, streamOperator); jobVertex = new JobVertex(vertexId, parallelism, VertexType.SINK, streamOperator);
Stream parentStream = stream.getInputStream(); Stream parentStream = stream.getInputStream();
int inputVertexId = parentStream.getId(); int inputVertexId = parentStream.getId();
PlanEdge planEdge = new PlanEdge(inputVertexId, vertexId, parentStream.getPartition()); JobEdge jobEdge = new JobEdge(inputVertexId, vertexId, parentStream.getPartition());
this.plan.addEdge(planEdge); this.jobGraph.addEdge(jobEdge);
processStream(parentStream); processStream(parentStream);
} else if (stream instanceof StreamSource) { } else if (stream instanceof StreamSource) {
planVertex = new PlanVertex(vertexId, parallelism, VertexType.SOURCE, streamOperator); jobVertex = new JobVertex(vertexId, parallelism, VertexType.SOURCE, streamOperator);
} else if (stream instanceof DataStream || stream instanceof PythonDataStream) { } else if (stream instanceof DataStream || stream instanceof PythonDataStream) {
planVertex = new PlanVertex(vertexId, parallelism, VertexType.PROCESS, streamOperator); jobVertex = new JobVertex(vertexId, parallelism, VertexType.PROCESS, streamOperator);
Stream parentStream = stream.getInputStream(); Stream parentStream = stream.getInputStream();
int inputVertexId = parentStream.getId(); int inputVertexId = parentStream.getId();
PlanEdge planEdge = new PlanEdge(inputVertexId, vertexId, parentStream.getPartition()); JobEdge jobEdge = new JobEdge(inputVertexId, vertexId, parentStream.getPartition());
this.plan.addEdge(planEdge); this.jobGraph.addEdge(jobEdge);
processStream(parentStream); processStream(parentStream);
} }
this.plan.addVertex(planVertex); this.jobGraph.addVertex(jobVertex);
} }
private int getEdgeId() { private int getEdgeId() {

View file

@ -1,19 +1,20 @@
package org.ray.streaming.plan; package org.ray.streaming.jobgraph;
import com.google.common.base.MoreObjects;
import java.io.Serializable; import java.io.Serializable;
import org.ray.streaming.operator.StreamOperator; import org.ray.streaming.operator.StreamOperator;
/** /**
* PlanVertex is a cell node where logic is executed. * Job vertex is a cell node where logic is executed.
*/ */
public class PlanVertex implements Serializable { public class JobVertex implements Serializable {
private int vertexId; private int vertexId;
private int parallelism; private int parallelism;
private VertexType vertexType; private VertexType vertexType;
private StreamOperator streamOperator; private StreamOperator streamOperator;
public PlanVertex(int vertexId, int parallelism, VertexType vertexType, public JobVertex(int vertexId, int parallelism, VertexType vertexType,
StreamOperator streamOperator) { StreamOperator streamOperator) {
this.vertexId = vertexId; this.vertexId = vertexId;
this.parallelism = parallelism; this.parallelism = parallelism;
@ -39,11 +40,11 @@ public class PlanVertex implements Serializable {
@Override @Override
public String toString() { public String toString() {
return "PlanVertex{" + return MoreObjects.toStringHelper(this)
"vertexId=" + vertexId + .add("vertexId", vertexId)
", parallelism=" + parallelism + .add("parallelism", parallelism)
", vertexType=" + vertexType + .add("vertexType", vertexType)
", streamOperator=" + streamOperator + .add("streamOperator", streamOperator)
'}'; .toString();
} }
} }

View file

@ -1,4 +1,4 @@
package org.ray.streaming.plan; package org.ray.streaming.jobgraph;
/** /**
* Different roles for a node. * Different roles for a node.

View file

@ -9,29 +9,33 @@ import org.ray.streaming.message.Record;
public abstract class StreamOperator<F extends Function> implements Operator { public abstract class StreamOperator<F extends Function> implements Operator {
protected String name;
protected F function; protected F function;
protected List<Collector> collectorList; protected List<Collector> collectorList;
protected RuntimeContext runtimeContext; protected RuntimeContext runtimeContext;
public StreamOperator(F function) { public StreamOperator(F function) {
this.name = getClass().getSimpleName();
this.function = function; this.function = function;
} }
@Override
public void open(List<Collector> collectorList, RuntimeContext runtimeContext) { public void open(List<Collector> collectorList, RuntimeContext runtimeContext) {
this.collectorList = collectorList; this.collectorList = collectorList;
this.runtimeContext = runtimeContext; this.runtimeContext = runtimeContext;
} }
@Override
public void finish() { public void finish() {
} }
@Override
public void close() { public void close() {
} }
protected void collect(Record record) { protected void collect(Record record) {
for (Collector collector : this.collectorList) { for (Collector collector : this.collectorList) {
collector.collect(record); collector.collect(record);
@ -44,4 +48,7 @@ public abstract class StreamOperator<F extends Function> implements Operator {
} }
} }
public String getName() {
return name;
}
} }

View file

@ -1,58 +0,0 @@
package org.ray.streaming.plan;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The logical execution plan.
*/
public class Plan implements Serializable {
private static final Logger LOGGER = LoggerFactory.getLogger(Plan.class);
private List<PlanVertex> planVertexList;
private List<PlanEdge> planEdgeList;
public Plan() {
this.planVertexList = new ArrayList<>();
this.planEdgeList = new ArrayList<>();
}
public void addVertex(PlanVertex vertex) {
this.planVertexList.add(vertex);
}
public void addEdge(PlanEdge planEdge) {
this.planEdgeList.add(planEdge);
}
public List<PlanVertex> getPlanVertexList() {
return planVertexList;
}
public List<PlanEdge> getPlanEdgeList() {
return planEdgeList;
}
public String getGraphVizPlan() {
return "";
}
public void printPlan() {
if (!LOGGER.isInfoEnabled()) {
return;
}
LOGGER.info("Printing logic plan:");
for (PlanVertex planVertex : planVertexList) {
LOGGER.info(planVertex.toString());
}
for (PlanEdge planEdge : planEdgeList) {
LOGGER.info(planEdge.toString());
}
}
}

View file

@ -2,8 +2,7 @@ package org.ray.streaming.schedule;
import java.util.Map; import java.util.Map;
import org.ray.streaming.jobgraph.JobGraph;
import org.ray.streaming.plan.Plan;
/** /**
* Interface of the job scheduler. * Interface of the job scheduler.
@ -13,7 +12,7 @@ public interface JobScheduler {
/** /**
* Assign logical plan to physical execution graph, and schedule job to run. * Assign logical plan to physical execution graph, and schedule job to run.
* *
* @param plan The logical plan. * @param jobGraph The logical plan.
*/ */
void schedule(Plan plan, Map<String, Object> conf); void schedule(JobGraph jobGraph, Map<String, String> conf);
} }

View file

@ -0,0 +1,96 @@
package org.ray.streaming.jobgraph;
import com.google.common.collect.Lists;
import java.util.List;
import org.ray.streaming.api.context.StreamingContext;
import org.ray.streaming.api.partition.impl.KeyPartition;
import org.ray.streaming.api.partition.impl.RoundRobinPartition;
import org.ray.streaming.api.stream.DataStream;
import org.ray.streaming.api.stream.DataStreamSource;
import org.ray.streaming.api.stream.StreamSink;
import org.ray.streaming.api.stream.StreamSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
import org.testng.annotations.Test;
public class JobGraphBuilderTest {
private static final Logger LOG = LoggerFactory.getLogger(JobGraphBuilderTest.class);
@Test
public void testDataSync() {
JobGraph jobGraph = buildDataSyncJobGraph();
List<JobVertex> jobVertexList = jobGraph.getJobVertexList();
List<JobEdge> jobEdgeList = jobGraph.getJobEdgeList();
Assert.assertEquals(jobVertexList.size(), 2);
Assert.assertEquals(jobEdgeList.size(), 1);
JobEdge jobEdge = jobEdgeList.get(0);
Assert.assertEquals(jobEdge.getPartition().getClass(), RoundRobinPartition.class);
JobVertex sinkVertex = jobVertexList.get(1);
JobVertex sourceVertex = jobVertexList.get(0);
Assert.assertEquals(sinkVertex.getVertexType(), VertexType.SINK);
Assert.assertEquals(sourceVertex.getVertexType(), VertexType.SOURCE);
}
public JobGraph buildDataSyncJobGraph() {
StreamingContext streamingContext = StreamingContext.buildContext();
DataStream<String> dataStream = DataStreamSource.buildSource(streamingContext,
Lists.newArrayList("a", "b", "c"));
StreamSink streamSink = dataStream.sink(x -> LOG.info(x));
JobGraphBuilder jobGraphBuilder = new JobGraphBuilder(Lists.newArrayList(streamSink));
JobGraph jobGraph = jobGraphBuilder.build();
return jobGraph;
}
@Test
public void testKeyByJobGraph() {
JobGraph jobGraph = buildKeyByJobGraph();
List<JobVertex> jobVertexList = jobGraph.getJobVertexList();
List<JobEdge> jobEdgeList = jobGraph.getJobEdgeList();
Assert.assertEquals(jobVertexList.size(), 3);
Assert.assertEquals(jobEdgeList.size(), 2);
JobVertex source = jobVertexList.get(0);
JobVertex map = jobVertexList.get(1);
JobVertex sink = jobVertexList.get(2);
Assert.assertEquals(source.getVertexType(), VertexType.SOURCE);
Assert.assertEquals(map.getVertexType(), VertexType.PROCESS);
Assert.assertEquals(sink.getVertexType(), VertexType.SINK);
JobEdge keyBy2Sink = jobEdgeList.get(0);
JobEdge source2KeyBy = jobEdgeList.get(1);
Assert.assertEquals(keyBy2Sink.getPartition().getClass(), KeyPartition.class);
Assert.assertEquals(source2KeyBy.getPartition().getClass(), RoundRobinPartition.class);
}
public JobGraph buildKeyByJobGraph() {
StreamingContext streamingContext = StreamingContext.buildContext();
DataStream<String> dataStream = DataStreamSource.buildSource(streamingContext,
Lists.newArrayList("1", "2", "3", "4"));
StreamSink streamSink = dataStream.keyBy(x -> x)
.sink(x -> LOG.info(x));
JobGraphBuilder jobGraphBuilder = new JobGraphBuilder(Lists.newArrayList(streamSink));
JobGraph jobGraph = jobGraphBuilder.build();
return jobGraph;
}
@Test
public void testJobGraphViz() {
JobGraph jobGraph = buildKeyByJobGraph();
jobGraph.generateDigraph();
String diGraph = jobGraph.getDigraph();
System.out.println(diGraph);
Assert.assertTrue(diGraph.contains("1-SourceOperator -> 2-KeyByOperator"));
Assert.assertTrue(diGraph.contains("2-KeyByOperator -> 3-SinkOperator"));
}
}

View file

@ -1,87 +0,0 @@
package org.ray.streaming.plan;
import com.google.common.collect.Lists;
import org.ray.streaming.api.context.StreamingContext;
import org.ray.streaming.api.partition.impl.KeyPartition;
import org.ray.streaming.api.partition.impl.RoundRobinPartition;
import org.ray.streaming.api.stream.DataStream;
import org.ray.streaming.api.stream.DataStreamSink;
import org.ray.streaming.api.stream.DataStreamSource;
import java.util.List;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
import org.testng.annotations.Test;
public class PlanBuilderTest {
private static final Logger LOGGER = LoggerFactory.getLogger(PlanBuilderTest.class);
@Test
public void testDataSync() {
Plan plan = buildDataSyncPlan();
List<PlanVertex> planVertexList = plan.getPlanVertexList();
List<PlanEdge> planEdgeList = plan.getPlanEdgeList();
Assert.assertEquals(planVertexList.size(), 2);
Assert.assertEquals(planEdgeList.size(), 1);
PlanEdge planEdge = planEdgeList.get(0);
Assert.assertEquals(planEdge.getPartition().getClass(), RoundRobinPartition.class);
PlanVertex sinkVertex = planVertexList.get(1);
PlanVertex sourceVertex = planVertexList.get(0);
Assert.assertEquals(sinkVertex.getVertexType(), VertexType.SINK);
Assert.assertEquals(sourceVertex.getVertexType(), VertexType.SOURCE);
}
public Plan buildDataSyncPlan() {
StreamingContext streamingContext = StreamingContext.buildContext();
DataStream<String> dataStream = DataStreamSource.buildSource(streamingContext,
Lists.newArrayList("a", "b", "c"));
DataStreamSink streamSink = dataStream.sink(x -> LOGGER.info(x));
PlanBuilder planBuilder = new PlanBuilder(Lists.newArrayList(streamSink));
Plan plan = planBuilder.buildPlan();
return plan;
}
@Test
public void testKeyByPlan() {
Plan plan = buildKeyByPlan();
List<PlanVertex> planVertexList = plan.getPlanVertexList();
List<PlanEdge> planEdgeList = plan.getPlanEdgeList();
Assert.assertEquals(planVertexList.size(), 3);
Assert.assertEquals(planEdgeList.size(), 2);
PlanVertex source = planVertexList.get(0);
PlanVertex map = planVertexList.get(1);
PlanVertex sink = planVertexList.get(2);
Assert.assertEquals(source.getVertexType(), VertexType.SOURCE);
Assert.assertEquals(map.getVertexType(), VertexType.PROCESS);
Assert.assertEquals(sink.getVertexType(), VertexType.SINK);
PlanEdge keyBy2Sink = planEdgeList.get(0);
PlanEdge source2KeyBy = planEdgeList.get(1);
Assert.assertEquals(keyBy2Sink.getPartition().getClass(), KeyPartition.class);
Assert.assertEquals(source2KeyBy.getPartition().getClass(), RoundRobinPartition.class);
}
public Plan buildKeyByPlan() {
StreamingContext streamingContext = StreamingContext.buildContext();
DataStream<String> dataStream = DataStreamSource.buildSource(streamingContext,
Lists.newArrayList("1", "2", "3", "4"));
DataStreamSink streamSink = dataStream.keyBy(x -> x)
.sink(x -> LOGGER.info(x));
PlanBuilder planBuilder = new PlanBuilder(Lists.newArrayList(streamSink));
Plan plan = planBuilder.buildPlan();
return plan;
}
}

View file

@ -2,7 +2,6 @@ package org.ray.streaming.runtime.cluster;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.ray.api.Ray; import org.ray.api.Ray;
import org.ray.api.RayActor; import org.ray.api.RayActor;
import org.ray.streaming.runtime.worker.JobWorker; import org.ray.streaming.runtime.worker.JobWorker;

View file

@ -1,7 +1,6 @@
package org.ray.streaming.runtime.core.graph; package org.ray.streaming.runtime.core.graph;
import java.io.Serializable; import java.io.Serializable;
import org.ray.streaming.api.partition.Partition; import org.ray.streaming.api.partition.Partition;
/** /**

View file

@ -6,7 +6,6 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.ray.api.RayActor; import org.ray.api.RayActor;
import org.ray.streaming.runtime.worker.JobWorker; import org.ray.streaming.runtime.worker.JobWorker;

View file

@ -3,13 +3,14 @@ package org.ray.streaming.runtime.core.graph;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.ray.streaming.jobgraph.VertexType;
import org.ray.streaming.operator.StreamOperator; import org.ray.streaming.operator.StreamOperator;
import org.ray.streaming.plan.VertexType;
/** /**
* A node in the physical execution graph. * A node in the physical execution graph.
*/ */
public class ExecutionNode implements Serializable { public class ExecutionNode implements Serializable {
private int nodeId; private int nodeId;
private int parallelism; private int parallelism;
private NodeType nodeType; private NodeType nodeType;

View file

@ -1,7 +1,6 @@
package org.ray.streaming.runtime.core.graph; package org.ray.streaming.runtime.core.graph;
import java.io.Serializable; import java.io.Serializable;
import org.ray.api.RayActor; import org.ray.api.RayActor;
import org.ray.streaming.runtime.worker.JobWorker; import org.ray.streaming.runtime.worker.JobWorker;

View file

@ -6,8 +6,8 @@ import java.util.Map;
import org.ray.api.Ray; import org.ray.api.Ray;
import org.ray.api.RayActor; import org.ray.api.RayActor;
import org.ray.api.RayObject; import org.ray.api.RayObject;
import org.ray.streaming.plan.Plan; import org.ray.streaming.jobgraph.JobGraph;
import org.ray.streaming.plan.PlanVertex; import org.ray.streaming.jobgraph.JobVertex;
import org.ray.streaming.runtime.cluster.ResourceManager; import org.ray.streaming.runtime.cluster.ResourceManager;
import org.ray.streaming.runtime.core.graph.ExecutionGraph; import org.ray.streaming.runtime.core.graph.ExecutionGraph;
import org.ray.streaming.runtime.core.graph.ExecutionNode; import org.ray.streaming.runtime.core.graph.ExecutionNode;
@ -21,8 +21,8 @@ import org.ray.streaming.schedule.JobScheduler;
* from ResourceManager. * from ResourceManager.
*/ */
public class JobSchedulerImpl implements JobScheduler { public class JobSchedulerImpl implements JobScheduler {
private Plan plan; private JobGraph jobGraph;
private Map<String, Object> jobConfig; private Map<String, String> jobConfig;
private ResourceManager resourceManager; private ResourceManager resourceManager;
private TaskAssigner taskAssigner; private TaskAssigner taskAssigner;
@ -35,14 +35,14 @@ public class JobSchedulerImpl implements JobScheduler {
* Schedule physical plan to execution graph, and call streaming worker to init and run. * Schedule physical plan to execution graph, and call streaming worker to init and run.
*/ */
@Override @Override
public void schedule(Plan plan, Map<String, Object> jobConfig) { public void schedule(JobGraph jobGraph, Map<String, String> jobConfig) {
this.jobConfig = jobConfig; this.jobConfig = jobConfig;
this.plan = plan; this.jobGraph = jobGraph;
System.setProperty("ray.raylet.config.num_workers_per_process_java", "1"); System.setProperty("ray.raylet.config.num_workers_per_process_java", "1");
Ray.init(); Ray.init();
List<RayActor<JobWorker>> workers = this.resourceManager.createWorkers(getPlanWorker()); List<RayActor<JobWorker>> workers = this.resourceManager.createWorkers(getPlanWorker());
ExecutionGraph executionGraph = this.taskAssigner.assign(this.plan, workers); ExecutionGraph executionGraph = this.taskAssigner.assign(this.jobGraph, workers);
List<ExecutionNode> executionNodes = executionGraph.getExecutionNodeList(); List<ExecutionNode> executionNodes = executionGraph.getExecutionNodeList();
List<RayObject<Boolean>> waits = new ArrayList<>(); List<RayObject<Boolean>> waits = new ArrayList<>();
@ -59,7 +59,7 @@ public class JobSchedulerImpl implements JobScheduler {
} }
private int getPlanWorker() { private int getPlanWorker() {
List<PlanVertex> planVertexList = plan.getPlanVertexList(); List<JobVertex> jobVertexList = jobGraph.getJobVertexList();
return planVertexList.stream().map(PlanVertex::getParallelism).reduce(0, Integer::sum); return jobVertexList.stream().map(JobVertex::getParallelism).reduce(0, Integer::sum);
} }
} }

View file

@ -3,7 +3,7 @@ package org.ray.streaming.runtime.schedule;
import java.io.Serializable; import java.io.Serializable;
import java.util.List; import java.util.List;
import org.ray.api.RayActor; import org.ray.api.RayActor;
import org.ray.streaming.plan.Plan; import org.ray.streaming.jobgraph.JobGraph;
import org.ray.streaming.runtime.core.graph.ExecutionGraph; import org.ray.streaming.runtime.core.graph.ExecutionGraph;
import org.ray.streaming.runtime.worker.JobWorker; import org.ray.streaming.runtime.worker.JobWorker;
@ -15,6 +15,6 @@ public interface TaskAssigner extends Serializable {
/** /**
* Assign logical plan to physical execution graph. * Assign logical plan to physical execution graph.
*/ */
ExecutionGraph assign(Plan plan, List<RayActor<JobWorker>> workers); ExecutionGraph assign(JobGraph jobGraph, List<RayActor<JobWorker>> workers);
} }

View file

@ -6,9 +6,9 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.ray.api.RayActor; import org.ray.api.RayActor;
import org.ray.streaming.plan.Plan; import org.ray.streaming.jobgraph.JobEdge;
import org.ray.streaming.plan.PlanEdge; import org.ray.streaming.jobgraph.JobGraph;
import org.ray.streaming.plan.PlanVertex; import org.ray.streaming.jobgraph.JobVertex;
import org.ray.streaming.runtime.core.graph.ExecutionEdge; import org.ray.streaming.runtime.core.graph.ExecutionEdge;
import org.ray.streaming.runtime.core.graph.ExecutionGraph; import org.ray.streaming.runtime.core.graph.ExecutionGraph;
import org.ray.streaming.runtime.core.graph.ExecutionNode; import org.ray.streaming.runtime.core.graph.ExecutionNode;
@ -20,37 +20,37 @@ public class TaskAssignerImpl implements TaskAssigner {
/** /**
* Assign an optimized logical plan to execution graph. * Assign an optimized logical plan to execution graph.
* *
* @param plan The logical plan. * @param jobGraph The logical plan.
* @param workers The worker actors. * @param workers The worker actors.
* @return The physical execution graph. * @return The physical execution graph.
*/ */
@Override @Override
public ExecutionGraph assign(Plan plan, List<RayActor<JobWorker>> workers) { public ExecutionGraph assign(JobGraph jobGraph, List<RayActor<JobWorker>> workers) {
List<PlanVertex> planVertices = plan.getPlanVertexList(); List<JobVertex> jobVertices = jobGraph.getJobVertexList();
List<PlanEdge> planEdges = plan.getPlanEdgeList(); List<JobEdge> jobEdges = jobGraph.getJobEdgeList();
int taskId = 0; int taskId = 0;
Map<Integer, ExecutionNode> idToExecutionNode = new HashMap<>(); Map<Integer, ExecutionNode> idToExecutionNode = new HashMap<>();
for (PlanVertex planVertex : planVertices) { for (JobVertex jobVertex : jobVertices) {
ExecutionNode executionNode = new ExecutionNode(planVertex.getVertexId(), ExecutionNode executionNode = new ExecutionNode(jobVertex.getVertexId(),
planVertex.getParallelism()); jobVertex.getParallelism());
executionNode.setNodeType(planVertex.getVertexType()); executionNode.setNodeType(jobVertex.getVertexType());
List<ExecutionTask> vertexTasks = new ArrayList<>(); List<ExecutionTask> vertexTasks = new ArrayList<>();
for (int taskIndex = 0; taskIndex < planVertex.getParallelism(); taskIndex++) { for (int taskIndex = 0; taskIndex < jobVertex.getParallelism(); taskIndex++) {
vertexTasks.add(new ExecutionTask(taskId, taskIndex, workers.get(taskId))); vertexTasks.add(new ExecutionTask(taskId, taskIndex, workers.get(taskId)));
taskId++; taskId++;
} }
executionNode.setExecutionTasks(vertexTasks); executionNode.setExecutionTasks(vertexTasks);
executionNode.setStreamOperator(planVertex.getStreamOperator()); executionNode.setStreamOperator(jobVertex.getStreamOperator());
idToExecutionNode.put(executionNode.getNodeId(), executionNode); idToExecutionNode.put(executionNode.getNodeId(), executionNode);
} }
for (PlanEdge planEdge : planEdges) { for (JobEdge jobEdge : jobEdges) {
int srcNodeId = planEdge.getSrcVertexId(); int srcNodeId = jobEdge.getSrcVertexId();
int targetNodeId = planEdge.getTargetVertexId(); int targetNodeId = jobEdge.getTargetVertexId();
ExecutionEdge executionEdge = new ExecutionEdge(srcNodeId, targetNodeId, ExecutionEdge executionEdge = new ExecutionEdge(srcNodeId, targetNodeId,
planEdge.getPartition()); jobEdge.getPartition());
idToExecutionNode.get(srcNodeId).addExecutionEdge(executionEdge); idToExecutionNode.get(srcNodeId).addExecutionEdge(executionEdge);
idToExecutionNode.get(targetNodeId).addInputEdge(executionEdge); idToExecutionNode.get(targetNodeId).addInputEdge(executionEdge);
} }

View file

@ -36,7 +36,7 @@ public class JobWorker implements Serializable {
} }
private int taskId; private int taskId;
private Map<String, Object> config; private Map<String, String> config;
private WorkerContext workerContext; private WorkerContext workerContext;
private ExecutionNode executionNode; private ExecutionNode executionNode;
private ExecutionTask executionTask; private ExecutionTask executionTask;
@ -88,7 +88,7 @@ public class JobWorker implements Serializable {
return taskId; return taskId;
} }
public Map<String, Object> getConfig() { public Map<String, String> getConfig() {
return config; return config;
} }

View file

@ -3,7 +3,6 @@ package org.ray.streaming.runtime.worker.context;
import static org.ray.streaming.util.Config.STREAMING_BATCH_MAX_COUNT; import static org.ray.streaming.util.Config.STREAMING_BATCH_MAX_COUNT;
import java.util.Map; import java.util.Map;
import org.ray.streaming.api.context.RuntimeContext; import org.ray.streaming.api.context.RuntimeContext;
import org.ray.streaming.runtime.core.graph.ExecutionTask; import org.ray.streaming.runtime.core.graph.ExecutionTask;
@ -16,16 +15,16 @@ public class RayRuntimeContext implements RuntimeContext {
private int parallelism; private int parallelism;
private Long batchId; private Long batchId;
private final Long maxBatch; private final Long maxBatch;
private Map<String, Object> config; private Map<String, String> config;
public RayRuntimeContext(ExecutionTask executionTask, Map<String, Object> config, public RayRuntimeContext(ExecutionTask executionTask, Map<String, String> config,
int parallelism) { int parallelism) {
this.taskId = executionTask.getTaskId(); this.taskId = executionTask.getTaskId();
this.config = config; this.config = config;
this.taskIndex = executionTask.getTaskIndex(); this.taskIndex = executionTask.getTaskIndex();
this.parallelism = parallelism; this.parallelism = parallelism;
if (config.containsKey(STREAMING_BATCH_MAX_COUNT)) { if (config.containsKey(STREAMING_BATCH_MAX_COUNT)) {
this.maxBatch = Long.valueOf(String.valueOf(config.get(STREAMING_BATCH_MAX_COUNT))); this.maxBatch = Long.valueOf(config.get(STREAMING_BATCH_MAX_COUNT));
} else { } else {
this.maxBatch = Long.MAX_VALUE; this.maxBatch = Long.MAX_VALUE;
} }

View file

@ -11,9 +11,9 @@ public class WorkerContext implements Serializable {
private int taskId; private int taskId;
private ExecutionGraph executionGraph; private ExecutionGraph executionGraph;
private Map<String, Object> config; private Map<String, String> config;
public WorkerContext(int taskId, ExecutionGraph executionGraph, Map<String, Object> jobConfig) { public WorkerContext(int taskId, ExecutionGraph executionGraph, Map<String, String> jobConfig) {
this.taskId = taskId; this.taskId = taskId;
this.executionGraph = executionGraph; this.executionGraph = executionGraph;
this.config = jobConfig; this.config = jobConfig;
@ -35,7 +35,7 @@ public class WorkerContext implements Serializable {
this.executionGraph = executionGraph; this.executionGraph = executionGraph;
} }
public Map<String, Object> getConfig() { public Map<String, String> getConfig() {
return config; return config;
} }
} }

View file

@ -30,8 +30,8 @@ public class WordCountTest extends BaseUnitTest implements Serializable {
@Test @Test
public void testWordCount() { public void testWordCount() {
StreamingContext streamingContext = StreamingContext.buildContext(); StreamingContext streamingContext = StreamingContext.buildContext();
Map<String, Object> config = new HashMap<>(); Map<String, String> config = new HashMap<>();
config.put(Config.STREAMING_BATCH_MAX_COUNT, 1); config.put(Config.STREAMING_BATCH_MAX_COUNT, "1");
config.put(Config.CHANNEL_TYPE, Config.MEMORY_CHANNEL); config.put(Config.CHANNEL_TYPE, Config.MEMORY_CHANNEL);
streamingContext.withConfig(config); streamingContext.withConfig(config);
List<String> text = new ArrayList<>(); List<String> text = new ArrayList<>();
@ -50,7 +50,7 @@ public class WordCountTest extends BaseUnitTest implements Serializable {
.sink((SinkFunction<WordAndCount>) .sink((SinkFunction<WordAndCount>)
result -> wordCount.put(result.word, result.count)); result -> wordCount.put(result.word, result.count));
streamingContext.execute(); streamingContext.execute("testWordCount");
// Sleep until the count for every word is computed. // Sleep until the count for every word is computed.
while (wordCount.size() < 3) { while (wordCount.size() < 3) {

View file

@ -19,8 +19,8 @@ import org.ray.streaming.runtime.core.graph.ExecutionGraph;
import org.ray.streaming.runtime.core.graph.ExecutionNode; import org.ray.streaming.runtime.core.graph.ExecutionNode;
import org.ray.streaming.runtime.core.graph.ExecutionNode.NodeType; import org.ray.streaming.runtime.core.graph.ExecutionNode.NodeType;
import org.ray.streaming.runtime.worker.JobWorker; import org.ray.streaming.runtime.worker.JobWorker;
import org.ray.streaming.plan.Plan; import org.ray.streaming.jobgraph.JobGraph;
import org.ray.streaming.plan.PlanBuilder; import org.ray.streaming.jobgraph.JobGraphBuilder;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.testng.Assert; import org.testng.Assert;
@ -32,15 +32,15 @@ public class TaskAssignerImplTest extends BaseUnitTest {
@Test @Test
public void testTaskAssignImpl() { public void testTaskAssignImpl() {
Plan plan = buildDataSyncPlan(); JobGraph jobGraph = buildDataSyncPlan();
List<RayActor<JobWorker>> workers = new ArrayList<>(); List<RayActor<JobWorker>> workers = new ArrayList<>();
for(int i = 0; i < plan.getPlanVertexList().size(); i++) { for(int i = 0; i < jobGraph.getJobVertexList().size(); i++) {
workers.add(new LocalModeRayActor(ActorId.fromRandom(), ObjectId.fromRandom())); workers.add(new LocalModeRayActor(ActorId.fromRandom(), ObjectId.fromRandom()));
} }
TaskAssigner taskAssigner = new TaskAssignerImpl(); TaskAssigner taskAssigner = new TaskAssignerImpl();
ExecutionGraph executionGraph = taskAssigner.assign(plan, workers); ExecutionGraph executionGraph = taskAssigner.assign(jobGraph, workers);
List<ExecutionNode> executionNodeList = executionGraph.getExecutionNodeList(); List<ExecutionNode> executionNodeList = executionGraph.getExecutionNodeList();
@ -63,14 +63,14 @@ public class TaskAssignerImplTest extends BaseUnitTest {
Assert.assertEquals(sinkNode.getOutputEdges().size(), 0); Assert.assertEquals(sinkNode.getOutputEdges().size(), 0);
} }
public Plan buildDataSyncPlan() { public JobGraph buildDataSyncPlan() {
StreamingContext streamingContext = StreamingContext.buildContext(); StreamingContext streamingContext = StreamingContext.buildContext();
DataStream<String> dataStream = DataStreamSource.buildSource(streamingContext, DataStream<String> dataStream = DataStreamSource.buildSource(streamingContext,
Lists.newArrayList("a", "b", "c")); Lists.newArrayList("a", "b", "c"));
DataStreamSink streamSink = dataStream.sink(x -> LOGGER.info(x)); DataStreamSink streamSink = dataStream.sink(x -> LOGGER.info(x));
PlanBuilder planBuilder = new PlanBuilder(Lists.newArrayList(streamSink)); JobGraphBuilder jobGraphBuilder = new JobGraphBuilder(Lists.newArrayList(streamSink));
Plan plan = planBuilder.buildPlan(); JobGraph jobGraph = jobGraphBuilder.build();
return plan; return jobGraph;
} }
} }

View file

@ -151,8 +151,8 @@ public class StreamingQueueTest extends BaseUnitTest implements Serializable {
Map<String, Integer> wordCount = new ConcurrentHashMap<>(); Map<String, Integer> wordCount = new ConcurrentHashMap<>();
StreamingContext streamingContext = StreamingContext.buildContext(); StreamingContext streamingContext = StreamingContext.buildContext();
Map<String, Object> config = new HashMap<>(); Map<String, String> config = new HashMap<>();
config.put(Config.STREAMING_BATCH_MAX_COUNT, 1); config.put(Config.STREAMING_BATCH_MAX_COUNT, "1");
config.put(Config.CHANNEL_TYPE, Config.NATIVE_CHANNEL); config.put(Config.CHANNEL_TYPE, Config.NATIVE_CHANNEL);
config.put(Config.CHANNEL_SIZE, "100000"); config.put(Config.CHANNEL_SIZE, "100000");
streamingContext.withConfig(config); streamingContext.withConfig(config);
@ -177,7 +177,7 @@ public class StreamingQueueTest extends BaseUnitTest implements Serializable {
serializeResultToFile(resultFile, wordCount); serializeResultToFile(resultFile, wordCount);
}); });
streamingContext.execute(); streamingContext.execute("testWordCount");
Map<String, Integer> checkWordCount = Map<String, Integer> checkWordCount =
(Map<String, Integer>) deserializeResultFromFile(resultFile); (Map<String, Integer>) deserializeResultFromFile(resultFile);