[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;
import java.util.List;
import org.ray.streaming.api.collector.Collector;
import org.ray.streaming.message.Record;
/**

View file

@ -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;
}
}

View file

@ -1,7 +1,6 @@
package org.ray.streaming.api.partition.impl;
import java.util.stream.IntStream;
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 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;

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.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() {

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 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();
}
}

View file

@ -1,4 +1,4 @@
package org.ray.streaming.plan;
package org.ray.streaming.jobgraph;
/**
* 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 {
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;
}
}

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 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);
}

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.List;
import org.ray.api.Ray;
import org.ray.api.RayActor;
import org.ray.streaming.runtime.worker.JobWorker;

View file

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

View file

@ -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;

View file

@ -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;

View file

@ -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;

View file

@ -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);
}
}

View file

@ -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);
}

View file

@ -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);
}

View file

@ -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;
}

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 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;
}

View file

@ -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;
}
}

View file

@ -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) {

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.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;
}
}

View file

@ -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);