mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[Streaming] Streaming scheduler - part1-1: job graph (#6712)
This commit is contained in:
parent
4227fd1b60
commit
9a4da1951e
27 changed files with 322 additions and 250 deletions
|
@ -1,7 +1,6 @@
|
|||
package org.ray.streaming.api.collector;
|
||||
|
||||
import java.util.List;
|
||||
import org.ray.streaming.api.collector.Collector;
|
||||
import org.ray.streaming.message.Record;
|
||||
|
||||
/**
|
||||
|
|
|
@ -10,8 +10,8 @@ import java.util.Map;
|
|||
import java.util.ServiceLoader;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.ray.streaming.api.stream.StreamSink;
|
||||
import org.ray.streaming.plan.Plan;
|
||||
import org.ray.streaming.plan.PlanBuilder;
|
||||
import org.ray.streaming.jobgraph.JobGraph;
|
||||
import org.ray.streaming.jobgraph.JobGraphBuilder;
|
||||
import org.ray.streaming.schedule.JobScheduler;
|
||||
|
||||
/**
|
||||
|
@ -20,15 +20,21 @@ import org.ray.streaming.schedule.JobScheduler;
|
|||
public class StreamingContext implements Serializable {
|
||||
|
||||
private transient AtomicInteger idGenerator;
|
||||
|
||||
/**
|
||||
* The sinks of this streaming job.
|
||||
*/
|
||||
private List<StreamSink> streamSinks;
|
||||
private Map<String, Object> jobConfig;
|
||||
|
||||
/**
|
||||
* The user custom streaming job configuration.
|
||||
*/
|
||||
private Map<String, String> jobConfig;
|
||||
|
||||
/**
|
||||
* The logic plan.
|
||||
*/
|
||||
private Plan plan;
|
||||
private JobGraph jobGraph;
|
||||
|
||||
private StreamingContext() {
|
||||
this.idGenerator = new AtomicInteger(0);
|
||||
|
@ -43,17 +49,17 @@ public class StreamingContext implements Serializable {
|
|||
/**
|
||||
* Construct job DAG, and execute the job.
|
||||
*/
|
||||
public void execute() {
|
||||
PlanBuilder planBuilder = new PlanBuilder(this.streamSinks);
|
||||
this.plan = planBuilder.buildPlan();
|
||||
plan.printPlan();
|
||||
public void execute(String jobName) {
|
||||
JobGraphBuilder jobGraphBuilder = new JobGraphBuilder(this.streamSinks, jobName);
|
||||
this.jobGraph = jobGraphBuilder.build();
|
||||
jobGraph.printJobGraph();
|
||||
|
||||
ServiceLoader<JobScheduler> serviceLoader = ServiceLoader.load(JobScheduler.class);
|
||||
Iterator<JobScheduler> iterator = serviceLoader.iterator();
|
||||
Preconditions.checkArgument(iterator.hasNext(),
|
||||
"No JobScheduler implementation has been provided.");
|
||||
JobScheduler jobSchedule = iterator.next();
|
||||
jobSchedule.schedule(plan, jobConfig);
|
||||
jobSchedule.schedule(jobGraph, jobConfig);
|
||||
}
|
||||
|
||||
public int generateId() {
|
||||
|
@ -64,7 +70,7 @@ public class StreamingContext implements Serializable {
|
|||
streamSinks.add(streamSink);
|
||||
}
|
||||
|
||||
public void withConfig(Map<String, Object> jobConfig) {
|
||||
public void withConfig(Map<String, String> jobConfig) {
|
||||
this.jobConfig = jobConfig;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package org.ray.streaming.api.partition.impl;
|
||||
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import org.ray.streaming.api.partition.Partition;
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,18 +1,18 @@
|
|||
package org.ray.streaming.plan;
|
||||
package org.ray.streaming.jobgraph;
|
||||
|
||||
import java.io.Serializable;
|
||||
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 targetVertexId;
|
||||
private Partition partition;
|
||||
|
||||
public PlanEdge(int srcVertexId, int targetVertexId, Partition partition) {
|
||||
public JobEdge(int srcVertexId, int targetVertexId, Partition partition) {
|
||||
this.srcVertexId = srcVertexId;
|
||||
this.targetVertexId = targetVertexId;
|
||||
this.partition = partition;
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.ray.streaming.api.stream.DataStream;
|
||||
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.python.stream.PythonDataStream;
|
||||
|
||||
public class PlanBuilder {
|
||||
public class JobGraphBuilder {
|
||||
|
||||
private Plan plan;
|
||||
private JobGraph jobGraph;
|
||||
|
||||
private AtomicInteger edgeIdGenerator;
|
||||
private List<StreamSink> streamSinkList;
|
||||
|
||||
public PlanBuilder(List<StreamSink> streamSinkList) {
|
||||
this.plan = new Plan();
|
||||
public JobGraphBuilder(List<StreamSink> streamSinkList) {
|
||||
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.edgeIdGenerator = new AtomicInteger(0);
|
||||
}
|
||||
|
||||
public Plan buildPlan() {
|
||||
public JobGraph build() {
|
||||
for (StreamSink streamSink : streamSinkList) {
|
||||
processStream(streamSink);
|
||||
}
|
||||
return this.plan;
|
||||
return this.jobGraph;
|
||||
}
|
||||
|
||||
private void processStream(Stream stream) {
|
||||
|
@ -34,26 +45,26 @@ public class PlanBuilder {
|
|||
int parallelism = stream.getParallelism();
|
||||
|
||||
StreamOperator streamOperator = stream.getOperator();
|
||||
PlanVertex planVertex = null;
|
||||
JobVertex jobVertex = null;
|
||||
|
||||
if (stream instanceof StreamSink) {
|
||||
planVertex = new PlanVertex(vertexId, parallelism, VertexType.SINK, streamOperator);
|
||||
jobVertex = new JobVertex(vertexId, parallelism, VertexType.SINK, streamOperator);
|
||||
Stream parentStream = stream.getInputStream();
|
||||
int inputVertexId = parentStream.getId();
|
||||
PlanEdge planEdge = new PlanEdge(inputVertexId, vertexId, parentStream.getPartition());
|
||||
this.plan.addEdge(planEdge);
|
||||
JobEdge jobEdge = new JobEdge(inputVertexId, vertexId, parentStream.getPartition());
|
||||
this.jobGraph.addEdge(jobEdge);
|
||||
processStream(parentStream);
|
||||
} 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) {
|
||||
planVertex = new PlanVertex(vertexId, parallelism, VertexType.PROCESS, streamOperator);
|
||||
jobVertex = new JobVertex(vertexId, parallelism, VertexType.PROCESS, streamOperator);
|
||||
Stream parentStream = stream.getInputStream();
|
||||
int inputVertexId = parentStream.getId();
|
||||
PlanEdge planEdge = new PlanEdge(inputVertexId, vertexId, parentStream.getPartition());
|
||||
this.plan.addEdge(planEdge);
|
||||
JobEdge jobEdge = new JobEdge(inputVertexId, vertexId, parentStream.getPartition());
|
||||
this.jobGraph.addEdge(jobEdge);
|
||||
processStream(parentStream);
|
||||
}
|
||||
this.plan.addVertex(planVertex);
|
||||
this.jobGraph.addVertex(jobVertex);
|
||||
}
|
||||
|
||||
private int getEdgeId() {
|
|
@ -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 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 parallelism;
|
||||
private VertexType vertexType;
|
||||
private StreamOperator streamOperator;
|
||||
|
||||
public PlanVertex(int vertexId, int parallelism, VertexType vertexType,
|
||||
public JobVertex(int vertexId, int parallelism, VertexType vertexType,
|
||||
StreamOperator streamOperator) {
|
||||
this.vertexId = vertexId;
|
||||
this.parallelism = parallelism;
|
||||
|
@ -39,11 +40,11 @@ public class PlanVertex implements Serializable {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "PlanVertex{" +
|
||||
"vertexId=" + vertexId +
|
||||
", parallelism=" + parallelism +
|
||||
", vertexType=" + vertexType +
|
||||
", streamOperator=" + streamOperator +
|
||||
'}';
|
||||
return MoreObjects.toStringHelper(this)
|
||||
.add("vertexId", vertexId)
|
||||
.add("parallelism", parallelism)
|
||||
.add("vertexType", vertexType)
|
||||
.add("streamOperator", streamOperator)
|
||||
.toString();
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.ray.streaming.plan;
|
||||
package org.ray.streaming.jobgraph;
|
||||
|
||||
/**
|
||||
* Different roles for a node.
|
|
@ -9,29 +9,33 @@ import org.ray.streaming.message.Record;
|
|||
|
||||
public abstract class StreamOperator<F extends Function> implements Operator {
|
||||
|
||||
protected String name;
|
||||
protected F function;
|
||||
protected List<Collector> collectorList;
|
||||
protected RuntimeContext runtimeContext;
|
||||
|
||||
|
||||
public StreamOperator(F function) {
|
||||
this.name = getClass().getSimpleName();
|
||||
this.function = function;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(List<Collector> collectorList, RuntimeContext runtimeContext) {
|
||||
this.collectorList = collectorList;
|
||||
this.runtimeContext = runtimeContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
|
||||
}
|
||||
|
||||
|
||||
protected void collect(Record record) {
|
||||
for (Collector collector : this.collectorList) {
|
||||
collector.collect(record);
|
||||
|
@ -44,4 +48,7 @@ public abstract class StreamOperator<F extends Function> implements Operator {
|
|||
}
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -2,8 +2,7 @@ package org.ray.streaming.schedule;
|
|||
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.ray.streaming.plan.Plan;
|
||||
import org.ray.streaming.jobgraph.JobGraph;
|
||||
|
||||
/**
|
||||
* Interface of the job scheduler.
|
||||
|
@ -13,7 +12,7 @@ public interface JobScheduler {
|
|||
/**
|
||||
* 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);
|
||||
}
|
||||
|
|
|
@ -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"));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -2,7 +2,6 @@ package org.ray.streaming.runtime.cluster;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.ray.api.Ray;
|
||||
import org.ray.api.RayActor;
|
||||
import org.ray.streaming.runtime.worker.JobWorker;
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package org.ray.streaming.runtime.core.graph;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
import org.ray.streaming.api.partition.Partition;
|
||||
|
||||
/**
|
||||
|
|
|
@ -6,7 +6,6 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.ray.api.RayActor;
|
||||
import org.ray.streaming.runtime.worker.JobWorker;
|
||||
|
||||
|
|
|
@ -3,13 +3,14 @@ package org.ray.streaming.runtime.core.graph;
|
|||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.ray.streaming.jobgraph.VertexType;
|
||||
import org.ray.streaming.operator.StreamOperator;
|
||||
import org.ray.streaming.plan.VertexType;
|
||||
|
||||
/**
|
||||
* A node in the physical execution graph.
|
||||
*/
|
||||
public class ExecutionNode implements Serializable {
|
||||
|
||||
private int nodeId;
|
||||
private int parallelism;
|
||||
private NodeType nodeType;
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package org.ray.streaming.runtime.core.graph;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
import org.ray.api.RayActor;
|
||||
import org.ray.streaming.runtime.worker.JobWorker;
|
||||
|
||||
|
|
|
@ -6,8 +6,8 @@ import java.util.Map;
|
|||
import org.ray.api.Ray;
|
||||
import org.ray.api.RayActor;
|
||||
import org.ray.api.RayObject;
|
||||
import org.ray.streaming.plan.Plan;
|
||||
import org.ray.streaming.plan.PlanVertex;
|
||||
import org.ray.streaming.jobgraph.JobGraph;
|
||||
import org.ray.streaming.jobgraph.JobVertex;
|
||||
import org.ray.streaming.runtime.cluster.ResourceManager;
|
||||
import org.ray.streaming.runtime.core.graph.ExecutionGraph;
|
||||
import org.ray.streaming.runtime.core.graph.ExecutionNode;
|
||||
|
@ -21,8 +21,8 @@ import org.ray.streaming.schedule.JobScheduler;
|
|||
* from ResourceManager.
|
||||
*/
|
||||
public class JobSchedulerImpl implements JobScheduler {
|
||||
private Plan plan;
|
||||
private Map<String, Object> jobConfig;
|
||||
private JobGraph jobGraph;
|
||||
private Map<String, String> jobConfig;
|
||||
private ResourceManager resourceManager;
|
||||
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.
|
||||
*/
|
||||
@Override
|
||||
public void schedule(Plan plan, Map<String, Object> jobConfig) {
|
||||
public void schedule(JobGraph jobGraph, Map<String, String> jobConfig) {
|
||||
this.jobConfig = jobConfig;
|
||||
this.plan = plan;
|
||||
this.jobGraph = jobGraph;
|
||||
System.setProperty("ray.raylet.config.num_workers_per_process_java", "1");
|
||||
Ray.init();
|
||||
|
||||
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<RayObject<Boolean>> waits = new ArrayList<>();
|
||||
|
@ -59,7 +59,7 @@ public class JobSchedulerImpl implements JobScheduler {
|
|||
}
|
||||
|
||||
private int getPlanWorker() {
|
||||
List<PlanVertex> planVertexList = plan.getPlanVertexList();
|
||||
return planVertexList.stream().map(PlanVertex::getParallelism).reduce(0, Integer::sum);
|
||||
List<JobVertex> jobVertexList = jobGraph.getJobVertexList();
|
||||
return jobVertexList.stream().map(JobVertex::getParallelism).reduce(0, Integer::sum);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -3,7 +3,7 @@ package org.ray.streaming.runtime.schedule;
|
|||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
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.worker.JobWorker;
|
||||
|
||||
|
@ -15,6 +15,6 @@ public interface TaskAssigner extends Serializable {
|
|||
/**
|
||||
* Assign logical plan to physical execution graph.
|
||||
*/
|
||||
ExecutionGraph assign(Plan plan, List<RayActor<JobWorker>> workers);
|
||||
ExecutionGraph assign(JobGraph jobGraph, List<RayActor<JobWorker>> workers);
|
||||
|
||||
}
|
||||
|
|
|
@ -6,9 +6,9 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.ray.api.RayActor;
|
||||
import org.ray.streaming.plan.Plan;
|
||||
import org.ray.streaming.plan.PlanEdge;
|
||||
import org.ray.streaming.plan.PlanVertex;
|
||||
import org.ray.streaming.jobgraph.JobEdge;
|
||||
import org.ray.streaming.jobgraph.JobGraph;
|
||||
import org.ray.streaming.jobgraph.JobVertex;
|
||||
import org.ray.streaming.runtime.core.graph.ExecutionEdge;
|
||||
import org.ray.streaming.runtime.core.graph.ExecutionGraph;
|
||||
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.
|
||||
*
|
||||
* @param plan The logical plan.
|
||||
* @param jobGraph The logical plan.
|
||||
* @param workers The worker actors.
|
||||
* @return The physical execution graph.
|
||||
*/
|
||||
@Override
|
||||
public ExecutionGraph assign(Plan plan, List<RayActor<JobWorker>> workers) {
|
||||
List<PlanVertex> planVertices = plan.getPlanVertexList();
|
||||
List<PlanEdge> planEdges = plan.getPlanEdgeList();
|
||||
public ExecutionGraph assign(JobGraph jobGraph, List<RayActor<JobWorker>> workers) {
|
||||
List<JobVertex> jobVertices = jobGraph.getJobVertexList();
|
||||
List<JobEdge> jobEdges = jobGraph.getJobEdgeList();
|
||||
|
||||
int taskId = 0;
|
||||
Map<Integer, ExecutionNode> idToExecutionNode = new HashMap<>();
|
||||
for (PlanVertex planVertex : planVertices) {
|
||||
ExecutionNode executionNode = new ExecutionNode(planVertex.getVertexId(),
|
||||
planVertex.getParallelism());
|
||||
executionNode.setNodeType(planVertex.getVertexType());
|
||||
for (JobVertex jobVertex : jobVertices) {
|
||||
ExecutionNode executionNode = new ExecutionNode(jobVertex.getVertexId(),
|
||||
jobVertex.getParallelism());
|
||||
executionNode.setNodeType(jobVertex.getVertexType());
|
||||
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)));
|
||||
taskId++;
|
||||
}
|
||||
executionNode.setExecutionTasks(vertexTasks);
|
||||
executionNode.setStreamOperator(planVertex.getStreamOperator());
|
||||
executionNode.setStreamOperator(jobVertex.getStreamOperator());
|
||||
idToExecutionNode.put(executionNode.getNodeId(), executionNode);
|
||||
}
|
||||
|
||||
for (PlanEdge planEdge : planEdges) {
|
||||
int srcNodeId = planEdge.getSrcVertexId();
|
||||
int targetNodeId = planEdge.getTargetVertexId();
|
||||
for (JobEdge jobEdge : jobEdges) {
|
||||
int srcNodeId = jobEdge.getSrcVertexId();
|
||||
int targetNodeId = jobEdge.getTargetVertexId();
|
||||
|
||||
ExecutionEdge executionEdge = new ExecutionEdge(srcNodeId, targetNodeId,
|
||||
planEdge.getPartition());
|
||||
jobEdge.getPartition());
|
||||
idToExecutionNode.get(srcNodeId).addExecutionEdge(executionEdge);
|
||||
idToExecutionNode.get(targetNodeId).addInputEdge(executionEdge);
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ public class JobWorker implements Serializable {
|
|||
}
|
||||
|
||||
private int taskId;
|
||||
private Map<String, Object> config;
|
||||
private Map<String, String> config;
|
||||
private WorkerContext workerContext;
|
||||
private ExecutionNode executionNode;
|
||||
private ExecutionTask executionTask;
|
||||
|
@ -88,7 +88,7 @@ public class JobWorker implements Serializable {
|
|||
return taskId;
|
||||
}
|
||||
|
||||
public Map<String, Object> getConfig() {
|
||||
public Map<String, String> getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
|
|
|
@ -3,7 +3,6 @@ package org.ray.streaming.runtime.worker.context;
|
|||
import static org.ray.streaming.util.Config.STREAMING_BATCH_MAX_COUNT;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.ray.streaming.api.context.RuntimeContext;
|
||||
import org.ray.streaming.runtime.core.graph.ExecutionTask;
|
||||
|
||||
|
@ -16,16 +15,16 @@ public class RayRuntimeContext implements RuntimeContext {
|
|||
private int parallelism;
|
||||
private Long batchId;
|
||||
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) {
|
||||
this.taskId = executionTask.getTaskId();
|
||||
this.config = config;
|
||||
this.taskIndex = executionTask.getTaskIndex();
|
||||
this.parallelism = parallelism;
|
||||
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 {
|
||||
this.maxBatch = Long.MAX_VALUE;
|
||||
}
|
||||
|
|
|
@ -11,9 +11,9 @@ public class WorkerContext implements Serializable {
|
|||
|
||||
private int taskId;
|
||||
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.executionGraph = executionGraph;
|
||||
this.config = jobConfig;
|
||||
|
@ -35,7 +35,7 @@ public class WorkerContext implements Serializable {
|
|||
this.executionGraph = executionGraph;
|
||||
}
|
||||
|
||||
public Map<String, Object> getConfig() {
|
||||
public Map<String, String> getConfig() {
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,8 +30,8 @@ public class WordCountTest extends BaseUnitTest implements Serializable {
|
|||
@Test
|
||||
public void testWordCount() {
|
||||
StreamingContext streamingContext = StreamingContext.buildContext();
|
||||
Map<String, Object> config = new HashMap<>();
|
||||
config.put(Config.STREAMING_BATCH_MAX_COUNT, 1);
|
||||
Map<String, String> config = new HashMap<>();
|
||||
config.put(Config.STREAMING_BATCH_MAX_COUNT, "1");
|
||||
config.put(Config.CHANNEL_TYPE, Config.MEMORY_CHANNEL);
|
||||
streamingContext.withConfig(config);
|
||||
List<String> text = new ArrayList<>();
|
||||
|
@ -50,7 +50,7 @@ public class WordCountTest extends BaseUnitTest implements Serializable {
|
|||
.sink((SinkFunction<WordAndCount>)
|
||||
result -> wordCount.put(result.word, result.count));
|
||||
|
||||
streamingContext.execute();
|
||||
streamingContext.execute("testWordCount");
|
||||
|
||||
// Sleep until the count for every word is computed.
|
||||
while (wordCount.size() < 3) {
|
||||
|
|
|
@ -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.NodeType;
|
||||
import org.ray.streaming.runtime.worker.JobWorker;
|
||||
import org.ray.streaming.plan.Plan;
|
||||
import org.ray.streaming.plan.PlanBuilder;
|
||||
import org.ray.streaming.jobgraph.JobGraph;
|
||||
import org.ray.streaming.jobgraph.JobGraphBuilder;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.testng.Assert;
|
||||
|
@ -32,15 +32,15 @@ public class TaskAssignerImplTest extends BaseUnitTest {
|
|||
|
||||
@Test
|
||||
public void testTaskAssignImpl() {
|
||||
Plan plan = buildDataSyncPlan();
|
||||
JobGraph jobGraph = buildDataSyncPlan();
|
||||
|
||||
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()));
|
||||
}
|
||||
|
||||
TaskAssigner taskAssigner = new TaskAssignerImpl();
|
||||
ExecutionGraph executionGraph = taskAssigner.assign(plan, workers);
|
||||
ExecutionGraph executionGraph = taskAssigner.assign(jobGraph, workers);
|
||||
|
||||
List<ExecutionNode> executionNodeList = executionGraph.getExecutionNodeList();
|
||||
|
||||
|
@ -63,14 +63,14 @@ public class TaskAssignerImplTest extends BaseUnitTest {
|
|||
Assert.assertEquals(sinkNode.getOutputEdges().size(), 0);
|
||||
}
|
||||
|
||||
public Plan buildDataSyncPlan() {
|
||||
public JobGraph 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));
|
||||
JobGraphBuilder jobGraphBuilder = new JobGraphBuilder(Lists.newArrayList(streamSink));
|
||||
|
||||
Plan plan = planBuilder.buildPlan();
|
||||
return plan;
|
||||
JobGraph jobGraph = jobGraphBuilder.build();
|
||||
return jobGraph;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -151,8 +151,8 @@ public class StreamingQueueTest extends BaseUnitTest implements Serializable {
|
|||
|
||||
Map<String, Integer> wordCount = new ConcurrentHashMap<>();
|
||||
StreamingContext streamingContext = StreamingContext.buildContext();
|
||||
Map<String, Object> config = new HashMap<>();
|
||||
config.put(Config.STREAMING_BATCH_MAX_COUNT, 1);
|
||||
Map<String, String> config = new HashMap<>();
|
||||
config.put(Config.STREAMING_BATCH_MAX_COUNT, "1");
|
||||
config.put(Config.CHANNEL_TYPE, Config.NATIVE_CHANNEL);
|
||||
config.put(Config.CHANNEL_SIZE, "100000");
|
||||
streamingContext.withConfig(config);
|
||||
|
@ -177,7 +177,7 @@ public class StreamingQueueTest extends BaseUnitTest implements Serializable {
|
|||
serializeResultToFile(resultFile, wordCount);
|
||||
});
|
||||
|
||||
streamingContext.execute();
|
||||
streamingContext.execute("testWordCount");
|
||||
|
||||
Map<String, Integer> checkWordCount =
|
||||
(Map<String, Integer>) deserializeResultFromFile(resultFile);
|
||||
|
|
Loading…
Add table
Reference in a new issue