From 9fffd554f1e5fd6465989bb3568dfb6f2d854eeb Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 6 Jul 2017 10:22:27 +0800 Subject: [PATCH 01/34] Initial commit for MapReduceRunner. --- runners/map-reduce/pom.xml | 165 ++++++++++++++++++ .../mapreduce/MapReducePipelineOptions.java | 9 + .../runners/mapreduce/MapReduceRegistrar.java | 32 ++++ .../runners/mapreduce/MapReduceRunner.java | 15 ++ .../MapReducePipelineTranslator.java | 11 ++ 5 files changed, 232 insertions(+) create mode 100644 runners/map-reduce/pom.xml create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml new file mode 100644 index 000000000000..2e8a8c9a0120 --- /dev/null +++ b/runners/map-reduce/pom.xml @@ -0,0 +1,165 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-runners-parent + 2.1.0-SNAPSHOT + ../pom.xml + + + beam-runners-map-reduce + + Apache Beam :: Runners :: MapReduce + + jar + + + 2.8.1 + + + + + + local-validates-runner-tests + false + + + + org.apache.maven.plugins + maven-surefire-plugin + + + validates-runner-tests + integration-test + + test + + + + org.apache.beam.sdk.testing.ValidatesRunner + + none + true + + org.apache.beam:beam-sdks-java-core + + + + [ + "--runner=org.apache.beam.runners.mapreduce.MapReduceRunner" + ] + + + + + + + + + + + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${mapreduce.version} + + + + + org.apache.beam + beam-sdks-java-core + + + + org.apache.beam + beam-runners-core-java + + + + org.apache.beam + beam-runners-core-construction-java + + + + + com.google.auto.service + auto-service + true + + + com.google.auto.value + auto-value + + + + + org.apache.beam + beam-sdks-java-core + tests + test + + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + test + + + + + junit + junit + test + + + org.hamcrest + hamcrest-all + test + + + org.mockito + mockito-all + test + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + analyze-only + + + false + + + + + + + diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java new file mode 100644 index 000000000000..da2993188651 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -0,0 +1,9 @@ +package org.apache.beam.runners.mapreduce; + +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * {@link PipelineOptions} for {@link MapReduceRunner}. + */ +public interface MapReducePipelineOptions extends PipelineOptions { +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java new file mode 100644 index 000000000000..eb960b81b120 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java @@ -0,0 +1,32 @@ +package org.apache.beam.runners.mapreduce; + +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; +import org.apache.beam.sdk.PipelineRunner; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsRegistrar; +import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; + +/** + * Registrars for {@link MapReduceRunner}. + */ +public class MapReduceRegistrar { + private MapReduceRegistrar() { + } + + @AutoService(PipelineRunnerRegistrar.class) + public static class Runner implements PipelineRunnerRegistrar { + @Override + public Iterable>> getPipelineRunners() { + return ImmutableList.>> of(MapReduceRunner.class); + } + } + + @AutoService(PipelineOptionsRegistrar.class) + public static class Options implements PipelineOptionsRegistrar { + @Override + public Iterable> getPipelineOptions() { + return ImmutableList.> of(MapReducePipelineOptions.class); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java new file mode 100644 index 000000000000..bb9555eaf03c --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -0,0 +1,15 @@ +package org.apache.beam.runners.mapreduce; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.PipelineRunner; + +/** + * {@link PipelineRunner} for crunch. + */ +public class MapReduceRunner extends PipelineRunner { + @Override + public PipelineResult run(Pipeline pipeline) { + return null; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java new file mode 100644 index 000000000000..b4a2e7c4746b --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java @@ -0,0 +1,11 @@ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.runners.mapreduce.MapReduceRunner; +import org.apache.beam.sdk.Pipeline; + +/** + * Pipeline translator for {@link MapReduceRunner}. + */ +public class MapReducePipelineTranslator extends Pipeline.PipelineVisitor.Defaults { + +} From 092380cf87ada9d4a2b5faa7a42dff7005c44f17 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 11 Jul 2017 10:45:11 +0800 Subject: [PATCH 02/34] MapReduceRunner: add Graph and its visitors. --- .../runners/mapreduce/translation/Graph.java | 190 ++++++++++++++++++ .../mapreduce/translation/GraphConverter.java | 40 ++++ .../mapreduce/translation/GraphPlanner.java | 99 +++++++++ .../mapreduce/translation/GraphVisitor.java | 11 + .../MapReducePipelineTranslator.java | 11 - 5 files changed, 340 insertions(+), 11 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java delete mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java new file mode 100644 index 000000000000..a9831bd82df9 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -0,0 +1,190 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.util.LinkedList; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; + +/** + * Created by peihe on 06/07/2017. + */ +public class Graph { + + private final Map vertices; + private final Map edges; + private final Set leafVertices; + + public Graph() { + this.vertices = Maps.newHashMap(); + this.edges = Maps.newHashMap(); + this.leafVertices = Sets.newHashSet(); + } + + public Vertex addVertex(PTransform transform) { + checkState(!vertices.containsKey(transform)); + Vertex v = new Vertex(transform); + vertices.put(transform, v); + leafVertices.add(v); + return v; + } + + public Edge addEdge(Vertex head, Vertex tail) { + HeadTail headTail = HeadTail.of(head, tail); + checkState(!edges.containsKey(headTail)); + Edge e = new Edge(headTail); + edges.put(headTail, e); + head.addOutgoing(e); + tail.addIncoming(e); + leafVertices.remove(head); + return e; + } + + public Vertex getVertex(PTransform transform) { + return vertices.get(transform); + } + + public Edge getEdge(Vertex head, Vertex tail) { + return edges.get(HeadTail.of(head, tail)); + } + + public Set getLeafVertices() { + return leafVertices; + } + + public void accept(GraphVisitor visitor) { + for (Vertex v : leafVertices) { + v.accept(visitor); + } + } + + //TODO: add equals, hashCode, toString for following classses. + + public static class Vertex { + private final PTransform transform; + private final Set incoming; + private final Set outgoing; + + public Vertex(PTransform transform) { + this.transform = checkNotNull(transform, "transform"); + this.incoming = Sets.newHashSet(); + this.outgoing = Sets.newHashSet(); + } + + public PTransform getTransform() { + return transform; + } + + public Set getIncoming() { + return incoming; + } + + public Set getOutgoing() { + return outgoing; + } + + public boolean isSource() { + return transform instanceof Read.Bounded || transform instanceof Read.Unbounded; + } + + public boolean isGroupByKey() { + return transform instanceof GroupByKey; + } + + public void addIncoming(Edge edge) { + incoming.add(edge); + } + + public void addOutgoing(Edge edge) { + outgoing.add(edge); + } + + public void accept(GraphVisitor visitor) { + if (transform instanceof ParDo.SingleOutput || transform instanceof ParDo.MultiOutput) { + visitor.visitParDo(this); + } else if (transform instanceof GroupByKey) { + visitor.visitGroupByKey(this); + } else if (transform instanceof Read.Bounded) { + visitor.visitRead(this); + } else if (transform instanceof Flatten.PCollections + || transform instanceof Flatten.Iterables) { + visitor.visitFlatten(this); + } else { + throw new RuntimeException("Unexpected transform type: " + transform.getClass()); + } + } + } + + public static class Edge { + private final HeadTail headTail; + private final Set paths; + + public static Edge of(Vertex head, Vertex tail) { + return of(HeadTail.of(head, tail)); + } + + public static Edge of(HeadTail headTail) { + return new Edge(headTail); + } + + private Edge(HeadTail headTail) { + this.headTail = checkNotNull(headTail, "headTail"); + this.paths = Sets.newHashSet(); + } + + public Vertex getHead() { + return headTail.getHead(); + } + + public Vertex getTail() { + return headTail.getTail(); + } + + public Set getPaths() { + return paths; + } + + public void addPath(NodePath path) { + paths.add(checkNotNull(path, "path")); + } + } + + public static class NodePath { + private final LinkedList> path; + + public NodePath() { + this.path = new LinkedList<>(); + } + + public NodePath(NodePath nodePath) { + this.path = new LinkedList<>(nodePath.path); + } + + public void addFirst(PTransform transform) { + path.addFirst(transform); + } + + public void addLast(PTransform transform) { + path.addLast(transform); + } + } + + @AutoValue + public abstract static class HeadTail { + abstract Vertex getHead(); + abstract Vertex getTail(); + + public static HeadTail of(Vertex head, Vertex tail) { + return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graph_HeadTail(head, tail); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java new file mode 100644 index 000000000000..306e58e71fa5 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -0,0 +1,40 @@ +package org.apache.beam.runners.mapreduce.translation; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.util.Map; +import java.util.Set; +import org.apache.beam.runners.mapreduce.MapReduceRunner; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.values.PValue; + +/** + * Pipeline translator for {@link MapReduceRunner}. + */ +public class GraphConverter extends Pipeline.PipelineVisitor.Defaults { + + private final Map outputToProducer; + private final Graph graph; + + public GraphConverter() { + this.outputToProducer = Maps.newHashMap(); + this.graph = new Graph(); + } + + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + Graph.Vertex v = new Graph.Vertex(node.getTransform()); + + for (PValue input : node.getInputs().values()) { + if (outputToProducer.containsKey(input)) { + Graph.Vertex producer = outputToProducer.get(input); + graph.addEdge(producer, v); + } + } + + for (PValue output : node.getOutputs().values()) { + outputToProducer.put(output, v); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java new file mode 100644 index 000000000000..d4fa2d9bf355 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -0,0 +1,99 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * Created by peihe on 06/07/2017. + */ +public class GraphPlanner { + + public Graph plan(Graph initGraph) { + FusionVisitor fusionVisitor = new FusionVisitor(); + initGraph.accept(fusionVisitor); + return fusionVisitor.getFusedGraph(); + } + + private class FusionVisitor implements GraphVisitor { + + private Graph fusedGraph; + private Graph.Vertex workingVertex; + private Graph.NodePath workingPath; + + FusionVisitor() { + fusedGraph = new Graph(); + workingVertex = null; + workingPath = null; + } + + @Override + public void visitRead(Graph.Vertex read) { + if (workingVertex == null) { + // drop if read is leaf vertex. + return; + } + Graph.Vertex v = fusedGraph.addVertex(read.getTransform()); + workingPath.addFirst(read.getTransform()); + Graph.Edge edge = fusedGraph.addEdge(v, workingVertex); + edge.addPath(workingPath); + } + + @Override + public void visitParDo(Graph.Vertex parDo) { + checkArgument( + parDo.getTransform().getAdditionalInputs().isEmpty(), + "Side inputs are not supported."); + if (workingVertex == null) { + // Leaf vertex + workingVertex = fusedGraph.addVertex(parDo.getTransform()); + workingPath = new Graph.NodePath(); + } else { + workingPath.addFirst(parDo.getTransform()); + } + checkArgument( + parDo.getIncoming().size() == 1, + "Side inputs are not supported."); + processParent(parDo.getIncoming().iterator().next().getHead()); + } + + @Override + public void visitFlatten(Graph.Vertex flatten) { + if (workingVertex == null) { + return; + } + Graph.NodePath basePath = workingPath; + Graph.Vertex baseVertex = workingVertex; + for (Graph.Edge e : flatten.getIncoming()) { + workingPath = new Graph.NodePath(basePath); + workingVertex = baseVertex; + processParent(e.getHead()); + } + } + + @Override + public void visitGroupByKey(Graph.Vertex groupByKey) { + if (workingVertex == null) { + return; + } + Graph.Vertex v = fusedGraph.addVertex(groupByKey.getTransform()); + workingPath.addFirst(groupByKey.getTransform()); + Graph.Edge edge = fusedGraph.addEdge(v, workingVertex); + edge.addPath(workingPath); + } + + public Graph getFusedGraph() { + return fusedGraph; + } + + private void processParent(Graph.Vertex parent) { + Graph.Vertex v = fusedGraph.getVertex(parent.getTransform()); + if (v == null) { + parent.accept(this); + } else { + // TODO: parent is consumed more than once. + // It is duplicated in multiple outgoing path. Figure out the impact. + workingPath.addFirst(parent.getTransform()); + fusedGraph.getEdge(v, workingVertex).addPath(workingPath); + } + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java new file mode 100644 index 000000000000..fe4a76f0ddeb --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java @@ -0,0 +1,11 @@ +package org.apache.beam.runners.mapreduce.translation; + +/** + * Created by peihe on 06/07/2017. + */ +public interface GraphVisitor { + void visitRead(Graph.Vertex read); + void visitParDo(Graph.Vertex parDo); + void visitFlatten(Graph.Vertex flatten); + void visitGroupByKey(Graph.Vertex groupByKey); +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java deleted file mode 100644 index b4a2e7c4746b..000000000000 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReducePipelineTranslator.java +++ /dev/null @@ -1,11 +0,0 @@ -package org.apache.beam.runners.mapreduce.translation; - -import org.apache.beam.runners.mapreduce.MapReduceRunner; -import org.apache.beam.sdk.Pipeline; - -/** - * Pipeline translator for {@link MapReduceRunner}. - */ -public class MapReducePipelineTranslator extends Pipeline.PipelineVisitor.Defaults { - -} From a8b366de9e4e0c79a7800184afc79b377477b8ed Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 13 Jul 2017 14:09:10 +0800 Subject: [PATCH 03/34] MapReduceRunner: add unit tests for GraphConverter and GraphPlanner. --- .../runners/mapreduce/MapReduceRunner.java | 12 ++ .../runners/mapreduce/translation/Graph.java | 89 ++++++++++++++- .../mapreduce/translation/GraphConverter.java | 6 +- .../mapreduce/translation/GraphPlanner.java | 1 + .../beam/runners/mapreduce/WordCountTest.java | 108 ++++++++++++++++++ .../translation/GraphConverterTest.java | 39 +++++++ .../translation/GraphPlannerTest.java | 42 +++++++ 7 files changed, 294 insertions(+), 3 deletions(-) create mode 100644 runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java create mode 100644 runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java create mode 100644 runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index bb9555eaf03c..247a8e5fdaf0 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -3,11 +3,23 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineRunner; +import org.apache.beam.sdk.options.PipelineOptions; /** * {@link PipelineRunner} for crunch. */ public class MapReduceRunner extends PipelineRunner { + + /** + * Construct a runner from the provided options. + * + * @param options Properties which configure the runner. + * @return The newly created runner. + */ + public static MapReduceRunner fromOptions(PipelineOptions options) { + return new MapReduceRunner(); + } + @Override public PipelineResult run(Pipeline pipeline) { return null; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index a9831bd82df9..1ca5a0531c68 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -4,16 +4,21 @@ import static com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.util.LinkedList; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.commons.lang.builder.ReflectionToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; /** * Created by peihe on 06/07/2017. @@ -57,8 +62,16 @@ public Edge getEdge(Vertex head, Vertex tail) { return edges.get(HeadTail.of(head, tail)); } - public Set getLeafVertices() { - return leafVertices; + public Iterable getAllVertices() { + return vertices.values(); + } + + public Iterable getAllEdges() { + return edges.values(); + } + + public Iterable getLeafVertices() { + return ImmutableList.copyOf(leafVertices); } public void accept(GraphVisitor visitor) { @@ -122,6 +135,29 @@ public void accept(GraphVisitor visitor) { throw new RuntimeException("Unexpected transform type: " + transform.getClass()); } } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof Vertex) { + Vertex other = (Vertex) obj; + return transform.equals(other.transform); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass(), transform); + } + + @Override + public String toString() { + return new ReflectionToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .setExcludeFieldNames(new String[] { "outgoing", "incoming" }).toString(); + } } public static class Edge { @@ -156,6 +192,28 @@ public Set getPaths() { public void addPath(NodePath path) { paths.add(checkNotNull(path, "path")); } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof Edge) { + Edge other = (Edge) obj; + return headTail.equals(other.headTail) && paths.equals(paths); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(headTail, paths); + } + + @Override + public String toString() { + return ReflectionToStringBuilder.toString(this, ToStringStyle.SHORT_PREFIX_STYLE); + } } public static class NodePath { @@ -176,6 +234,33 @@ public void addFirst(PTransform transform) { public void addLast(PTransform transform) { path.addLast(transform); } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof NodePath) { + NodePath other = (NodePath) obj; + return path.equals(other.path); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass(), path.hashCode()); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + for (PTransform collect : path) { + sb.append(collect.getName() + "|"); + } + // sb.deleteCharAt(sb.length() - 1); + return sb.toString(); + } } @AutoValue diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java index 306e58e71fa5..359a6e28bfa6 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -24,7 +24,7 @@ public GraphConverter() { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { - Graph.Vertex v = new Graph.Vertex(node.getTransform()); + Graph.Vertex v = graph.addVertex(node.getTransform()); for (PValue input : node.getInputs().values()) { if (outputToProducer.containsKey(input)) { @@ -37,4 +37,8 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { outputToProducer.put(output, v); } } + + public Graph getGraph() { + return graph; + } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index d4fa2d9bf355..793efd75c252 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -78,6 +78,7 @@ public void visitGroupByKey(Graph.Vertex groupByKey) { workingPath.addFirst(groupByKey.getTransform()); Graph.Edge edge = fusedGraph.addEdge(v, workingVertex); edge.addPath(workingPath); + processParent(groupByKey.getIncoming().iterator().next().getHead()); } public Graph getFusedGraph() { diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java new file mode 100644 index 000000000000..51c26f244673 --- /dev/null +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java @@ -0,0 +1,108 @@ +package org.apache.beam.runners.mapreduce; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Test that runs WordCount. + */ +@RunWith(JUnit4.class) +public class WordCountTest { + + public static final String TOKENIZER_PATTERN = "[^\\p{L}]+"; + + /** + * Concept #2: You can make your pipeline assembly code less verbose by defining your DoFns + * statically out-of-line. This DoFn tokenizes lines of text into individual words; we pass it + * to a ParDo in the pipeline. + */ + static class ExtractWordsFn extends DoFn { + private final Counter emptyLines = Metrics.counter(ExtractWordsFn.class, "emptyLines"); + + @ProcessElement + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.inc(); + } + + // Split the line into words. + String[] words = c.element().split(TOKENIZER_PATTERN); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + /** A SimpleFunction that converts a Word and Count into a printable string. */ + public static class FormatAsTextFn extends SimpleFunction, String> { + @Override + public String apply(KV input) { + return input.getKey() + ": " + input.getValue(); + } + } + + /** + * A PTransform that converts a PCollection containing lines of text into a PCollection of + * formatted word counts. + * + *

Concept #3: This is a custom composite transform that bundles two transforms (ParDo and + * Count) as a reusable PTransform subclass. Using composite transforms allows for easy reuse, + * modular testing, and an improved monitoring experience. + */ + public static class CountWords extends PTransform, + PCollection>> { + @Override + public PCollection> expand(PCollection lines) { + + // Convert lines of text into individual words. + PCollection words = lines.apply( + ParDo.of(new ExtractWordsFn())); + + // Count the number of times each word occurs. + PCollection> wordCounts = + words.apply(Count.perElement()); + + return wordCounts; + } + } + + @Test + public void testWordCount() { + String input = "gs://apache-beam-samples/shakespeare/kinglear.txt"; + String output = "./output"; + PipelineOptions options = PipelineOptionsFactory.create(); + options.setRunner(MapReduceRunner.class); + Pipeline p = Pipeline.create(options); + + // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the + // static FormatAsTextFn() to the ParDo transform. + p.apply("ReadLines", TextIO.read().from(input)) + .apply(new CountWords()) + .apply(MapElements.via(new FormatAsTextFn())) + .apply("WriteCounts", TextIO.write().to(output)); + + p.run().waitUntilFinish(); + } +} diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java new file mode 100644 index 000000000000..4f0c28340650 --- /dev/null +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java @@ -0,0 +1,39 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Iterables; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link GraphConverter}. + */ +@RunWith(JUnit4.class) +public class GraphConverterTest { + + @Test + public void testCombine() throws Exception { + Pipeline p = Pipeline.create(); + PCollection> input = p + .apply(Create.empty(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) + .apply(Sum.integersPerKey()); + GraphConverter graphConverter = new GraphConverter(); + p.traverseTopologically(graphConverter); + + Graph graph = graphConverter.getGraph(); + + assertEquals(3, Iterables.size(graph.getAllVertices())); + assertEquals(2, Iterables.size(graph.getAllEdges())); + assertEquals(1, Iterables.size(graph.getLeafVertices())); + } +} diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java new file mode 100644 index 000000000000..c98f81789a96 --- /dev/null +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java @@ -0,0 +1,42 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Iterables; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link GraphPlanner}. + */ +@RunWith(JUnit4.class) +public class GraphPlannerTest { + + @Test + public void testCombine() throws Exception { + Pipeline p = Pipeline.create(); + PCollection> input = p + .apply(Create.empty(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) + .apply(Sum.integersPerKey()); + GraphConverter graphConverter = new GraphConverter(); + p.traverseTopologically(graphConverter); + + Graph graph = graphConverter.getGraph(); + + GraphPlanner planner = new GraphPlanner(); + Graph fusedGraph = planner.plan(graph); + + assertEquals(3, Iterables.size(fusedGraph.getAllVertices())); + assertEquals(2, Iterables.size(fusedGraph.getAllEdges())); + assertEquals(1, Iterables.size(fusedGraph.getLeafVertices())); + } +} From a884a2f0b33b6621ef3a2fff6f5467109707df54 Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 21 Jul 2017 13:46:36 +0800 Subject: [PATCH 04/34] mr-runner: support BoundedSource with BeamInputFormat. --- runners/map-reduce/pom.xml | 14 +- .../runners/mapreduce/MapReduceWordCount.java | 218 ++++++++++++++++++ .../translation/BeamInputFormat.java | 154 +++++++++++++ .../mapreduce/translation/BeamMapper.java | 30 +++ 4 files changed, 415 insertions(+), 1 deletion(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml index 2e8a8c9a0120..d18eee84d5be 100644 --- a/runners/map-reduce/pom.xml +++ b/runners/map-reduce/pom.xml @@ -84,7 +84,19 @@ hadoop-mapreduce-client-core ${mapreduce.version} - + + + org.apache.hadoop + hadoop-mapreduce-client-common + ${mapreduce.version} + + + + org.apache.hadoop + hadoop-common + ${mapreduce.version} + + org.apache.beam diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java new file mode 100644 index 000000000000..4ba3a29bff8d --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java @@ -0,0 +1,218 @@ +package org.apache.beam.runners.mapreduce; + +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.StringTokenizer; + +import javax.annotation.Nullable; +import org.apache.beam.runners.mapreduce.translation.BeamInputFormat; +import org.apache.beam.runners.mapreduce.translation.BeamMapper; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.OffsetBasedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.KV; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; +import org.apache.log4j.BasicConfigurator; + +public class MapReduceWordCount { + + public static class CreateSource extends OffsetBasedSource { + private final List allElementsBytes; + private final long totalSize; + private final Coder coder; + + public static CreateSource fromIterable(Iterable elements, Coder elemCoder) + throws CoderException, IOException { + ImmutableList.Builder allElementsBytes = ImmutableList.builder(); + long totalSize = 0L; + for (T element : elements) { + byte[] bytes = CoderUtils.encodeToByteArray(elemCoder, element); + allElementsBytes.add(bytes); + totalSize += bytes.length; + } + return new CreateSource<>(allElementsBytes.build(), totalSize, elemCoder); + } + + /** + * Create a new source with the specified bytes. The new source owns the input element bytes, + * which must not be modified after this constructor is called. + */ + private CreateSource(List elementBytes, long totalSize, Coder coder) { + super(0, elementBytes.size(), 1); + this.allElementsBytes = ImmutableList.copyOf(elementBytes); + this.totalSize = totalSize; + this.coder = coder; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return totalSize; + } + + @Override + public BoundedSource.BoundedReader createReader(PipelineOptions options) + throws IOException { + return new BytesReader<>(this); + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return coder; + } + + @Override + public long getMaxEndOffset(PipelineOptions options) throws Exception { + return allElementsBytes.size(); + } + + @Override + public OffsetBasedSource createSourceForSubrange(long start, long end) { + List primaryElems = allElementsBytes.subList((int) start, (int) end); + long primarySizeEstimate = + (long) (totalSize * primaryElems.size() / (double) allElementsBytes.size()); + return new CreateSource<>(primaryElems, primarySizeEstimate, coder); + } + + @Override + public long getBytesPerOffset() { + if (allElementsBytes.size() == 0) { + return 1L; + } + return Math.max(1, totalSize / allElementsBytes.size()); + } + + private static class BytesReader extends OffsetBasedReader { + private int index; + /** + * Use an optional to distinguish between null next element (as Optional.absent()) and no next + * element (next is null). + */ + @Nullable + private Optional next; + + public BytesReader(CreateSource source) { + super(source); + index = -1; + } + + @Override + @Nullable + public T getCurrent() throws NoSuchElementException { + if (next == null) { + throw new NoSuchElementException(); + } + return next.orNull(); + } + + @Override + public void close() throws IOException {} + + @Override + protected long getCurrentOffset() { + return index; + } + + @Override + protected boolean startImpl() throws IOException { + return advanceImpl(); + } + + @Override + public synchronized CreateSource getCurrentSource() { + return (CreateSource) super.getCurrentSource(); + } + + @Override + protected boolean advanceImpl() throws IOException { + CreateSource source = getCurrentSource(); + if (index + 1 >= source.allElementsBytes.size()) { + next = null; + return false; + } + index++; + next = + Optional.fromNullable( + CoderUtils.decodeFromByteArray(source.coder, source.allElementsBytes.get(index))); + return true; + } + } + } + + public static class TokenizerMapper + extends Mapper{ + + private final static IntWritable one = new IntWritable(1); + private Text word = new Text(); + + public void map(Object key, Text value, Context context + ) throws IOException, InterruptedException { + StringTokenizer itr = new StringTokenizer(value.toString()); + while (itr.hasMoreTokens()) { + word.set(itr.nextToken()); + context.write(word, one); + } + } + } + + public static class IntSumReducer + extends Reducer { + private IntWritable result = new IntWritable(); + + public void reduce(Text key, Iterable values, Context context) + throws IOException, InterruptedException { + int sum = 0; + for (IntWritable val : values) { + sum += val.get(); + } + result.set(sum); + context.write(key, result); + } + } + + public static void main(String[] args) throws Exception { + BasicConfigurator.configure(); + + Configuration conf = new Configuration(); + + BoundedSource> source = CreateSource.fromIterable( + ImmutableList.of(KV.of("k1", 10), KV.of("k2", 2)), + KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); + + conf.set( + "source", + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(source))); + + Job job = Job.getInstance(conf, "word count"); + job.setJarByClass(MapReduceWordCount.class); + job.setInputFormatClass(BeamInputFormat.class); + job.setMapperClass(BeamMapper.class); + //job.setMapperClass(TokenizerMapper.class); + //job.setCombinerClass(IntSumReducer.class); + //job.setReducerClass(IntSumReducer.class); + //job.setOutputKeyClass(Text.class); + //job.setOutputValueClass(IntWritable.class); + //FileInputFormat.addInputPath(job, new Path(args[0])); + job.setOutputFormatClass(NullOutputFormat.class); + System.exit(job.waitForCompletion(true) ? 0 : 1); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java new file mode 100644 index 000000000000..8c4155afb5fe --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -0,0 +1,154 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.KV; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; + +/** + * Adaptor from Beam {@link BoundedSource} to MapReduce {@link InputFormat}. + */ +public class BeamInputFormat extends InputFormat { + + private static final long DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES = 5 * 1000 * 1000; + + private BoundedSource> source; + private PipelineOptions options; + + public BeamInputFormat() { + } + + public BeamInputFormat(BoundedSource> source, PipelineOptions options) { + this.source = checkNotNull(source, "source"); + this.options = checkNotNull(options, "options"); + } + + @Override + public List getSplits(JobContext context) throws IOException, InterruptedException { + source = (BoundedSource>) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(context.getConfiguration().get("source")), + ""); + try { + return FluentIterable.from(source.split(DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES, options)) + .transform(new Function>, InputSplit>() { + @Override + public InputSplit apply(BoundedSource> source) { + try { + return new BeamInputSplit(source.getEstimatedSizeBytes(options)); + } catch (Exception e) { + throw new RuntimeException(e); + } + }}) + .toList(); + } catch (Exception e) { + throw new IOException(e); + } + } + + @Override + public RecordReader createRecordReader( + InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { + source = (BoundedSource>) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(context.getConfiguration().get("source")), + ""); + return new BeamRecordReader<>(source.createReader(options)); + } + + public static class BeamInputSplit extends InputSplit implements Writable { + private long estimatedSizeBytes; + + public BeamInputSplit() { + } + + BeamInputSplit(long estimatedSizeBytes) { + this.estimatedSizeBytes = estimatedSizeBytes; + } + + @Override + public long getLength() throws IOException, InterruptedException { + return estimatedSizeBytes; + } + + @Override + public String[] getLocations() throws IOException, InterruptedException { + return new String[0]; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(estimatedSizeBytes); + } + + @Override + public void readFields(DataInput in) throws IOException { + estimatedSizeBytes = in.readLong(); + } + } + + private class BeamRecordReader extends RecordReader { + + private final BoundedSource.BoundedReader> reader; + private boolean started; + + public BeamRecordReader(BoundedSource.BoundedReader> reader) { + this.reader = checkNotNull(reader, "reader"); + this.started = false; + } + + @Override + public void initialize( + InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { + } + + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + if (!started) { + return reader.start(); + } else { + return reader.advance(); + } + } + + @Override + public Object getCurrentKey() throws IOException, InterruptedException { + return reader.getCurrent().getKey(); + } + + @Override + public Object getCurrentValue() throws IOException, InterruptedException { + return reader.getCurrent().getValue(); + } + + @Override + public float getProgress() throws IOException, InterruptedException { + Double progress = reader.getFractionConsumed(); + if (progress != null) { + return progress.floatValue(); + } else { + return 0; + } + } + + @Override + public void close() throws IOException { + reader.close(); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java new file mode 100644 index 000000000000..88fc8d6fe64b --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -0,0 +1,30 @@ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.values.KV; +import org.apache.hadoop.mapreduce.Mapper; + +/** + * Created by peihe on 21/07/2017. + */ +public class BeamMapper + extends Mapper { + + private DoFnInvoker, KV> doFnInvoker; + + @Override + protected void setup(Mapper.Context context) { + } + + @Override + protected void map( + KeyInT key, + ValueInT value, + Mapper.Context context) { + System.out.print(String.format("key: %s, value: %s", key, value)); + } + + @Override + protected void cleanup(Mapper.Context context) { + } +} From 0cbdc5b75ed5581ffef8d129b4e61e339d459697 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 24 Jul 2017 20:15:37 +0800 Subject: [PATCH 05/34] mr-runner: add JobPrototype and translate it to a MR job. --- .../mapreduce/MapReducePipelineOptions.java | 5 + .../runners/mapreduce/MapReduceRunner.java | 41 +++++++- .../runners/mapreduce/MapReduceWordCount.java | 2 +- .../translation/BeamInputFormat.java | 44 +++++---- .../mapreduce/translation/BeamMapper.java | 75 ++++++++++++--- .../runners/mapreduce/translation/Graph.java | 5 + .../mapreduce/translation/JobPrototype.java | 95 +++++++++++++++++++ .../beam/runners/mapreduce/WordCountTest.java | 42 +++----- 8 files changed, 244 insertions(+), 65 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java index da2993188651..ce8f937e63fb 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -1,9 +1,14 @@ package org.apache.beam.runners.mapreduce; +import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; /** * {@link PipelineOptions} for {@link MapReduceRunner}. */ public interface MapReducePipelineOptions extends PipelineOptions { + + @Description("The jar class of the user Beam program.") + Class getJarClass(); + void setJarClass(Class jarClass); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index 247a8e5fdaf0..0e3142cc360b 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -1,9 +1,21 @@ package org.apache.beam.runners.mapreduce; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Throwables; +import java.io.IOException; +import org.apache.beam.runners.mapreduce.translation.Graph; +import org.apache.beam.runners.mapreduce.translation.GraphConverter; +import org.apache.beam.runners.mapreduce.translation.GraphPlanner; +import org.apache.beam.runners.mapreduce.translation.JobPrototype; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineRunner; +import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.Job; /** * {@link PipelineRunner} for crunch. @@ -17,11 +29,38 @@ public class MapReduceRunner extends PipelineRunner { * @return The newly created runner. */ public static MapReduceRunner fromOptions(PipelineOptions options) { - return new MapReduceRunner(); + return new MapReduceRunner(options.as(MapReducePipelineOptions.class)); + } + + private final MapReducePipelineOptions options; + + MapReduceRunner(MapReducePipelineOptions options) { + this.options = checkNotNull(options, "options"); } @Override public PipelineResult run(Pipeline pipeline) { + GraphConverter graphConverter = new GraphConverter(); + pipeline.traverseTopologically(graphConverter); + + Graph graph = graphConverter.getGraph(); + + GraphPlanner planner = new GraphPlanner(); + Graph fusedGraph = planner.plan(graph); + for (Graph.Vertex vertex : fusedGraph.getAllVertices()) { + if (vertex.getTransform() instanceof GroupByKey + || vertex.getTransform() instanceof Read.Bounded) { + continue; + } else { + JobPrototype jobPrototype = JobPrototype.create(1, vertex); + try { + Job job = jobPrototype.build(options.getJarClass(), new Configuration()); + job.waitForCompletion(true); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + } + } + } return null; } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java index 4ba3a29bff8d..d0c7b7882b00 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java @@ -199,7 +199,7 @@ public static void main(String[] args) throws Exception { KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); conf.set( - "source", + BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, Base64.encodeBase64String(SerializableUtils.serializeToByteArray(source))); Job job = Job.getInstance(conf, "word count"); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java index 8c4155afb5fe..0cfb14b973a9 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -3,17 +3,19 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.base.Function; +import com.google.common.base.Strings; import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.io.Serializable; import java.util.List; -import javax.annotation.Nullable; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; @@ -25,31 +27,30 @@ /** * Adaptor from Beam {@link BoundedSource} to MapReduce {@link InputFormat}. */ -public class BeamInputFormat extends InputFormat { +public class BeamInputFormat extends InputFormat { + public static final String BEAM_SERIALIZED_BOUNDED_SOURCE = "beam-serialized-bounded-source"; private static final long DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES = 5 * 1000 * 1000; - private BoundedSource> source; + private BoundedSource source; private PipelineOptions options; public BeamInputFormat() { } - public BeamInputFormat(BoundedSource> source, PipelineOptions options) { - this.source = checkNotNull(source, "source"); - this.options = checkNotNull(options, "options"); - } - @Override public List getSplits(JobContext context) throws IOException, InterruptedException { - source = (BoundedSource>) SerializableUtils.deserializeFromByteArray( - Base64.decodeBase64(context.getConfiguration().get("source")), - ""); + String serializedBoundedSource = context.getConfiguration().get(BEAM_SERIALIZED_BOUNDED_SOURCE); + if (Strings.isNullOrEmpty(serializedBoundedSource)) { + return ImmutableList.of(); + } + source = (BoundedSource) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(serializedBoundedSource), "BoundedSource"); try { return FluentIterable.from(source.split(DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES, options)) - .transform(new Function>, InputSplit>() { + .transform(new Function, InputSplit>() { @Override - public InputSplit apply(BoundedSource> source) { + public InputSplit apply(BoundedSource source) { try { return new BeamInputSplit(source.getEstimatedSizeBytes(options)); } catch (Exception e) { @@ -65,8 +66,8 @@ public InputSplit apply(BoundedSource> source) { @Override public RecordReader createRecordReader( InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { - source = (BoundedSource>) SerializableUtils.deserializeFromByteArray( - Base64.decodeBase64(context.getConfiguration().get("source")), + source = (BoundedSource) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(context.getConfiguration().get(BEAM_SERIALIZED_BOUNDED_SOURCE)), ""); return new BeamRecordReader<>(source.createReader(options)); } @@ -102,12 +103,12 @@ public void readFields(DataInput in) throws IOException { } } - private class BeamRecordReader extends RecordReader { + private class BeamRecordReader extends RecordReader { - private final BoundedSource.BoundedReader> reader; + private final BoundedSource.BoundedReader reader; private boolean started; - public BeamRecordReader(BoundedSource.BoundedReader> reader) { + public BeamRecordReader(BoundedSource.BoundedReader reader) { this.reader = checkNotNull(reader, "reader"); this.started = false; } @@ -128,12 +129,13 @@ public boolean nextKeyValue() throws IOException, InterruptedException { @Override public Object getCurrentKey() throws IOException, InterruptedException { - return reader.getCurrent().getKey(); + return "global"; } @Override public Object getCurrentValue() throws IOException, InterruptedException { - return reader.getCurrent().getValue(); + return WindowedValue.timestampedValueInGlobalWindow( + reader.getCurrent(), reader.getCurrentTimestamp()); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index 88fc8d6fe64b..9d2f80dc16fc 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -1,30 +1,83 @@ package org.apache.beam.runners.mapreduce.translation; -import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; -import org.apache.beam.sdk.values.KV; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.NullSideInputReader; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.mapreduce.Mapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Created by peihe on 21/07/2017. */ -public class BeamMapper - extends Mapper { +public class BeamMapper + extends Mapper, Object, WindowedValue> { + + public static final String BEAM_SERIALIZED_DO_FN = "beam-serialized-do-fn"; + private static final Logger LOG = LoggerFactory.getLogger(BeamMapper.class); - private DoFnInvoker, KV> doFnInvoker; + private DoFnRunner doFnRunner; + private PipelineOptions options; @Override - protected void setup(Mapper.Context context) { + protected void setup( + Mapper, Object, WindowedValue>.Context context) { + String serializedDoFn = checkNotNull( + context.getConfiguration().get(BEAM_SERIALIZED_DO_FN), + BEAM_SERIALIZED_DO_FN); + doFnRunner = DoFnRunners.simpleRunner( + options, + (DoFn) SerializableUtils + .deserializeFromByteArray( + Base64.decodeBase64(serializedDoFn), "DoFn"), + NullSideInputReader.empty(), + new MROutputManager(context), + null, + ImmutableList.>of(), + null, + WindowingStrategy.globalDefault()); } @Override protected void map( - KeyInT key, - ValueInT value, - Mapper.Context context) { - System.out.print(String.format("key: %s, value: %s", key, value)); + Object key, + WindowedValue value, + Mapper, Object, WindowedValue>.Context context) { + LOG.info("key: {}, value: {}.", key, value); + doFnRunner.processElement(value); } @Override - protected void cleanup(Mapper.Context context) { + protected void cleanup( + Mapper, Object, WindowedValue>.Context context) { + } + + class MROutputManager implements DoFnRunners.OutputManager { + + private final Mapper.Context context; + + MROutputManager(Mapper.Context context) { + this.context = (Mapper.Context) context; + } + + @Override + public void output(TupleTag tag, WindowedValue output) { + try { + context.write("global", output); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + } + } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index 1ca5a0531c68..da31f8958f28 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -8,6 +8,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -235,6 +236,10 @@ public void addLast(PTransform transform) { path.addLast(transform); } + public Iterable> transforms() { + return path; + } + @Override public boolean equals(Object obj) { if (obj == this) { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java new file mode 100644 index 000000000000..bdbbe5d3f516 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -0,0 +1,95 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; + +/** + * Created by peihe on 24/07/2017. + */ +public class JobPrototype { + + public static JobPrototype create(int stageId, Graph.Vertex vertex) { + return new JobPrototype(stageId, vertex); + } + + private final int stageId; + private final Graph.Vertex vertex; + private final Set dependencies; + + private JobPrototype(int stageId, Graph.Vertex vertex) { + this.stageId = stageId; + this.vertex = checkNotNull(vertex, "vertex"); + this.dependencies = Sets.newHashSet(); + } + + public Job build(Class jarClass, Configuration conf) throws IOException { + Job job = new Job(conf); + conf = job.getConfiguration(); + job.setJarByClass(jarClass); + + // Setup BoundedSources in BeamInputFormat. + // TODO: support more than one inputs + Graph.Vertex head = Iterables.getOnlyElement(vertex.getIncoming()).getHead(); + checkState(head.getTransform() instanceof Read.Bounded); + Read.Bounded read = (Read.Bounded) head.getTransform(); + conf.set( + BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(read.getSource()))); + job.setInputFormatClass(BeamInputFormat.class); + + // Setup DoFns in BeamMapper. + // TODO: support more than one out going edge. + Graph.Edge outEdge = Iterables.getOnlyElement(head.getOutgoing()); + Graph.NodePath outPath = Iterables.getOnlyElement(outEdge.getPaths()); + List doFns = new ArrayList<>(); + doFns.addAll(FluentIterable.from(outPath.transforms()) + .filter(new Predicate>() { + @Override + public boolean apply(PTransform input) { + return !(input instanceof Read.Bounded); + } + }) + .transform(new Function, DoFn>() { + @Override + public DoFn apply(PTransform input) { + checkArgument( + input instanceof ParDo.SingleOutput, "Only support ParDo.SingleOutput."); + ParDo.SingleOutput parDo = (ParDo.SingleOutput) input; + return parDo.getFn(); + }}) + .toList()); + if (vertex.getTransform() instanceof ParDo.SingleOutput) { + doFns.add(((ParDo.SingleOutput) vertex.getTransform()).getFn()); + } else if (vertex.getTransform() instanceof ParDo.MultiOutput) { + doFns.add(((ParDo.MultiOutput) vertex.getTransform()).getFn()); + } + conf.set( + BeamMapper.BEAM_SERIALIZED_DO_FN, + Base64.encodeBase64String(SerializableUtils.serializeToByteArray( + Iterables.getOnlyElement(doFns)))); + job.setMapperClass(BeamMapper.class); + job.setOutputFormatClass(NullOutputFormat.class); + return job; + } +} diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java index 51c26f244673..80df3e1ccac8 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java @@ -17,6 +17,7 @@ import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.log4j.BasicConfigurator; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -63,46 +64,25 @@ public String apply(KV input) { } } - /** - * A PTransform that converts a PCollection containing lines of text into a PCollection of - * formatted word counts. - * - *

Concept #3: This is a custom composite transform that bundles two transforms (ParDo and - * Count) as a reusable PTransform subclass. Using composite transforms allows for easy reuse, - * modular testing, and an improved monitoring experience. - */ - public static class CountWords extends PTransform, - PCollection>> { - @Override - public PCollection> expand(PCollection lines) { - - // Convert lines of text into individual words. - PCollection words = lines.apply( - ParDo.of(new ExtractWordsFn())); - - // Count the number of times each word occurs. - PCollection> wordCounts = - words.apply(Count.perElement()); - - return wordCounts; - } - } - @Test public void testWordCount() { - String input = "gs://apache-beam-samples/shakespeare/kinglear.txt"; + BasicConfigurator.configure(); + + String input = "/Users/peihe/github/beam/LICENSE"; String output = "./output"; - PipelineOptions options = PipelineOptionsFactory.create(); + MapReducePipelineOptions options = PipelineOptionsFactory.as(MapReducePipelineOptions.class); + options.setJarClass(this.getClass()); options.setRunner(MapReduceRunner.class); Pipeline p = Pipeline.create(options); // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the // static FormatAsTextFn() to the ParDo transform. p.apply("ReadLines", TextIO.read().from(input)) - .apply(new CountWords()) - .apply(MapElements.via(new FormatAsTextFn())) - .apply("WriteCounts", TextIO.write().to(output)); + .apply(ParDo.of(new ExtractWordsFn())); +// .apply(Count.perElement()) +// .apply(MapElements.via(new FormatAsTextFn())) +// .apply("WriteCounts", TextIO.write().to(output)); - p.run().waitUntilFinish(); + p.run(); } } From 389b02b576e1d9ea5123905048de3004e462a89a Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 25 Jul 2017 21:44:34 +0800 Subject: [PATCH 06/34] mr-runner: add ParDoOperation and support ParDos chaining. --- .../runners/mapreduce/MapReduceRunner.java | 6 +- .../mapreduce/translation/BeamMapper.java | 57 ++----- .../runners/mapreduce/translation/Graph.java | 71 ++++++--- .../mapreduce/translation/GraphConverter.java | 29 ++-- .../mapreduce/translation/GraphPlanner.java | 25 +-- .../mapreduce/translation/JobPrototype.java | 71 ++++++--- .../mapreduce/translation/OutputReceiver.java | 53 +++++++ .../mapreduce/translation/ParDoOperation.java | 149 ++++++++++++++++++ .../SerializedPipelineOptions.java | 76 +++++++++ .../mapreduce/translation/package-info.java | 22 +++ .../beam/runners/mapreduce/WordCountTest.java | 4 +- 11 files changed, 438 insertions(+), 125 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializedPipelineOptions.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/package-info.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index 0e3142cc360b..11ac9a73af73 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -3,7 +3,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.base.Throwables; -import java.io.IOException; import org.apache.beam.runners.mapreduce.translation.Graph; import org.apache.beam.runners.mapreduce.translation.GraphConverter; import org.apache.beam.runners.mapreduce.translation.GraphPlanner; @@ -48,10 +47,7 @@ public PipelineResult run(Pipeline pipeline) { GraphPlanner planner = new GraphPlanner(); Graph fusedGraph = planner.plan(graph); for (Graph.Vertex vertex : fusedGraph.getAllVertices()) { - if (vertex.getTransform() instanceof GroupByKey - || vertex.getTransform() instanceof Read.Bounded) { - continue; - } else { + if (vertex.getStep().getTransform() instanceof GroupByKey) { JobPrototype jobPrototype = JobPrototype.create(1, vertex); try { Job job = jobPrototype.build(options.getJarClass(), new Configuration()); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index 9d2f80dc16fc..b74797dc4b01 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -2,17 +2,8 @@ import static com.google.common.base.Preconditions.checkNotNull; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.DoFnRunners; -import org.apache.beam.runners.core.NullSideInputReader; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.mapreduce.Mapper; import org.slf4j.Logger; @@ -24,29 +15,19 @@ public class BeamMapper extends Mapper, Object, WindowedValue> { - public static final String BEAM_SERIALIZED_DO_FN = "beam-serialized-do-fn"; - private static final Logger LOG = LoggerFactory.getLogger(BeamMapper.class); + public static final String BEAM_SERIALIZED_PAR_DO_OPERATION = "beam-serialized-par-do-op"; - private DoFnRunner doFnRunner; - private PipelineOptions options; + private ParDoOperation parDoOperation; @Override protected void setup( Mapper, Object, WindowedValue>.Context context) { - String serializedDoFn = checkNotNull( - context.getConfiguration().get(BEAM_SERIALIZED_DO_FN), - BEAM_SERIALIZED_DO_FN); - doFnRunner = DoFnRunners.simpleRunner( - options, - (DoFn) SerializableUtils - .deserializeFromByteArray( - Base64.decodeBase64(serializedDoFn), "DoFn"), - NullSideInputReader.empty(), - new MROutputManager(context), - null, - ImmutableList.>of(), - null, - WindowingStrategy.globalDefault()); + String serializedParDo = checkNotNull( + context.getConfiguration().get(BEAM_SERIALIZED_PAR_DO_OPERATION), + BEAM_SERIALIZED_PAR_DO_OPERATION); + parDoOperation = (ParDoOperation) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(serializedParDo), "DoFn"); + parDoOperation.start(); } @Override @@ -54,30 +35,12 @@ protected void map( Object key, WindowedValue value, Mapper, Object, WindowedValue>.Context context) { - LOG.info("key: {}, value: {}.", key, value); - doFnRunner.processElement(value); + parDoOperation.process(value); } @Override protected void cleanup( Mapper, Object, WindowedValue>.Context context) { - } - - class MROutputManager implements DoFnRunners.OutputManager { - - private final Mapper.Context context; - - MROutputManager(Mapper.Context context) { - this.context = (Mapper.Context) context; - } - - @Override - public void output(TupleTag tag, WindowedValue output) { - try { - context.write("global", output); - } catch (Exception e) { - Throwables.throwIfUnchecked(e); - } - } + parDoOperation.finish(); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index da31f8958f28..6ea774bc1f9f 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -18,6 +18,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TupleTag; import org.apache.commons.lang.builder.ReflectionToStringBuilder; import org.apache.commons.lang.builder.ToStringStyle; @@ -26,7 +27,7 @@ */ public class Graph { - private final Map vertices; + private final Map vertices; private final Map edges; private final Set leafVertices; @@ -36,10 +37,10 @@ public Graph() { this.leafVertices = Sets.newHashSet(); } - public Vertex addVertex(PTransform transform) { - checkState(!vertices.containsKey(transform)); - Vertex v = new Vertex(transform); - vertices.put(transform, v); + public Vertex addVertex(Step step) { + checkState(!vertices.containsKey(step)); + Vertex v = new Vertex(step); + vertices.put(step, v); leafVertices.add(v); return v; } @@ -55,8 +56,8 @@ public Edge addEdge(Vertex head, Vertex tail) { return e; } - public Vertex getVertex(PTransform transform) { - return vertices.get(transform); + public Vertex getVertex(Step step) { + return vertices.get(step); } public Edge getEdge(Vertex head, Vertex tail) { @@ -84,18 +85,18 @@ public void accept(GraphVisitor visitor) { //TODO: add equals, hashCode, toString for following classses. public static class Vertex { - private final PTransform transform; + private final Step step; private final Set incoming; private final Set outgoing; - public Vertex(PTransform transform) { - this.transform = checkNotNull(transform, "transform"); + public Vertex(Step step) { + this.step = checkNotNull(step, "step"); this.incoming = Sets.newHashSet(); this.outgoing = Sets.newHashSet(); } - public PTransform getTransform() { - return transform; + public Step getStep() { + return step; } public Set getIncoming() { @@ -107,11 +108,12 @@ public Set getOutgoing() { } public boolean isSource() { + PTransform transform = step.getTransform(); return transform instanceof Read.Bounded || transform instanceof Read.Unbounded; } public boolean isGroupByKey() { - return transform instanceof GroupByKey; + return step.getTransform() instanceof GroupByKey; } public void addIncoming(Edge edge) { @@ -123,6 +125,7 @@ public void addOutgoing(Edge edge) { } public void accept(GraphVisitor visitor) { + PTransform transform = step.getTransform(); if (transform instanceof ParDo.SingleOutput || transform instanceof ParDo.MultiOutput) { visitor.visitParDo(this); } else if (transform instanceof GroupByKey) { @@ -144,14 +147,14 @@ public boolean equals(Object obj) { } if (obj instanceof Vertex) { Vertex other = (Vertex) obj; - return transform.equals(other.transform); + return step.equals(other.step); } return false; } @Override public int hashCode() { - return Objects.hash(this.getClass(), transform); + return Objects.hash(this.getClass(), step); } @Override @@ -218,7 +221,7 @@ public String toString() { } public static class NodePath { - private final LinkedList> path; + private final LinkedList path; public NodePath() { this.path = new LinkedList<>(); @@ -228,16 +231,16 @@ public NodePath(NodePath nodePath) { this.path = new LinkedList<>(nodePath.path); } - public void addFirst(PTransform transform) { - path.addFirst(transform); + public void addFirst(Step step) { + path.addFirst(step); } - public void addLast(PTransform transform) { - path.addLast(transform); + public void addLast(Step step) { + path.addLast(step); } - public Iterable> transforms() { - return path; + public Iterable steps() { + return ImmutableList.copyOf(path); } @Override @@ -260,14 +263,32 @@ public int hashCode() { @Override public String toString() { StringBuilder sb = new StringBuilder(); - for (PTransform collect : path) { - sb.append(collect.getName() + "|"); + for (Step step : path) { + sb.append(step.getFullName() + "|"); } - // sb.deleteCharAt(sb.length() - 1); + sb.deleteCharAt(sb.length() - 1); return sb.toString(); } } + @AutoValue + public abstract static class Step { + abstract String getFullName(); + // TODO: remove public + public abstract PTransform getTransform(); + abstract List> getInputs(); + abstract List> getOutputs(); + + public static Step of( + String fullName, + PTransform transform, + List> inputs, + List> outputs) { + return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graph_Step( + fullName, transform, inputs, outputs); + } + } + @AutoValue public abstract static class HeadTail { abstract Vertex getHead(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java index 359a6e28bfa6..bd56ac5748f3 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -1,40 +1,49 @@ package org.apache.beam.runners.mapreduce.translation; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import java.util.Map; -import java.util.Set; import org.apache.beam.runners.mapreduce.MapReduceRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; /** * Pipeline translator for {@link MapReduceRunner}. */ public class GraphConverter extends Pipeline.PipelineVisitor.Defaults { - private final Map outputToProducer; + private final Map> pValueToTupleTag; + private final Map, Graph.Vertex> outputToProducer; private final Graph graph; public GraphConverter() { + this.pValueToTupleTag = Maps.newHashMap(); this.outputToProducer = Maps.newHashMap(); this.graph = new Graph(); } @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { - Graph.Vertex v = graph.addVertex(node.getTransform()); - - for (PValue input : node.getInputs().values()) { - if (outputToProducer.containsKey(input)) { - Graph.Vertex producer = outputToProducer.get(input); + Graph.Step step = Graph.Step.of( + node.getFullName(), + node.getTransform(), + ImmutableList.copyOf(node.getInputs().keySet()), + ImmutableList.copyOf(node.getOutputs().keySet())); + Graph.Vertex v = graph.addVertex(step); + + for (PValue pValue : node.getInputs().values()) { + TupleTag tag = pValueToTupleTag.get(pValue); + if (outputToProducer.containsKey(tag)) { + Graph.Vertex producer = outputToProducer.get(tag); graph.addEdge(producer, v); } } - for (PValue output : node.getOutputs().values()) { - outputToProducer.put(output, v); + for (Map.Entry, PValue> entry : node.getOutputs().entrySet()) { + pValueToTupleTag.put(entry.getValue(), entry.getKey()); + outputToProducer.put(entry.getKey(), v); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 793efd75c252..9198b28f4418 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -31,23 +31,24 @@ public void visitRead(Graph.Vertex read) { // drop if read is leaf vertex. return; } - Graph.Vertex v = fusedGraph.addVertex(read.getTransform()); - workingPath.addFirst(read.getTransform()); + Graph.Vertex v = fusedGraph.addVertex(read.getStep()); + workingPath.addFirst(read.getStep()); Graph.Edge edge = fusedGraph.addEdge(v, workingVertex); edge.addPath(workingPath); } @Override public void visitParDo(Graph.Vertex parDo) { + Graph.Step step = parDo.getStep(); checkArgument( - parDo.getTransform().getAdditionalInputs().isEmpty(), - "Side inputs are not supported."); + step.getTransform().getAdditionalInputs().isEmpty(), + "Side inputs are not " + "supported."); if (workingVertex == null) { // Leaf vertex - workingVertex = fusedGraph.addVertex(parDo.getTransform()); + workingVertex = fusedGraph.addVertex(step); workingPath = new Graph.NodePath(); } else { - workingPath.addFirst(parDo.getTransform()); + workingPath.addFirst(step); } checkArgument( parDo.getIncoming().size() == 1, @@ -74,10 +75,11 @@ public void visitGroupByKey(Graph.Vertex groupByKey) { if (workingVertex == null) { return; } - Graph.Vertex v = fusedGraph.addVertex(groupByKey.getTransform()); - workingPath.addFirst(groupByKey.getTransform()); - Graph.Edge edge = fusedGraph.addEdge(v, workingVertex); + Graph.Step step = groupByKey.getStep(); + Graph.Vertex addedGroupByKey = fusedGraph.addVertex(step); + Graph.Edge edge = fusedGraph.addEdge(addedGroupByKey, workingVertex); edge.addPath(workingPath); + workingVertex = addedGroupByKey; processParent(groupByKey.getIncoming().iterator().next().getHead()); } @@ -86,13 +88,14 @@ public Graph getFusedGraph() { } private void processParent(Graph.Vertex parent) { - Graph.Vertex v = fusedGraph.getVertex(parent.getTransform()); + Graph.Step step = parent.getStep(); + Graph.Vertex v = fusedGraph.getVertex(step); if (v == null) { parent.accept(this); } else { // TODO: parent is consumed more than once. // It is duplicated in multiple outgoing path. Figure out the impact. - workingPath.addFirst(parent.getTransform()); + workingPath.addFirst(step); fusedGraph.getEdge(v, workingVertex).addPath(workingPath); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index bdbbe5d3f516..7cdf6979a2fb 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -4,21 +4,23 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Set; -import javax.annotation.Nullable; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Job; @@ -51,8 +53,9 @@ public Job build(Class jarClass, Configuration conf) throws IOException { // Setup BoundedSources in BeamInputFormat. // TODO: support more than one inputs Graph.Vertex head = Iterables.getOnlyElement(vertex.getIncoming()).getHead(); - checkState(head.getTransform() instanceof Read.Bounded); - Read.Bounded read = (Read.Bounded) head.getTransform(); + Graph.Step headStep = head.getStep(); + checkState(headStep.getTransform() instanceof Read.Bounded); + Read.Bounded read = (Read.Bounded) headStep.getTransform(); conf.set( BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, Base64.encodeBase64String(SerializableUtils.serializeToByteArray(read.getSource()))); @@ -62,34 +65,52 @@ public Job build(Class jarClass, Configuration conf) throws IOException { // TODO: support more than one out going edge. Graph.Edge outEdge = Iterables.getOnlyElement(head.getOutgoing()); Graph.NodePath outPath = Iterables.getOnlyElement(outEdge.getPaths()); - List doFns = new ArrayList<>(); - doFns.addAll(FluentIterable.from(outPath.transforms()) - .filter(new Predicate>() { + List parDos = new ArrayList<>(); + parDos.addAll(FluentIterable.from(outPath.steps()) + .filter(new Predicate() { @Override - public boolean apply(PTransform input) { - return !(input instanceof Read.Bounded); - } - }) - .transform(new Function, DoFn>() { - @Override - public DoFn apply(PTransform input) { - checkArgument( - input instanceof ParDo.SingleOutput, "Only support ParDo.SingleOutput."); - ParDo.SingleOutput parDo = (ParDo.SingleOutput) input; - return parDo.getFn(); + public boolean apply(Graph.Step input) { + PTransform transform = input.getTransform(); + return transform instanceof ParDo.SingleOutput + || transform instanceof ParDo.MultiOutput; }}) .toList()); - if (vertex.getTransform() instanceof ParDo.SingleOutput) { - doFns.add(((ParDo.SingleOutput) vertex.getTransform()).getFn()); - } else if (vertex.getTransform() instanceof ParDo.MultiOutput) { - doFns.add(((ParDo.MultiOutput) vertex.getTransform()).getFn()); + Graph.Step vertexStep = vertex.getStep(); + if (vertexStep.getTransform() instanceof ParDo.SingleOutput + || vertexStep.getTransform() instanceof ParDo.MultiOutput) { + parDos.add(vertexStep); + } + + ParDoOperation root = null; + ParDoOperation prev = null; + for (Graph.Step step : parDos) { + ParDoOperation current = new ParDoOperation( + getDoFn(step.getTransform()), + PipelineOptionsFactory.create(), + (TupleTag) step.getOutputs().iterator().next(), + ImmutableList.>of(), + WindowingStrategy.globalDefault()); + if (root == null) { + root = current; + } else { + // TODO: set a proper outputNum for ParDo.MultiOutput instead of zero. + current.attachInput(prev, 0); + } + prev = current; } conf.set( - BeamMapper.BEAM_SERIALIZED_DO_FN, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray( - Iterables.getOnlyElement(doFns)))); + BeamMapper.BEAM_SERIALIZED_PAR_DO_OPERATION, + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(root))); job.setMapperClass(BeamMapper.class); job.setOutputFormatClass(NullOutputFormat.class); return job; } + + private DoFn getDoFn(PTransform transform) { + if (transform instanceof ParDo.SingleOutput) { + return ((ParDo.SingleOutput) transform).getFn(); + } else { + return ((ParDo.MultiOutput) transform).getFn(); + } + } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java new file mode 100644 index 000000000000..59cee3c557b4 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** + * OutputReceiver that forwards each input it receives to each of a list of down stream + * ParDoOperations. + */ +public class OutputReceiver implements Serializable { + private final List receiverParDos = new ArrayList<>(); + + /** + * Adds a new receiver that this OutputReceiver forwards to. + */ + public void addOutput(ParDoOperation receiver) { + receiverParDos.add(receiver); + } + + public List getReceiverParDos() { + return ImmutableList.copyOf(receiverParDos); + } + + /** + * Processes the element. + */ + public void process(Object elem) { + for (ParDoOperation out : receiverParDos) { + if (out != null) { + out.process(elem); + } + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java new file mode 100644 index 000000000000..97473bb03ff8 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.NullSideInputReader; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operation for ParDo. + */ +public class ParDoOperation implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(ParDoOperation.class); + + private final DoFn doFn; + private final SerializedPipelineOptions options; + private final TupleTag mainOutputTag; + private final List> sideOutputTags; + private final WindowingStrategy windowingStrategy; + private final OutputReceiver[] receivers; + + private DoFnRunner fnRunner; + + public ParDoOperation( + DoFn doFn, + PipelineOptions options, + TupleTag mainOutputTag, + List> sideOutputTags, + WindowingStrategy windowingStrategy) { + this.doFn = checkNotNull(doFn, "doFn"); + this.options = new SerializedPipelineOptions(checkNotNull(options, "options")); + this.mainOutputTag = checkNotNull(mainOutputTag, "mainOutputTag"); + this.sideOutputTags = checkNotNull(sideOutputTags, "sideOutputTags"); + this.windowingStrategy = checkNotNull(windowingStrategy, "windowingStrategy"); + int numOutputs = 1 + sideOutputTags.size(); + this.receivers = new OutputReceiver[numOutputs]; + for (int i = 0; i < numOutputs; ++i) { + receivers[i] = new OutputReceiver(); + } + } + + /** + * Adds an input to this ParDoOperation, coming from the given output of the given source. + */ + public void attachInput(ParDoOperation source, int outputNum) { + OutputReceiver fanOut = source.receivers[outputNum]; + fanOut.addOutput(this); + } + + /** + * Starts this Operation's execution. + * + *

Called after all successors consuming operations have been started. + */ + public void start() { + fnRunner = DoFnRunners.simpleRunner( + options.getPipelineOptions(), + doFn, + NullSideInputReader.empty(), + new ParDoOutputManager(), + mainOutputTag, + sideOutputTags, + null, + windowingStrategy); + fnRunner.startBundle(); + for (OutputReceiver receiver : receivers) { + if (receiver == null) { + continue; + } + for (ParDoOperation parDo : receiver.getReceiverParDos()) { + parDo.start(); + } + } + } + + /** + * Processes the element. + */ + public void process(Object elem) { + LOG.info("elem: {}.", elem); + fnRunner.processElement((WindowedValue) elem); + } + + /** + * Finishes this Operation's execution. + * + *

Called after all predecessors producing operations have been finished. + */ + public void finish() { + for (OutputReceiver receiver : receivers) { + if (receiver == null) { + continue; + } + for (ParDoOperation parDo : receiver.getReceiverParDos()) { + parDo.finish(); + } + } + fnRunner.finishBundle(); + } + + private class ParDoOutputManager implements DoFnRunners.OutputManager { + + @Nullable + private OutputReceiver getReceiverOrNull(TupleTag tag) { + if (tag.equals(mainOutputTag)) { + return receivers[0]; + } else if (sideOutputTags.contains(tag)) { + return receivers[sideOutputTags.indexOf(tag) + 1]; + } else { + return null; + } + } + + @Override + public void output(TupleTag tupleTag, WindowedValue windowedValue) { + OutputReceiver receiver = getReceiverOrNull(tupleTag); + if (receiver != null) { + receiver.process(windowedValue); + } + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializedPipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializedPipelineOptions.java new file mode 100644 index 000000000000..5c37b7c49b0b --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializedPipelineOptions.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.common.ReflectHelpers; + +/** + * Encapsulates the PipelineOptions in serialized form to ship them to the cluster. + */ +public class SerializedPipelineOptions implements Serializable { + + private final byte[] serializedOptions; + + /** Lazily initialized copy of deserialized options. */ + private transient PipelineOptions pipelineOptions; + + public SerializedPipelineOptions(PipelineOptions options) { + checkNotNull(options, "PipelineOptions must not be null."); + + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + createMapper().writeValue(baos, options); + this.serializedOptions = baos.toByteArray(); + } catch (Exception e) { + throw new RuntimeException("Couldn't serialize PipelineOptions.", e); + } + + } + + public PipelineOptions getPipelineOptions() { + if (pipelineOptions == null) { + try { + pipelineOptions = createMapper().readValue(serializedOptions, PipelineOptions.class); + + FileSystems.setDefaultPipelineOptions(pipelineOptions); + } catch (IOException e) { + throw new RuntimeException("Couldn't deserialize the PipelineOptions.", e); + } + } + + return pipelineOptions; + } + + /** + * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing + * for user specified configuration injection into the ObjectMapper. This supports user custom + * types on {@link PipelineOptions}. + */ + private static ObjectMapper createMapper() { + return new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/package-info.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/package-info.java new file mode 100644 index 000000000000..c9360ac64cca --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Implementation for translating Beam pipelines to MapReduce jobs. + */ +package org.apache.beam.runners.mapreduce.translation; diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java index 80df3e1ccac8..f5eee28f9bb2 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java @@ -78,8 +78,8 @@ public void testWordCount() { // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the // static FormatAsTextFn() to the ParDo transform. p.apply("ReadLines", TextIO.read().from(input)) - .apply(ParDo.of(new ExtractWordsFn())); -// .apply(Count.perElement()) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Count.perElement()); // .apply(MapElements.via(new FormatAsTextFn())) // .apply("WriteCounts", TextIO.write().to(output)); From 923190dca2426711e30e5c5fe7093e14fcbefe07 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 26 Jul 2017 21:19:30 +0800 Subject: [PATCH 07/34] mr-runner: add BeamReducer and support GroupByKey. --- .../mapreduce/translation/BeamMapper.java | 13 +- .../mapreduce/translation/BeamReducer.java | 68 ++++++++++ .../runners/mapreduce/translation/Graph.java | 36 ++++-- .../mapreduce/translation/GraphConverter.java | 26 +++- .../mapreduce/translation/GraphPlanner.java | 28 +++- .../GroupAlsoByWindowsParDoOperation.java | 38 ++++++ ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 120 ++++++++++++++++++ .../mapreduce/translation/JobPrototype.java | 46 +++++-- .../translation/NormalParDoOperation.java | 49 +++++++ .../mapreduce/translation/Operation.java | 69 ++++++++++ .../mapreduce/translation/OutputReceiver.java | 12 +- .../mapreduce/translation/ParDoOperation.java | 73 ++++------- .../mapreduce/translation/WriteOperation.java | 52 ++++++++ .../beam/runners/mapreduce/WordCountTest.java | 7 - 14 files changed, 534 insertions(+), 103 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index b74797dc4b01..11ecc8dee099 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -6,8 +6,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.mapreduce.Mapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; /** * Created by peihe on 21/07/2017. @@ -15,7 +14,7 @@ public class BeamMapper extends Mapper, Object, WindowedValue> { - public static final String BEAM_SERIALIZED_PAR_DO_OPERATION = "beam-serialized-par-do-op"; + public static final String BEAM_PAR_DO_OPERATION_MAPPER = "beam-par-do-op-mapper"; private ParDoOperation parDoOperation; @@ -23,11 +22,11 @@ public class BeamMapper protected void setup( Mapper, Object, WindowedValue>.Context context) { String serializedParDo = checkNotNull( - context.getConfiguration().get(BEAM_SERIALIZED_PAR_DO_OPERATION), - BEAM_SERIALIZED_PAR_DO_OPERATION); + context.getConfiguration().get(BEAM_PAR_DO_OPERATION_MAPPER), + BEAM_PAR_DO_OPERATION_MAPPER); parDoOperation = (ParDoOperation) SerializableUtils.deserializeFromByteArray( - Base64.decodeBase64(serializedParDo), "DoFn"); - parDoOperation.start(); + Base64.decodeBase64(serializedParDo), "ParDoOperation"); + parDoOperation.start((TaskInputOutputContext) context); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java new file mode 100644 index 000000000000..8eb79382b5bc --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java @@ -0,0 +1,68 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.FluentIterable; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; + +/** + * Created by peihe on 25/07/2017. + */ +public class BeamReducer + extends Reducer> { + + public static final String BEAM_PAR_DO_OPERATION_REDUCER = "beam-par-do-op-reducer"; + + private ParDoOperation parDoOperation; + + @Override + protected void setup( + Reducer>.Context context) { + String serializedParDo = checkNotNull( + context.getConfiguration().get(BEAM_PAR_DO_OPERATION_REDUCER), + BEAM_PAR_DO_OPERATION_REDUCER); + parDoOperation = (ParDoOperation) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(serializedParDo), "ParDoOperation"); + parDoOperation.start((TaskInputOutputContext) context); + } + + @Override + protected void reduce( + Object key, + Iterable values, + Reducer>.Context context) { + Iterable decodedValues = FluentIterable.from(values) + .transform(new Function() { + @Override + public Object apply(byte[] input) { + ByteArrayInputStream inStream = new ByteArrayInputStream(input); + try { + // TODO: setup coders. + return NullableCoder.of(BigEndianLongCoder.of()).decode(inStream); + } catch (IOException e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + }); + parDoOperation.process( + WindowedValue.valueInGlobalWindow(KV.of(key, decodedValues))); + } + + @Override + protected void cleanup( + Reducer>.Context context) { + parDoOperation.finish(); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index 6ea774bc1f9f..867d1af34745 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -12,13 +12,14 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.commons.lang.builder.ReflectionToStringBuilder; import org.apache.commons.lang.builder.ToStringStyle; @@ -45,10 +46,10 @@ public Vertex addVertex(Step step) { return v; } - public Edge addEdge(Vertex head, Vertex tail) { + public Edge addEdge(Vertex head, Vertex tail, Coder coder) { HeadTail headTail = HeadTail.of(head, tail); checkState(!edges.containsKey(headTail)); - Edge e = new Edge(headTail); + Edge e = new Edge(headTail, coder); edges.put(headTail, e); head.addOutgoing(e); tail.addIncoming(e); @@ -166,18 +167,16 @@ public String toString() { public static class Edge { private final HeadTail headTail; + private final Coder coder; private final Set paths; - public static Edge of(Vertex head, Vertex tail) { - return of(HeadTail.of(head, tail)); + public static Edge of(HeadTail headTail, Coder coder) { + return new Edge(headTail, coder); } - public static Edge of(HeadTail headTail) { - return new Edge(headTail); - } - - private Edge(HeadTail headTail) { + private Edge(HeadTail headTail, Coder coder) { this.headTail = checkNotNull(headTail, "headTail"); + this.coder = checkNotNull(coder, "coder"); this.paths = Sets.newHashSet(); } @@ -189,6 +188,10 @@ public Vertex getTail() { return headTail.getTail(); } + public Coder getCoder() { + return coder; + } + public Set getPaths() { return paths; } @@ -204,14 +207,15 @@ public boolean equals(Object obj) { } if (obj instanceof Edge) { Edge other = (Edge) obj; - return headTail.equals(other.headTail) && paths.equals(paths); + return headTail.equals(other.headTail) + && paths.equals(other.paths) && coder.equals(other.coder); } return false; } @Override public int hashCode() { - return Objects.hash(headTail, paths); + return Objects.hash(headTail, paths, coder); } @Override @@ -266,7 +270,9 @@ public String toString() { for (Step step : path) { sb.append(step.getFullName() + "|"); } - sb.deleteCharAt(sb.length() - 1); + if (path.size() > 0) { + sb.deleteCharAt(sb.length() - 1); + } return sb.toString(); } } @@ -276,16 +282,18 @@ public abstract static class Step { abstract String getFullName(); // TODO: remove public public abstract PTransform getTransform(); + abstract WindowingStrategy getWindowingStrategy(); abstract List> getInputs(); abstract List> getOutputs(); public static Step of( String fullName, PTransform transform, + WindowingStrategy windowingStrategy, List> inputs, List> outputs) { return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graph_Step( - fullName, transform, inputs, outputs); + fullName, transform, windowingStrategy, inputs, outputs); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java index bd56ac5748f3..e7e7598c1e56 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -6,8 +6,13 @@ import org.apache.beam.runners.mapreduce.MapReduceRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** * Pipeline translator for {@link MapReduceRunner}. @@ -26,9 +31,12 @@ public GraphConverter() { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { + WindowingStrategy windowingStrategy = + getWindowingStrategy(node.getOutputs().values().iterator().next()); Graph.Step step = Graph.Step.of( node.getFullName(), node.getTransform(), + windowingStrategy, ImmutableList.copyOf(node.getInputs().keySet()), ImmutableList.copyOf(node.getOutputs().keySet())); Graph.Vertex v = graph.addVertex(step); @@ -37,7 +45,9 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { TupleTag tag = pValueToTupleTag.get(pValue); if (outputToProducer.containsKey(tag)) { Graph.Vertex producer = outputToProducer.get(tag); - graph.addEdge(producer, v); + + PCollection pc = (PCollection) pValue; + graph.addEdge(producer, v, pc.getCoder()); } } @@ -47,6 +57,20 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { } } + private WindowingStrategy getWindowingStrategy(PValue pValue) { + if (pValue instanceof PCollection) { + return ((PCollection) pValue).getWindowingStrategy(); + } else if (pValue instanceof PCollectionList) { + return ((PCollectionList) pValue).get(0).getWindowingStrategy(); + } else if (pValue instanceof PCollectionTuple) { + return ((PCollectionTuple) pValue).getAll().values().iterator().next().getWindowingStrategy(); + } else if (pValue instanceof PCollectionView) { + return ((PCollectionView) pValue).getPCollection().getWindowingStrategy(); + } else { + throw new RuntimeException("Unexpected pValue type: " + pValue.getClass()); + } + } + public Graph getGraph() { return graph; } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 9198b28f4418..9ae836585969 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -2,6 +2,8 @@ import static com.google.common.base.Preconditions.checkArgument; +import org.apache.beam.sdk.coders.Coder; + /** * Created by peihe on 06/07/2017. */ @@ -18,6 +20,7 @@ private class FusionVisitor implements GraphVisitor { private Graph fusedGraph; private Graph.Vertex workingVertex; private Graph.NodePath workingPath; + private Coder workingEdgeCoder; FusionVisitor() { fusedGraph = new Graph(); @@ -33,7 +36,7 @@ public void visitRead(Graph.Vertex read) { } Graph.Vertex v = fusedGraph.addVertex(read.getStep()); workingPath.addFirst(read.getStep()); - Graph.Edge edge = fusedGraph.addEdge(v, workingVertex); + Graph.Edge edge = fusedGraph.addEdge(v, workingVertex, workingEdgeCoder); edge.addPath(workingPath); } @@ -43,17 +46,20 @@ public void visitParDo(Graph.Vertex parDo) { checkArgument( step.getTransform().getAdditionalInputs().isEmpty(), "Side inputs are not " + "supported."); + checkArgument( + parDo.getIncoming().size() == 1, + "Side inputs are not supported."); + Graph.Edge inEdge = parDo.getIncoming().iterator().next(); + if (workingVertex == null) { // Leaf vertex workingVertex = fusedGraph.addVertex(step); workingPath = new Graph.NodePath(); + workingEdgeCoder = inEdge.getCoder(); } else { workingPath.addFirst(step); } - checkArgument( - parDo.getIncoming().size() == 1, - "Side inputs are not supported."); - processParent(parDo.getIncoming().iterator().next().getHead()); + processParent(inEdge.getHead()); } @Override @@ -66,6 +72,7 @@ public void visitFlatten(Graph.Vertex flatten) { for (Graph.Edge e : flatten.getIncoming()) { workingPath = new Graph.NodePath(basePath); workingVertex = baseVertex; + workingEdgeCoder = e.getCoder(); processParent(e.getHead()); } } @@ -77,10 +84,17 @@ public void visitGroupByKey(Graph.Vertex groupByKey) { } Graph.Step step = groupByKey.getStep(); Graph.Vertex addedGroupByKey = fusedGraph.addVertex(step); - Graph.Edge edge = fusedGraph.addEdge(addedGroupByKey, workingVertex); + + Graph.Edge edge = fusedGraph.addEdge( + addedGroupByKey, + workingVertex, + workingEdgeCoder); edge.addPath(workingPath); + Graph.Edge inEdge = groupByKey.getIncoming().iterator().next(); workingVertex = addedGroupByKey; - processParent(groupByKey.getIncoming().iterator().next().getHead()); + workingPath = new Graph.NodePath(); + workingEdgeCoder = inEdge.getCoder(); + processParent(inEdge.getHead()); } public Graph getFusedGraph() { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java new file mode 100644 index 000000000000..1da8d262212f --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java @@ -0,0 +1,38 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.util.List; +import org.apache.beam.runners.core.SystemReduceFn; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * Created by peihe on 26/07/2017. + */ +public class GroupAlsoByWindowsParDoOperation extends ParDoOperation { + + private final Coder inputCoder; + + public GroupAlsoByWindowsParDoOperation( + PipelineOptions options, + TupleTag mainOutputTag, + List> sideOutputTags, + WindowingStrategy windowingStrategy, + Coder inputCoder) { + super(options, mainOutputTag, sideOutputTags, windowingStrategy); + this.inputCoder = checkNotNull(inputCoder, "inputCoder"); + } + + @Override + DoFn getDoFn() { + return new GroupAlsoByWindowsViaOutputBufferDoFn( + windowingStrategy, + SystemReduceFn.buffering(inputCoder), + mainOutputTag, + createOutputManager()); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java new file mode 100644 index 000000000000..0b8a8761c83c --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -0,0 +1,120 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.List; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.InMemoryStateInternals; +import org.apache.beam.runners.core.InMemoryTimerInternals; +import org.apache.beam.runners.core.NullSideInputReader; +import org.apache.beam.runners.core.OutputWindowedValue; +import org.apache.beam.runners.core.ReduceFnRunner; +import org.apache.beam.runners.core.SystemReduceFn; +import org.apache.beam.runners.core.construction.TriggerTranslation; +import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; +import org.apache.beam.runners.core.triggers.TriggerStateMachines; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.joda.time.Instant; + +/** + * The default batch implementation, if no specialized "fast path" implementation is applicable. + */ +public class GroupAlsoByWindowsViaOutputBufferDoFn + extends DoFn>>, KV> { + + private final WindowingStrategy windowingStrategy; + private final SystemReduceFn reduceFn; + private final TupleTag> mainTag; + private transient DoFnRunners.OutputManager outputManager; + + public GroupAlsoByWindowsViaOutputBufferDoFn( + WindowingStrategy windowingStrategy, + SystemReduceFn reduceFn, + TupleTag> mainTag, + DoFnRunners.OutputManager outputManager) { + this.windowingStrategy = checkNotNull(windowingStrategy, "windowingStrategy"); + this.reduceFn = checkNotNull(reduceFn, "reduceFn"); + this.mainTag = checkNotNull(mainTag, "mainTag"); + this.outputManager = checkNotNull(outputManager, "outputManager"); + } + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + K key = c.element().getKey(); + // Used with Batch, we know that all the data is available for this key. We can't use the + // timer manager from the context because it doesn't exist. So we create one and emulate the + // watermark, knowing that we have all data and it is in timestamp order. + InMemoryTimerInternals timerInternals = new InMemoryTimerInternals(); + ReduceFnRunner runner = new ReduceFnRunner<>( + key, + windowingStrategy, + ExecutableTriggerStateMachine.create( + TriggerStateMachines.stateMachineForTrigger( + TriggerTranslation.toProto(windowingStrategy.getTrigger()))), + InMemoryStateInternals.forKey(key), + timerInternals, + outputWindowedValue(), + NullSideInputReader.empty(), + reduceFn, + c.getPipelineOptions()); + + Iterable>> chunks = + Iterables.partition(c.element().getValue(), 1000); + for (Iterable> chunk : chunks) { + // Process the chunk of elements. + runner.processElements(chunk); + + // Then, since elements are sorted by their timestamp, advance the input watermark + // to the first element, and fire any timers that may have been scheduled. + // TODO: re-enable once elements are sorted. + // timerInternals.advanceInputWatermark(chunk.iterator().next().getTimestamp()); + + // Fire any processing timers that need to fire + timerInternals.advanceProcessingTime(Instant.now()); + + // Leave the output watermark undefined. Since there's no late data in batch mode + // there's really no need to track it as we do for streaming. + } + + // Finish any pending windows by advancing the input watermark to infinity. + timerInternals.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); + + // Finally, advance the processing time to infinity to fire any timers. + timerInternals.advanceProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE); + + runner.persist(); + } + + private OutputWindowedValue> outputWindowedValue() { + return new OutputWindowedValue>() { + @Override + public void outputWindowedValue( + KV output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + outputManager.output(mainTag, + WindowedValue.of(output, timestamp, windows, pane)); + } + + @Override + public void outputWindowedValue( + TupleTag tag, + AdditionalOutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + outputManager.output(tag, + WindowedValue.of(output, timestamp, windows, pane)); + } + }; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 7cdf6979a2fb..34266f4f8bcf 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -1,6 +1,5 @@ package org.apache.beam.runners.mapreduce.translation; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -16,13 +15,14 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; @@ -49,10 +49,15 @@ public Job build(Class jarClass, Configuration conf) throws IOException { Job job = new Job(conf); conf = job.getConfiguration(); job.setJarByClass(jarClass); + conf.set( + "io.serializations", + "org.apache.hadoop.io.serializer.WritableSerialization," + + "org.apache.hadoop.io.serializer.JavaSerialization"); // Setup BoundedSources in BeamInputFormat. - // TODO: support more than one inputs - Graph.Vertex head = Iterables.getOnlyElement(vertex.getIncoming()).getHead(); + // TODO: support more than one in-edge + Graph.Edge inEdge = Iterables.getOnlyElement(vertex.getIncoming()); + Graph.Vertex head = inEdge.getHead(); Graph.Step headStep = head.getStep(); checkState(headStep.getTransform() instanceof Read.Bounded); Read.Bounded read = (Read.Bounded) headStep.getTransform(); @@ -62,11 +67,10 @@ public Job build(Class jarClass, Configuration conf) throws IOException { job.setInputFormatClass(BeamInputFormat.class); // Setup DoFns in BeamMapper. - // TODO: support more than one out going edge. - Graph.Edge outEdge = Iterables.getOnlyElement(head.getOutgoing()); - Graph.NodePath outPath = Iterables.getOnlyElement(outEdge.getPaths()); + // TODO: support more than one in-path. + Graph.NodePath inPath = Iterables.getOnlyElement(inEdge.getPaths()); List parDos = new ArrayList<>(); - parDos.addAll(FluentIterable.from(outPath.steps()) + parDos.addAll(FluentIterable.from(inPath.steps()) .filter(new Predicate() { @Override public boolean apply(Graph.Step input) { @@ -84,12 +88,12 @@ public boolean apply(Graph.Step input) { ParDoOperation root = null; ParDoOperation prev = null; for (Graph.Step step : parDos) { - ParDoOperation current = new ParDoOperation( + ParDoOperation current = new NormalParDoOperation( getDoFn(step.getTransform()), PipelineOptionsFactory.create(), (TupleTag) step.getOutputs().iterator().next(), ImmutableList.>of(), - WindowingStrategy.globalDefault()); + step.getWindowingStrategy()); if (root == null) { root = current; } else { @@ -98,10 +102,30 @@ public boolean apply(Graph.Step input) { } prev = current; } + // TODO: get coders from pipeline. + WriteOperation writeOperation = new WriteOperation(inEdge.getCoder()); + writeOperation.attachInput(prev, 0); conf.set( - BeamMapper.BEAM_SERIALIZED_PAR_DO_OPERATION, + BeamMapper.BEAM_PAR_DO_OPERATION_MAPPER, Base64.encodeBase64String(SerializableUtils.serializeToByteArray(root))); job.setMapperClass(BeamMapper.class); + + if (vertexStep.getTransform() instanceof GroupByKey) { + // Setup BeamReducer + ParDoOperation operation = new GroupAlsoByWindowsParDoOperation( + PipelineOptionsFactory.create(), + (TupleTag) vertexStep.getOutputs().iterator().next(), + ImmutableList.>of(), + vertexStep.getWindowingStrategy(), + inEdge.getCoder()); + // TODO: handle the map output key type. + job.setMapOutputKeyClass(BytesWritable.class); + job.setMapOutputValueClass(byte[].class); + conf.set( + BeamReducer.BEAM_PAR_DO_OPERATION_REDUCER, + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(operation))); + job.setReducerClass(BeamReducer.class); + } job.setOutputFormatClass(NullOutputFormat.class); return job; } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java new file mode 100644 index 000000000000..1da39a9b955f --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.util.List; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * Created by peihe on 26/07/2017. + */ +public class NormalParDoOperation extends ParDoOperation { + + private final DoFn doFn; + + public NormalParDoOperation( + DoFn doFn, + PipelineOptions options, + TupleTag mainOutputTag, + List> sideOutputTags, + WindowingStrategy windowingStrategy) { + super(options, mainOutputTag, sideOutputTags, windowingStrategy); + this.doFn = checkNotNull(doFn, "doFn"); + } + + @Override + DoFn getDoFn() { + return doFn; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java new file mode 100644 index 000000000000..5700e899f5f2 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java @@ -0,0 +1,69 @@ +package org.apache.beam.runners.mapreduce.translation; + +import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.List; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; + +/** + * Created by peihe on 26/07/2017. + */ +public abstract class Operation implements Serializable { + private final OutputReceiver[] receivers; + + public Operation(int numOutputs) { + this.receivers = new OutputReceiver[numOutputs]; + for (int i = 0; i < numOutputs; ++i) { + receivers[i] = new OutputReceiver(); + } + } + + /** + * Starts this Operation's execution. + * + *

Called after all successors consuming operations have been started. + */ + public void start(TaskInputOutputContext taskContext) { + for (OutputReceiver receiver : receivers) { + if (receiver == null) { + continue; + } + for (Operation operation : receiver.getReceivingOperations()) { + operation.start(taskContext); + } + } + } + + /** + * Processes the element. + */ + public abstract void process(Object elem); + + /** + * Finishes this Operation's execution. + * + *

Called after all predecessors producing operations have been finished. + */ + public void finish() { + for (OutputReceiver receiver : receivers) { + if (receiver == null) { + continue; + } + for (Operation operation : receiver.getReceivingOperations()) { + operation.finish(); + } + } + } + + public List getOutputReceivers() { + return ImmutableList.copyOf(receivers); + } + + /** + * Adds an input to this ParDoOperation, coming from the given output of the given source. + */ + public void attachInput(Operation source, int outputNum) { + OutputReceiver fanOut = source.receivers[outputNum]; + fanOut.addOutput(this); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java index 59cee3c557b4..334767239501 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java @@ -27,24 +27,24 @@ * ParDoOperations. */ public class OutputReceiver implements Serializable { - private final List receiverParDos = new ArrayList<>(); + private final List receivingOperations = new ArrayList<>(); /** * Adds a new receiver that this OutputReceiver forwards to. */ - public void addOutput(ParDoOperation receiver) { - receiverParDos.add(receiver); + public void addOutput(Operation receiver) { + receivingOperations.add(receiver); } - public List getReceiverParDos() { - return ImmutableList.copyOf(receiverParDos); + public List getReceivingOperations() { + return ImmutableList.copyOf(receivingOperations); } /** * Processes the element. */ public void process(Object elem) { - for (ParDoOperation out : receiverParDos) { + for (Operation out : receivingOperations) { if (out != null) { out.process(elem); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java index 97473bb03ff8..2627d20a37b1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -19,7 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import java.io.Serializable; import java.util.List; import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunner; @@ -30,109 +29,83 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Operation for ParDo. */ -public class ParDoOperation implements Serializable { +public abstract class ParDoOperation extends Operation { private static final Logger LOG = LoggerFactory.getLogger(ParDoOperation.class); - private final DoFn doFn; - private final SerializedPipelineOptions options; - private final TupleTag mainOutputTag; + protected final SerializedPipelineOptions options; + protected final TupleTag mainOutputTag; private final List> sideOutputTags; - private final WindowingStrategy windowingStrategy; - private final OutputReceiver[] receivers; + protected final WindowingStrategy windowingStrategy; private DoFnRunner fnRunner; public ParDoOperation( - DoFn doFn, PipelineOptions options, TupleTag mainOutputTag, List> sideOutputTags, WindowingStrategy windowingStrategy) { - this.doFn = checkNotNull(doFn, "doFn"); + super(1 + sideOutputTags.size()); this.options = new SerializedPipelineOptions(checkNotNull(options, "options")); this.mainOutputTag = checkNotNull(mainOutputTag, "mainOutputTag"); this.sideOutputTags = checkNotNull(sideOutputTags, "sideOutputTags"); this.windowingStrategy = checkNotNull(windowingStrategy, "windowingStrategy"); - int numOutputs = 1 + sideOutputTags.size(); - this.receivers = new OutputReceiver[numOutputs]; - for (int i = 0; i < numOutputs; ++i) { - receivers[i] = new OutputReceiver(); - } } /** - * Adds an input to this ParDoOperation, coming from the given output of the given source. + * Returns a {@link DoFn} for processing inputs. */ - public void attachInput(ParDoOperation source, int outputNum) { - OutputReceiver fanOut = source.receivers[outputNum]; - fanOut.addOutput(this); - } + abstract DoFn getDoFn(); - /** - * Starts this Operation's execution. - * - *

Called after all successors consuming operations have been started. - */ - public void start() { + @Override + public void start(TaskInputOutputContext taskContext) { fnRunner = DoFnRunners.simpleRunner( options.getPipelineOptions(), - doFn, + getDoFn(), NullSideInputReader.empty(), - new ParDoOutputManager(), + createOutputManager(), mainOutputTag, sideOutputTags, null, windowingStrategy); fnRunner.startBundle(); - for (OutputReceiver receiver : receivers) { - if (receiver == null) { - continue; - } - for (ParDoOperation parDo : receiver.getReceiverParDos()) { - parDo.start(); - } - } + super.start(taskContext); } /** * Processes the element. */ + @Override public void process(Object elem) { LOG.info("elem: {}.", elem); fnRunner.processElement((WindowedValue) elem); } - /** - * Finishes this Operation's execution. - * - *

Called after all predecessors producing operations have been finished. - */ + @Override public void finish() { - for (OutputReceiver receiver : receivers) { - if (receiver == null) { - continue; - } - for (ParDoOperation parDo : receiver.getReceiverParDos()) { - parDo.finish(); - } - } + super.finish(); fnRunner.finishBundle(); } + protected DoFnRunners.OutputManager createOutputManager() { + return new ParDoOutputManager(); + } + private class ParDoOutputManager implements DoFnRunners.OutputManager { @Nullable private OutputReceiver getReceiverOrNull(TupleTag tag) { + List receivers = getOutputReceivers(); if (tag.equals(mainOutputTag)) { - return receivers[0]; + return receivers.get(0); } else if (sideOutputTags.contains(tag)) { - return receivers[sideOutputTags.indexOf(tag) + 1]; + return receivers.get(sideOutputTags.indexOf(tag) + 1); } else { return null; } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java new file mode 100644 index 000000000000..97201d0e436a --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java @@ -0,0 +1,52 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Throwables; +import java.io.ByteArrayOutputStream; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; + +/** + * Created by peihe on 26/07/2017. + */ +public class WriteOperation extends Operation { + + private final Coder keyCoder; + private final Coder nullableValueCoder; + + private transient TaskInputOutputContext taskContext; + + public WriteOperation(Coder coder) { + super(0); + KvCoder kvCoder = (KvCoder) checkNotNull(coder, "coder"); + this.keyCoder = kvCoder.getKeyCoder(); + this.nullableValueCoder = NullableCoder.of(kvCoder.getValueCoder()); + } + + @Override + public void start(TaskInputOutputContext taskContext) { + this.taskContext = checkNotNull(taskContext, "taskContext"); + } + + @Override + public void process(Object elem) { + WindowedValue> windowedElem = (WindowedValue>) elem; + try { + ByteArrayOutputStream keyStream = new ByteArrayOutputStream(); + keyCoder.encode(windowedElem.getValue().getKey(), keyStream); + + ByteArrayOutputStream valueStream = new ByteArrayOutputStream(); + nullableValueCoder.encode(windowedElem.getValue().getValue(), valueStream); + taskContext.write(new BytesWritable(keyStream.toByteArray()), valueStream.toByteArray()); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } +} diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java index f5eee28f9bb2..5fa499ab68ee 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java @@ -4,19 +4,12 @@ import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; import org.apache.log4j.BasicConfigurator; import org.junit.Test; import org.junit.runner.RunWith; From d09fb42773daccac01e9e680c52e1666fd03cfe1 Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 27 Jul 2017 15:01:22 +0800 Subject: [PATCH 08/34] core-java: InMemoryTimerInternals expose getTimers() for timer firings in mr-runner. --- .../org/apache/beam/runners/core/InMemoryTimerInternals.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java index c29ea199ae9e..c1d42d6877ce 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java @@ -23,6 +23,7 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.HashBasedTable; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Table; import java.util.NavigableSet; import java.util.TreeSet; @@ -65,6 +66,10 @@ public Instant currentOutputWatermarkTime() { return outputWatermarkTime; } + public Iterable getTimers(TimeDomain domain) { + return ImmutableList.copyOf(timersForDomain(domain)); + } + /** * Returns when the next timer in the given time domain will fire, or {@code null} * if there are no timers scheduled in that time domain. From c6a3a18d2c71c8f523deb54b323f26408c7de207 Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 27 Jul 2017 10:52:32 +0800 Subject: [PATCH 09/34] mr-runner: support reduce side ParDos and WordCount. --- runners/map-reduce/pom.xml | 2 +- .../mapreduce/translation/BeamMapper.java | 12 +- .../mapreduce/translation/BeamReducer.java | 56 ++++-- .../runners/mapreduce/translation/Graph.java | 4 +- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 5 + .../mapreduce/translation/JobPrototype.java | 164 +++++++++++++----- .../mapreduce/translation/Operation.java | 8 +- .../mapreduce/translation/OutputReceiver.java | 3 +- ...eifyTimestampAndWindowsParDoOperation.java | 46 +++++ .../translation/WindowAssignOperation.java | 75 ++++++++ .../mapreduce/translation/WriteOperation.java | 13 +- .../beam/runners/mapreduce/WordCountTest.java | 13 +- 12 files changed, 318 insertions(+), 83 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml index d18eee84d5be..226c5c02bbcd 100644 --- a/runners/map-reduce/pom.xml +++ b/runners/map-reduce/pom.xml @@ -20,7 +20,7 @@ org.apache.beam beam-runners-parent - 2.1.0-SNAPSHOT + 2.2.0-SNAPSHOT ../pom.xml diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index 11ecc8dee099..b5e4edcf7e1e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -16,7 +16,7 @@ public class BeamMapper public static final String BEAM_PAR_DO_OPERATION_MAPPER = "beam-par-do-op-mapper"; - private ParDoOperation parDoOperation; + private Operation operation; @Override protected void setup( @@ -24,9 +24,9 @@ protected void setup( String serializedParDo = checkNotNull( context.getConfiguration().get(BEAM_PAR_DO_OPERATION_MAPPER), BEAM_PAR_DO_OPERATION_MAPPER); - parDoOperation = (ParDoOperation) SerializableUtils.deserializeFromByteArray( - Base64.decodeBase64(serializedParDo), "ParDoOperation"); - parDoOperation.start((TaskInputOutputContext) context); + operation = (Operation) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(serializedParDo), "Operation"); + operation.start((TaskInputOutputContext) context); } @Override @@ -34,12 +34,12 @@ protected void map( Object key, WindowedValue value, Mapper, Object, WindowedValue>.Context context) { - parDoOperation.process(value); + operation.process(value); } @Override protected void cleanup( Mapper, Object, WindowedValue>.Context context) { - parDoOperation.finish(); + operation.finish(); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java index 8eb79382b5bc..9b8bd8244c8d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java @@ -5,14 +5,17 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; +import com.google.common.collect.Lists; import java.io.ByteArrayInputStream; import java.io.IOException; -import org.apache.beam.sdk.coders.BigEndianLongCoder; -import org.apache.beam.sdk.coders.NullableCoder; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.TaskInputOutputContext; @@ -20,49 +23,64 @@ * Created by peihe on 25/07/2017. */ public class BeamReducer - extends Reducer> { + extends Reducer> { + public static final String BEAM_REDUCER_KV_CODER = "beam-reducer-kv-coder"; public static final String BEAM_PAR_DO_OPERATION_REDUCER = "beam-par-do-op-reducer"; - private ParDoOperation parDoOperation; + private Coder keyCoder; + private Coder valueCoder; + private Operation operation; @Override protected void setup( - Reducer>.Context context) { + Reducer>.Context context) { + String serializedValueCoder = checkNotNull( + context.getConfiguration().get(BEAM_REDUCER_KV_CODER), + BEAM_REDUCER_KV_CODER); String serializedParDo = checkNotNull( context.getConfiguration().get(BEAM_PAR_DO_OPERATION_REDUCER), BEAM_PAR_DO_OPERATION_REDUCER); - parDoOperation = (ParDoOperation) SerializableUtils.deserializeFromByteArray( - Base64.decodeBase64(serializedParDo), "ParDoOperation"); - parDoOperation.start((TaskInputOutputContext) context); + KvCoder kvCoder = (KvCoder) SerializableUtils + .deserializeFromByteArray(Base64.decodeBase64(serializedValueCoder), "Coder"); + keyCoder = kvCoder.getKeyCoder(); + valueCoder = kvCoder.getValueCoder(); + operation = (Operation) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(serializedParDo), "Operation"); + operation.start((TaskInputOutputContext) context); } @Override protected void reduce( - Object key, + BytesWritable key, Iterable values, - Reducer>.Context context) { - Iterable decodedValues = FluentIterable.from(values) + Reducer>.Context context) { + List decodedValues = Lists.newArrayList(FluentIterable.from(values) .transform(new Function() { @Override public Object apply(byte[] input) { ByteArrayInputStream inStream = new ByteArrayInputStream(input); try { - // TODO: setup coders. - return NullableCoder.of(BigEndianLongCoder.of()).decode(inStream); + return valueCoder.decode(inStream); } catch (IOException e) { Throwables.throwIfUnchecked(e); throw new RuntimeException(e); } - } - }); - parDoOperation.process( - WindowedValue.valueInGlobalWindow(KV.of(key, decodedValues))); + }})); + + try { + operation.process( + WindowedValue.valueInGlobalWindow( + KV.of(keyCoder.decode(new ByteArrayInputStream(key.getBytes())), decodedValues))); + } catch (IOException e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } } @Override protected void cleanup( - Reducer>.Context context) { - parDoOperation.finish(); + Reducer>.Context context) { + operation.finish(); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index 867d1af34745..e3604190a538 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -18,6 +18,7 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.commons.lang.builder.ReflectionToStringBuilder; @@ -127,7 +128,8 @@ public void addOutgoing(Edge edge) { public void accept(GraphVisitor visitor) { PTransform transform = step.getTransform(); - if (transform instanceof ParDo.SingleOutput || transform instanceof ParDo.MultiOutput) { + if (transform instanceof ParDo.SingleOutput || transform instanceof ParDo.MultiOutput + || transform instanceof Window.Assign) { visitor.visitParDo(this); } else if (transform instanceof GroupByKey) { visitor.visitGroupByKey(this); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java index 0b8a8761c83c..8ee616d7b7ea 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -15,6 +15,7 @@ import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -90,6 +91,10 @@ public void processElement(ProcessContext c) throws Exception { // Finally, advance the processing time to infinity to fire any timers. timerInternals.advanceProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE); + runner.onTimers(timerInternals.getTimers(TimeDomain.EVENT_TIME)); + runner.onTimers(timerInternals.getTimers(TimeDomain.PROCESSING_TIME)); + runner.onTimers(timerInternals.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME)); + runner.persist(); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 34266f4f8bcf..576c6bf56855 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -12,14 +12,20 @@ import java.util.ArrayList; import java.util.List; import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BytesWritable; @@ -69,72 +75,150 @@ public Job build(Class jarClass, Configuration conf) throws IOException { // Setup DoFns in BeamMapper. // TODO: support more than one in-path. Graph.NodePath inPath = Iterables.getOnlyElement(inEdge.getPaths()); - List parDos = new ArrayList<>(); - parDos.addAll(FluentIterable.from(inPath.steps()) - .filter(new Predicate() { - @Override - public boolean apply(Graph.Step input) { - PTransform transform = input.getTransform(); - return transform instanceof ParDo.SingleOutput - || transform instanceof ParDo.MultiOutput; - }}) - .toList()); + + Operation mapperParDoRoot = chainParDosInPath(inPath); + Operation mapperParDoTail = getTailOperation(mapperParDoRoot); Graph.Step vertexStep = vertex.getStep(); if (vertexStep.getTransform() instanceof ParDo.SingleOutput - || vertexStep.getTransform() instanceof ParDo.MultiOutput) { - parDos.add(vertexStep); - } - - ParDoOperation root = null; - ParDoOperation prev = null; - for (Graph.Step step : parDos) { - ParDoOperation current = new NormalParDoOperation( - getDoFn(step.getTransform()), + || vertexStep.getTransform() instanceof ParDo.MultiOutput + || vertexStep.getTransform() instanceof Window.Assign) { + // TODO: add a TailVertex type to simplify the translation. + Operation vertexParDo = translateToOperation(vertexStep); + Operation mapperWrite = new WriteOperation( + getKeyCoder(inEdge.getCoder()), + getReifyValueCoder(inEdge.getCoder(), vertexStep.getWindowingStrategy())); + mapperParDoTail.attachOutput(vertexParDo, 0); + vertexParDo.attachOutput(mapperWrite, 0); + } else if (vertexStep.getTransform() instanceof GroupByKey) { + Operation reifyOperation = new ReifyTimestampAndWindowsParDoOperation( PipelineOptionsFactory.create(), - (TupleTag) step.getOutputs().iterator().next(), + new TupleTag<>(), ImmutableList.>of(), - step.getWindowingStrategy()); - if (root == null) { - root = current; - } else { - // TODO: set a proper outputNum for ParDo.MultiOutput instead of zero. - current.attachInput(prev, 0); - } - prev = current; + vertexStep.getWindowingStrategy()); + Operation mapperWrite = new WriteOperation( + getKeyCoder(inEdge.getCoder()), + getReifyValueCoder(inEdge.getCoder(), vertexStep.getWindowingStrategy())); + mapperParDoTail.attachOutput(reifyOperation, 0); + reifyOperation.attachOutput(mapperWrite, 0); + } else { + throw new UnsupportedOperationException("Transform: " + vertexStep.getTransform()); } - // TODO: get coders from pipeline. - WriteOperation writeOperation = new WriteOperation(inEdge.getCoder()); - writeOperation.attachInput(prev, 0); + job.setMapOutputKeyClass(BytesWritable.class); + job.setMapOutputValueClass(byte[].class); conf.set( BeamMapper.BEAM_PAR_DO_OPERATION_MAPPER, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(root))); + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(mapperParDoRoot))); job.setMapperClass(BeamMapper.class); if (vertexStep.getTransform() instanceof GroupByKey) { // Setup BeamReducer - ParDoOperation operation = new GroupAlsoByWindowsParDoOperation( + Operation gabwOperation = new GroupAlsoByWindowsParDoOperation( PipelineOptionsFactory.create(), (TupleTag) vertexStep.getOutputs().iterator().next(), ImmutableList.>of(), vertexStep.getWindowingStrategy(), inEdge.getCoder()); - // TODO: handle the map output key type. - job.setMapOutputKeyClass(BytesWritable.class); - job.setMapOutputValueClass(byte[].class); + Graph.Edge outEdge = Iterables.getOnlyElement(vertex.getOutgoing()); + Graph.NodePath outPath = Iterables.getOnlyElement(outEdge.getPaths()); + Operation reducerParDoRoot = chainParDosInPath(outPath); + Operation reducerParDoTail = getTailOperation(reducerParDoRoot); + + Operation reducerTailParDo = translateToOperation(outEdge.getTail().getStep()); + if (reducerParDoRoot == null) { + gabwOperation.attachOutput(reducerTailParDo, 0); + } else { + gabwOperation.attachOutput(reducerParDoRoot, 0); + reducerParDoTail.attachOutput(reducerTailParDo, 0); + } + conf.set( + BeamReducer.BEAM_REDUCER_KV_CODER, + Base64.encodeBase64String(SerializableUtils.serializeToByteArray( + KvCoder.of( + getKeyCoder(inEdge.getCoder()), + getReifyValueCoder(inEdge.getCoder(), vertexStep.getWindowingStrategy()))))); conf.set( BeamReducer.BEAM_PAR_DO_OPERATION_REDUCER, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(operation))); + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(gabwOperation))); job.setReducerClass(BeamReducer.class); } job.setOutputFormatClass(NullOutputFormat.class); return job; } - private DoFn getDoFn(PTransform transform) { + private Coder getKeyCoder(Coder coder) { + KvCoder kvCoder = (KvCoder) checkNotNull(coder, "coder"); + return kvCoder.getKeyCoder(); + } + + private Coder getReifyValueCoder( + Coder coder, WindowingStrategy windowingStrategy) { + KvCoder kvCoder = (KvCoder) checkNotNull(coder, "coder"); + return (Coder) WindowedValue.getFullCoder( + kvCoder.getValueCoder(), windowingStrategy.getWindowFn().windowCoder()); + } + + private Operation getTailOperation(@Nullable Operation operation) { + if (operation == null) { + return null; + } + if (operation.getOutputReceivers().isEmpty()) { + return operation; + } + OutputReceiver receiver = Iterables.getOnlyElement(operation.getOutputReceivers()); + if (receiver.getReceivingOperations().isEmpty()) { + return operation; + } + return getTailOperation(Iterables.getOnlyElement(receiver.getReceivingOperations())); + } + + private Operation chainParDosInPath(Graph.NodePath path) { + List parDos = new ArrayList<>(); + // TODO: we should not need this filter. + parDos.addAll(FluentIterable.from(path.steps()) + .filter(new Predicate() { + @Override + public boolean apply(Graph.Step input) { + PTransform transform = input.getTransform(); + return !(transform instanceof Read.Bounded); + }}) + .toList()); + + Operation root = null; + Operation prev = null; + for (Graph.Step step : parDos) { + Operation current = translateToOperation(step); + if (prev == null) { + root = current; + } else { + // TODO: set a proper outputNum for ParDo.MultiOutput instead of zero. + prev.attachOutput(current, 0); + } + prev = current; + } + return root; + } + + private Operation translateToOperation(Graph.Step parDoStep) { + PTransform transform = parDoStep.getTransform(); + DoFn doFn; if (transform instanceof ParDo.SingleOutput) { - return ((ParDo.SingleOutput) transform).getFn(); + return new NormalParDoOperation( + ((ParDo.SingleOutput) transform).getFn(), + PipelineOptionsFactory.create(), + (TupleTag) parDoStep.getOutputs().iterator().next(), + ImmutableList.>of(), + parDoStep.getWindowingStrategy()); + } else if (transform instanceof ParDo.MultiOutput) { + return new NormalParDoOperation( + ((ParDo.MultiOutput) transform).getFn(), + PipelineOptionsFactory.create(), + (TupleTag) parDoStep.getOutputs().iterator().next(), + ImmutableList.>of(), + parDoStep.getWindowingStrategy()); + } else if (transform instanceof Window.Assign) { + return new WindowAssignOperation<>(1, parDoStep.getWindowingStrategy().getWindowFn()); } else { - return ((ParDo.MultiOutput) transform).getFn(); + throw new UnsupportedOperationException("Transform: " + transform); } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java index 5700e899f5f2..695190949d93 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java @@ -60,10 +60,10 @@ public List getOutputReceivers() { } /** - * Adds an input to this ParDoOperation, coming from the given output of the given source. + * Adds an output to this Operation. */ - public void attachInput(Operation source, int outputNum) { - OutputReceiver fanOut = source.receivers[outputNum]; - fanOut.addOutput(this); + public void attachOutput(Operation output, int outputNum) { + OutputReceiver fanOut = receivers[outputNum]; + fanOut.addOutput(output); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java index 334767239501..6aeefd2af228 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java @@ -23,8 +23,7 @@ import java.util.List; /** - * OutputReceiver that forwards each input it receives to each of a list of down stream - * ParDoOperations. + * OutputReceiver that forwards each input it receives to each of a list of down stream operations. */ public class OutputReceiver implements Serializable { private final List receivingOperations = new ArrayList<>(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java new file mode 100644 index 000000000000..ec954bb52d3f --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java @@ -0,0 +1,46 @@ +package org.apache.beam.runners.mapreduce.translation; + +import java.util.List; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * Created by peihe on 27/07/2017. + */ +public class ReifyTimestampAndWindowsParDoOperation extends ParDoOperation { + + public ReifyTimestampAndWindowsParDoOperation( + PipelineOptions options, + TupleTag mainOutputTag, + List> sideOutputTags, + WindowingStrategy windowingStrategy) { + super(options, mainOutputTag, sideOutputTags, windowingStrategy); + } + + @Override + DoFn getDoFn() { + return (DoFn) new ReifyTimestampAndWindowsDoFn<>(); + } + + public class ReifyTimestampAndWindowsDoFn + extends DoFn, KV>> { + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + KV kv = c.element(); + K key = kv.getKey(); + V value = kv.getValue(); + c.output(KV.of( + key, + WindowedValue.of( + value, + c.timestamp(), + window, + c.pane()))); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java new file mode 100644 index 000000000000..144ef16c6046 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java @@ -0,0 +1,75 @@ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import java.util.Collection; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.joda.time.Instant; + +/** + * Created by peihe on 27/07/2017. + */ +public class WindowAssignOperation extends Operation { + private final WindowFn windowFn; + + public WindowAssignOperation(int numOutputs, WindowFn windowFn) { + super(numOutputs); + this.windowFn = checkNotNull(windowFn, "windowFn"); + } + + @Override + public void process(Object elem) { + WindowedValue windowedValue = (WindowedValue) elem; + try { + Collection windows = windowFn.assignWindows(new AssignContextInternal<>(windowFn, windowedValue)); + for (W window : windows) { + OutputReceiver receiver = Iterables.getOnlyElement(getOutputReceivers()); + receiver.process(WindowedValue.of( + windowedValue.getValue(), + windowedValue.getTimestamp(), + window, + windowedValue.getPane())); + } + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + + private class AssignContextInternal + extends WindowFn.AssignContext { + private final WindowedValue value; + + AssignContextInternal(WindowFn fn, WindowedValue value) { + fn.super(); + checkArgument( + Iterables.size(value.getWindows()) == 1, + String.format( + "%s passed to window assignment must be in a single window, but it was in %s: %s", + WindowedValue.class.getSimpleName(), + Iterables.size(value.getWindows()), + value.getWindows())); + this.value = value; + } + + @Override + public InputT element() { + return value.getValue(); + } + + @Override + public Instant timestamp() { + return value.getTimestamp(); + } + + @Override + public BoundedWindow window() { + return Iterables.getOnlyElement(value.getWindows()); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java index 97201d0e436a..0585032a8a76 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java @@ -7,8 +7,10 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapreduce.TaskInputOutputContext; @@ -18,15 +20,14 @@ public class WriteOperation extends Operation { private final Coder keyCoder; - private final Coder nullableValueCoder; + private final Coder valueCoder; private transient TaskInputOutputContext taskContext; - public WriteOperation(Coder coder) { + public WriteOperation(Coder keyCoder, Coder valueCoder) { super(0); - KvCoder kvCoder = (KvCoder) checkNotNull(coder, "coder"); - this.keyCoder = kvCoder.getKeyCoder(); - this.nullableValueCoder = NullableCoder.of(kvCoder.getValueCoder()); + this.keyCoder = checkNotNull(keyCoder, "keyCoder"); + this.valueCoder = checkNotNull(valueCoder, "valueCoder"); } @Override @@ -42,7 +43,7 @@ public void process(Object elem) { keyCoder.encode(windowedElem.getValue().getKey(), keyStream); ByteArrayOutputStream valueStream = new ByteArrayOutputStream(); - nullableValueCoder.encode(windowedElem.getValue().getValue(), valueStream); + valueCoder.encode(windowedElem.getValue().getValue(), valueStream); taskContext.write(new BytesWritable(keyStream.toByteArray()), valueStream.toByteArray()); } catch (Exception e) { Throwables.throwIfUnchecked(e); diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java index 5fa499ab68ee..a548ba7fea17 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java @@ -7,10 +7,14 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.log4j.BasicConfigurator; +import org.joda.time.Duration; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -64,17 +68,18 @@ public void testWordCount() { String input = "/Users/peihe/github/beam/LICENSE"; String output = "./output"; MapReducePipelineOptions options = PipelineOptionsFactory.as(MapReducePipelineOptions.class); - options.setJarClass(this.getClass()); + //options.setJarClass(this.getClass()); options.setRunner(MapReduceRunner.class); Pipeline p = Pipeline.create(options); // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the // static FormatAsTextFn() to the ParDo transform. p.apply("ReadLines", TextIO.read().from(input)) + .apply(Window.into(SlidingWindows.of(Duration.millis(100)))) .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); -// .apply(MapElements.via(new FormatAsTextFn())) -// .apply("WriteCounts", TextIO.write().to(output)); + .apply(Count.perElement()) + .apply(MapElements.via(new FormatAsTextFn())); + //.apply("WriteCounts", TextIO.write().to(output)); p.run(); } From ee1cce92d620c78b0243ee0b9c7c0f6ae232b0cb Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 27 Jul 2017 15:05:06 +0800 Subject: [PATCH 10/34] mr-runner: add JarClassInstanceFactory to run ValidatesRunner tests. --- runners/map-reduce/pom.xml | 2 +- .../mapreduce/MapReducePipelineOptions.java | 51 +++++++++++++++++++ runners/pom.xml | 1 + 3 files changed, 53 insertions(+), 1 deletion(-) diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml index 226c5c02bbcd..06e5227380d7 100644 --- a/runners/map-reduce/pom.xml +++ b/runners/map-reduce/pom.xml @@ -39,7 +39,7 @@ local-validates-runner-tests - false + true diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java index ce8f937e63fb..7fe66baabc4c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -1,14 +1,65 @@ package org.apache.beam.runners.mapreduce; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterators; +import java.util.Iterator; +import java.util.Set; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; /** * {@link PipelineOptions} for {@link MapReduceRunner}. */ public interface MapReducePipelineOptions extends PipelineOptions { + /** Classes that are used as the boundary in the stack trace to find the callers class name. */ + Set PIPELINE_OPTIONS_FACTORY_CLASSES = ImmutableSet.of( + PipelineOptionsFactory.class.getName(), + PipelineOptionsFactory.Builder.class.getName(), + "org.apache.beam.sdk.options.ProxyInvocationHandler"); + + @Description("The jar class of the user Beam program.") + @Default.InstanceFactory(JarClassInstanceFactory.class) Class getJarClass(); void setJarClass(Class jarClass); + + class JarClassInstanceFactory implements DefaultValueFactory> { + @Override + public Class create(PipelineOptions options) { + return findCallersClassName(options); + } + + /** + * Returns the simple name of the calling class using the current threads stack. + */ + private static Class findCallersClassName(PipelineOptions options) { + Iterator elements = + Iterators.forArray(Thread.currentThread().getStackTrace()); + // First find the PipelineOptionsFactory/Builder class in the stack trace. + while (elements.hasNext()) { + StackTraceElement next = elements.next(); + if (PIPELINE_OPTIONS_FACTORY_CLASSES.contains(next.getClassName())) { + break; + } + } + // Then find the first instance after that is not the PipelineOptionsFactory/Builder class. + while (elements.hasNext()) { + StackTraceElement next = elements.next(); + if (!PIPELINE_OPTIONS_FACTORY_CLASSES.contains(next.getClassName()) + && !next.getClassName().contains("com.sun.proxy.$Proxy") + && !next.getClassName().equals(options.getRunner().getName())) { + try { + return Class.forName(next.getClassName()); + } catch (ClassNotFoundException e) { + break; + } + } + } + return null; + } + } } diff --git a/runners/pom.xml b/runners/pom.xml index b00ba9ccc56a..4cba41a17d4f 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -35,6 +35,7 @@ core-construction-java core-java + map-reduce direct-java flink google-cloud-dataflow-java From 16e63205bee3ade711ebffc3c74e18aec6d50c01 Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 28 Jul 2017 16:31:41 +0800 Subject: [PATCH 11/34] mr-runner: refactors and creates Graph data structures to handle general Beam pipelines. --- .../mapreduce/MapReducePipelineOptions.java | 17 + .../runners/mapreduce/MapReduceRegistrar.java | 17 + .../runners/mapreduce/MapReduceRunner.java | 48 ++- .../runners/mapreduce/MapReduceWordCount.java | 218 ---------- .../beam/runners/mapreduce/package-info.java | 21 + .../translation/BeamInputFormat.java | 21 +- .../mapreduce/translation/BeamMapper.java | 19 +- .../mapreduce/translation/BeamReducer.java | 20 +- .../translation/FlattenOperation.java | 37 ++ .../translation/FlattenTranslator.java | 37 ++ .../runners/mapreduce/translation/Graph.java | 400 ++++++------------ .../mapreduce/translation/GraphConverter.java | 108 +++-- .../mapreduce/translation/GraphPlanner.java | 142 ++----- .../mapreduce/translation/GraphVisitor.java | 11 - .../runners/mapreduce/translation/Graphs.java | 188 ++++++++ .../GroupAlsoByWindowsParDoOperation.java | 24 +- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 17 + .../translation/GroupByKeyOperation.java | 54 +++ .../translation/GroupByKeyTranslator.java | 46 ++ .../mapreduce/translation/JobPrototype.java | 257 +++++------ .../translation/NormalParDoOperation.java | 12 +- .../mapreduce/translation/Operation.java | 30 +- .../mapreduce/translation/OutputReceiver.java | 3 +- .../mapreduce/translation/ParDoOperation.java | 14 +- .../translation/ParDoTranslator.java | 46 ++ .../translation/ReadBoundedTranslator.java | 37 ++ .../mapreduce/translation/ReadOperation.java | 45 ++ ...eifyTimestampAndWindowsParDoOperation.java | 24 +- .../translation/TransformTranslator.java | 48 +++ .../translation/TranslationContext.java | 128 ++++++ .../translation/TranslatorRegistry.java | 58 +++ .../mapreduce/translation/ViewOperation.java | 59 +++ .../mapreduce/translation/ViewTranslator.java | 42 ++ .../translation/WindowAssignOperation.java | 35 +- .../translation/WindowAssignTranslator.java | 38 ++ .../mapreduce/translation/WriteOperation.java | 33 +- .../beam/runners/mapreduce/WordCountTest.java | 25 +- .../translation/GraphConverterTest.java | 33 +- .../translation/GraphPlannerTest.java | 37 +- 39 files changed, 1568 insertions(+), 881 deletions(-) delete mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java delete mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TransformTranslator.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java index 7fe66baabc4c..73c7d4711ff9 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce; import com.google.common.collect.ImmutableSet; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java index eb960b81b120..c8b0eeafe30e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce; import com.google.auto.service.AutoService; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index 11ac9a73af73..b6a82d1bc237 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -1,23 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce; import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.base.Throwables; -import org.apache.beam.runners.mapreduce.translation.Graph; +import org.apache.beam.runners.mapreduce.translation.Graphs; import org.apache.beam.runners.mapreduce.translation.GraphConverter; import org.apache.beam.runners.mapreduce.translation.GraphPlanner; import org.apache.beam.runners.mapreduce.translation.JobPrototype; +import org.apache.beam.runners.mapreduce.translation.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineRunner; -import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Job; /** - * {@link PipelineRunner} for crunch. + * {@link PipelineRunner} for MapReduce. */ public class MapReduceRunner extends PipelineRunner { @@ -39,22 +55,20 @@ public static MapReduceRunner fromOptions(PipelineOptions options) { @Override public PipelineResult run(Pipeline pipeline) { - GraphConverter graphConverter = new GraphConverter(); + TranslationContext context = new TranslationContext(options); + GraphConverter graphConverter = new GraphConverter(context); pipeline.traverseTopologically(graphConverter); - Graph graph = graphConverter.getGraph(); - GraphPlanner planner = new GraphPlanner(); - Graph fusedGraph = planner.plan(graph); - for (Graph.Vertex vertex : fusedGraph.getAllVertices()) { - if (vertex.getStep().getTransform() instanceof GroupByKey) { - JobPrototype jobPrototype = JobPrototype.create(1, vertex); - try { - Job job = jobPrototype.build(options.getJarClass(), new Configuration()); - job.waitForCompletion(true); - } catch (Exception e) { - Throwables.throwIfUnchecked(e); - } + Graphs.FusedGraph fusedGraph = planner.plan(context.getInitGraph()); + int stageId = 0; + for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + JobPrototype jobPrototype = JobPrototype.create(stageId++, fusedStep, options); + try { + Job job = jobPrototype.build(options.getJarClass(), new Configuration()); + job.waitForCompletion(true); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); } } return null; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java deleted file mode 100644 index d0c7b7882b00..000000000000 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceWordCount.java +++ /dev/null @@ -1,218 +0,0 @@ -package org.apache.beam.runners.mapreduce; - -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.StringTokenizer; - -import javax.annotation.Nullable; -import org.apache.beam.runners.mapreduce.translation.BeamInputFormat; -import org.apache.beam.runners.mapreduce.translation.BeamMapper; -import org.apache.beam.sdk.coders.BigEndianIntegerCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.OffsetBasedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.values.KV; -import org.apache.commons.codec.binary.Base64; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.Mapper; -import org.apache.hadoop.mapreduce.Reducer; -import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; -import org.apache.log4j.BasicConfigurator; - -public class MapReduceWordCount { - - public static class CreateSource extends OffsetBasedSource { - private final List allElementsBytes; - private final long totalSize; - private final Coder coder; - - public static CreateSource fromIterable(Iterable elements, Coder elemCoder) - throws CoderException, IOException { - ImmutableList.Builder allElementsBytes = ImmutableList.builder(); - long totalSize = 0L; - for (T element : elements) { - byte[] bytes = CoderUtils.encodeToByteArray(elemCoder, element); - allElementsBytes.add(bytes); - totalSize += bytes.length; - } - return new CreateSource<>(allElementsBytes.build(), totalSize, elemCoder); - } - - /** - * Create a new source with the specified bytes. The new source owns the input element bytes, - * which must not be modified after this constructor is called. - */ - private CreateSource(List elementBytes, long totalSize, Coder coder) { - super(0, elementBytes.size(), 1); - this.allElementsBytes = ImmutableList.copyOf(elementBytes); - this.totalSize = totalSize; - this.coder = coder; - } - - @Override - public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { - return totalSize; - } - - @Override - public BoundedSource.BoundedReader createReader(PipelineOptions options) - throws IOException { - return new BytesReader<>(this); - } - - @Override - public void validate() {} - - @Override - public Coder getDefaultOutputCoder() { - return coder; - } - - @Override - public long getMaxEndOffset(PipelineOptions options) throws Exception { - return allElementsBytes.size(); - } - - @Override - public OffsetBasedSource createSourceForSubrange(long start, long end) { - List primaryElems = allElementsBytes.subList((int) start, (int) end); - long primarySizeEstimate = - (long) (totalSize * primaryElems.size() / (double) allElementsBytes.size()); - return new CreateSource<>(primaryElems, primarySizeEstimate, coder); - } - - @Override - public long getBytesPerOffset() { - if (allElementsBytes.size() == 0) { - return 1L; - } - return Math.max(1, totalSize / allElementsBytes.size()); - } - - private static class BytesReader extends OffsetBasedReader { - private int index; - /** - * Use an optional to distinguish between null next element (as Optional.absent()) and no next - * element (next is null). - */ - @Nullable - private Optional next; - - public BytesReader(CreateSource source) { - super(source); - index = -1; - } - - @Override - @Nullable - public T getCurrent() throws NoSuchElementException { - if (next == null) { - throw new NoSuchElementException(); - } - return next.orNull(); - } - - @Override - public void close() throws IOException {} - - @Override - protected long getCurrentOffset() { - return index; - } - - @Override - protected boolean startImpl() throws IOException { - return advanceImpl(); - } - - @Override - public synchronized CreateSource getCurrentSource() { - return (CreateSource) super.getCurrentSource(); - } - - @Override - protected boolean advanceImpl() throws IOException { - CreateSource source = getCurrentSource(); - if (index + 1 >= source.allElementsBytes.size()) { - next = null; - return false; - } - index++; - next = - Optional.fromNullable( - CoderUtils.decodeFromByteArray(source.coder, source.allElementsBytes.get(index))); - return true; - } - } - } - - public static class TokenizerMapper - extends Mapper{ - - private final static IntWritable one = new IntWritable(1); - private Text word = new Text(); - - public void map(Object key, Text value, Context context - ) throws IOException, InterruptedException { - StringTokenizer itr = new StringTokenizer(value.toString()); - while (itr.hasMoreTokens()) { - word.set(itr.nextToken()); - context.write(word, one); - } - } - } - - public static class IntSumReducer - extends Reducer { - private IntWritable result = new IntWritable(); - - public void reduce(Text key, Iterable values, Context context) - throws IOException, InterruptedException { - int sum = 0; - for (IntWritable val : values) { - sum += val.get(); - } - result.set(sum); - context.write(key, result); - } - } - - public static void main(String[] args) throws Exception { - BasicConfigurator.configure(); - - Configuration conf = new Configuration(); - - BoundedSource> source = CreateSource.fromIterable( - ImmutableList.of(KV.of("k1", 10), KV.of("k2", 2)), - KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())); - - conf.set( - BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(source))); - - Job job = Job.getInstance(conf, "word count"); - job.setJarByClass(MapReduceWordCount.class); - job.setInputFormatClass(BeamInputFormat.class); - job.setMapperClass(BeamMapper.class); - //job.setMapperClass(TokenizerMapper.class); - //job.setCombinerClass(IntSumReducer.class); - //job.setReducerClass(IntSumReducer.class); - //job.setOutputKeyClass(Text.class); - //job.setOutputValueClass(IntWritable.class); - //FileInputFormat.addInputPath(job, new Path(args[0])); - job.setOutputFormatClass(NullOutputFormat.class); - System.exit(job.waitForCompletion(true) ? 0 : 1); - } -} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java new file mode 100644 index 000000000000..d5114058a230 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * MapReduce runner implementation. + */ +package org.apache.beam.runners.mapreduce; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java index 0cfb14b973a9..8a27a85a1cca 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; @@ -14,8 +31,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.TimestampedValue; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; @@ -66,6 +81,7 @@ public InputSplit apply(BoundedSource source) { @Override public RecordReader createRecordReader( InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { + // TODO: it should initiates from InputSplit. source = (BoundedSource) SerializableUtils.deserializeFromByteArray( Base64.decodeBase64(context.getConfiguration().get(BEAM_SERIALIZED_BOUNDED_SOURCE)), ""); @@ -121,6 +137,7 @@ public void initialize( @Override public boolean nextKeyValue() throws IOException, InterruptedException { if (!started) { + started = true; return reader.start(); } else { return reader.advance(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index b5e4edcf7e1e..bc52967751a5 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; @@ -9,7 +26,7 @@ import org.apache.hadoop.mapreduce.TaskInputOutputContext; /** - * Created by peihe on 21/07/2017. + * Adapter for executing Beam transforms in {@link Mapper}. */ public class BeamMapper extends Mapper, Object, WindowedValue> { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java index 9b8bd8244c8d..3490b3b1b95c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; @@ -19,8 +36,9 @@ import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.TaskInputOutputContext; + /** - * Created by peihe on 25/07/2017. + * Adapter for executing Beam transforms in {@link Reducer}. */ public class BeamReducer extends Reducer> { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java new file mode 100644 index 000000000000..191b34629e31 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.util.WindowedValue; + +/** + * Flatten operation. + */ +public class FlattenOperation extends Operation { + + public FlattenOperation() { + super(1); + } + + @Override + public void process(WindowedValue elem) { + for (OutputReceiver receiver : getOutputReceivers()) { + receiver.process(elem); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java new file mode 100644 index 000000000000..8860caf19d75 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.transforms.Flatten; + +/** + * Translates a {@link Flatten} to a {@link FlattenOperation}. + */ +public class FlattenTranslator extends TransformTranslator.Default> { + @Override + public void translateNode(Flatten.PCollections transform, TranslationContext context) { + TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); + + Operation operation = new FlattenOperation(); + context.addInitStep(Graphs.Step.of( + userGraphContext.getStepName(), + operation, + userGraphContext.getInputTags(), + userGraphContext.getOutputTags())); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index e3604190a538..b6900ccbac19 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -1,311 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - -import com.google.auto.value.AutoValue; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import java.util.LinkedList; +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Iterables; +import com.google.common.graph.ElementOrder; +import com.google.common.graph.GraphBuilder; +import com.google.common.graph.MutableGraph; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Set; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.commons.lang.builder.ReflectionToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; /** - * Created by peihe on 06/07/2017. + * Graph that represents a Beam DAG. */ -public class Graph { +public class Graph, TagT extends Graph.AbstractTag> { - private final Map vertices; - private final Map edges; - private final Set leafVertices; + private final MutableGraph graph; public Graph() { - this.vertices = Maps.newHashMap(); - this.edges = Maps.newHashMap(); - this.leafVertices = Sets.newHashSet(); + this.graph = GraphBuilder.directed() + .allowsSelfLoops(false) + .nodeOrder(ElementOrder.insertion()) + .build(); } - public Vertex addVertex(Step step) { - checkState(!vertices.containsKey(step)); - Vertex v = new Vertex(step); - vertices.put(step, v); - leafVertices.add(v); - return v; + /** + * Adds {@link StepT} to this {@link Graph}. + */ + public void addStep(StepT step) { + graph.addNode(step); + Set nodes = graph.nodes(); + for (TagT tag : step.getInputTags()) { + if (!nodes.contains(tag)) { + graph.addNode(tag); + } + graph.putEdge(tag, step); + } + for (TagT tag : step.getOutputTags()) { + if (!nodes.contains(tag)) { + graph.addNode(tag); + } + graph.putEdge(step, tag); + } } - public Edge addEdge(Vertex head, Vertex tail, Coder coder) { - HeadTail headTail = HeadTail.of(head, tail); - checkState(!edges.containsKey(headTail)); - Edge e = new Edge(headTail, coder); - edges.put(headTail, e); - head.addOutgoing(e); - tail.addIncoming(e); - leafVertices.remove(head); - return e; + public void removeStep(StepT step) { + graph.removeNode(step); } - public Vertex getVertex(Step step) { - return vertices.get(step); + public void removeTag(TagT tag) { + graph.removeNode(tag); } - public Edge getEdge(Vertex head, Vertex tail) { - return edges.get(HeadTail.of(head, tail)); + public void addEdge(TagT inTag, StepT step) { + graph.putEdge(inTag, step); } - public Iterable getAllVertices() { - return vertices.values(); + public void addEdge(StepT step, TagT outTag) { + graph.putEdge(step, outTag); } - public Iterable getAllEdges() { - return edges.values(); + public void removeEdge(TagT inTag, StepT step) { + graph.removeEdge(inTag, step); } - public Iterable getLeafVertices() { - return ImmutableList.copyOf(leafVertices); + public void removeEdge(StepT step, TagT outTag) { + graph.removeEdge(step, outTag); } - public void accept(GraphVisitor visitor) { - for (Vertex v : leafVertices) { - v.accept(visitor); - } + public List getSteps() { + return castToStepList(FluentIterable.from(graph.nodes()) + .filter(new Predicate() { + @Override + public boolean apply(Vertex input) { + return input instanceof AbstractStep; + }})) + .toList(); } - //TODO: add equals, hashCode, toString for following classses. - - public static class Vertex { - private final Step step; - private final Set incoming; - private final Set outgoing; - - public Vertex(Step step) { - this.step = checkNotNull(step, "step"); - this.incoming = Sets.newHashSet(); - this.outgoing = Sets.newHashSet(); - } - - public Step getStep() { - return step; - } - - public Set getIncoming() { - return incoming; - } - - public Set getOutgoing() { - return outgoing; - } - - public boolean isSource() { - PTransform transform = step.getTransform(); - return transform instanceof Read.Bounded || transform instanceof Read.Unbounded; - } - - public boolean isGroupByKey() { - return step.getTransform() instanceof GroupByKey; - } - - public void addIncoming(Edge edge) { - incoming.add(edge); - } - - public void addOutgoing(Edge edge) { - outgoing.add(edge); - } - - public void accept(GraphVisitor visitor) { - PTransform transform = step.getTransform(); - if (transform instanceof ParDo.SingleOutput || transform instanceof ParDo.MultiOutput - || transform instanceof Window.Assign) { - visitor.visitParDo(this); - } else if (transform instanceof GroupByKey) { - visitor.visitGroupByKey(this); - } else if (transform instanceof Read.Bounded) { - visitor.visitRead(this); - } else if (transform instanceof Flatten.PCollections - || transform instanceof Flatten.Iterables) { - visitor.visitFlatten(this); - } else { - throw new RuntimeException("Unexpected transform type: " + transform.getClass()); - } - } - - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } - if (obj instanceof Vertex) { - Vertex other = (Vertex) obj; - return step.equals(other.step); - } - return false; - } - - @Override - public int hashCode() { - return Objects.hash(this.getClass(), step); - } - - @Override - public String toString() { - return new ReflectionToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) - .setExcludeFieldNames(new String[] { "outgoing", "incoming" }).toString(); - } + public List getStartSteps() { + return castToStepList(FluentIterable.from(graph.nodes()) + .filter(new Predicate() { + @Override + public boolean apply(Vertex input) { + return input instanceof AbstractStep && graph.inDegree(input) == 0; + }})) + .toList(); } - public static class Edge { - private final HeadTail headTail; - private final Coder coder; - private final Set paths; - - public static Edge of(HeadTail headTail, Coder coder) { - return new Edge(headTail, coder); - } - - private Edge(HeadTail headTail, Coder coder) { - this.headTail = checkNotNull(headTail, "headTail"); - this.coder = checkNotNull(coder, "coder"); - this.paths = Sets.newHashSet(); - } - - public Vertex getHead() { - return headTail.getHead(); - } - - public Vertex getTail() { - return headTail.getTail(); - } - - public Coder getCoder() { - return coder; - } - - public Set getPaths() { - return paths; - } - - public void addPath(NodePath path) { - paths.add(checkNotNull(path, "path")); - } - - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } - if (obj instanceof Edge) { - Edge other = (Edge) obj; - return headTail.equals(other.headTail) - && paths.equals(other.paths) && coder.equals(other.coder); - } - return false; - } - - @Override - public int hashCode() { - return Objects.hash(headTail, paths, coder); - } - - @Override - public String toString() { - return ReflectionToStringBuilder.toString(this, ToStringStyle.SHORT_PREFIX_STYLE); - } + public List getInputTags() { + return castToTagList(FluentIterable.from(graph.nodes()) + .filter(new Predicate() { + @Override + public boolean apply(Vertex input) { + return input instanceof AbstractTag && graph.inDegree(input) == 0; + }})) + .toList(); } - public static class NodePath { - private final LinkedList path; - - public NodePath() { - this.path = new LinkedList<>(); - } - - public NodePath(NodePath nodePath) { - this.path = new LinkedList<>(nodePath.path); - } + public List getOutputTags() { + return castToTagList(FluentIterable.from(graph.nodes()) + .filter(new Predicate() { + @Override + public boolean apply(Vertex input) { + return input instanceof AbstractTag && graph.outDegree(input) == 0; + }})) + .toList(); + } - public void addFirst(Step step) { - path.addFirst(step); - } + public StepT getProducer(TagT tag) { + return (StepT) Iterables.getOnlyElement(graph.predecessors(tag)); + } - public void addLast(Step step) { - path.addLast(step); - } + public List getConsumers(TagT tag) { + return castToStepList(graph.successors(tag)).toList(); + } - public Iterable steps() { - return ImmutableList.copyOf(path); - } + private FluentIterable castToStepList(Iterable vertices) { + return FluentIterable.from(vertices) + .transform(new Function() { + @Override + public StepT apply(Vertex input) { + return (StepT) input; + }}); + } - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } - if (obj instanceof NodePath) { - NodePath other = (NodePath) obj; - return path.equals(other.path); - } - return false; - } + private FluentIterable castToTagList(Iterable vertices) { + return FluentIterable.from(vertices) + .transform(new Function() { + @Override + public TagT apply(Vertex input) { + return (TagT) input; + }}); + } - @Override - public int hashCode() { - return Objects.hash(this.getClass(), path.hashCode()); + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - for (Step step : path) { - sb.append(step.getFullName() + "|"); - } - if (path.size() > 0) { - sb.deleteCharAt(sb.length() - 1); - } - return sb.toString(); + if (obj instanceof Graph) { + Graph other = (Graph) obj; + return com.google.common.graph.Graphs.equivalent(this.graph, other.graph); } + return false; } - @AutoValue - public abstract static class Step { - abstract String getFullName(); - // TODO: remove public - public abstract PTransform getTransform(); - abstract WindowingStrategy getWindowingStrategy(); - abstract List> getInputs(); - abstract List> getOutputs(); + @Override + public int hashCode() { + return Objects.hash(this.getClass(), graph.nodes()); + } - public static Step of( - String fullName, - PTransform transform, - WindowingStrategy windowingStrategy, - List> inputs, - List> outputs) { - return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graph_Step( - fullName, transform, windowingStrategy, inputs, outputs); - } + /** + * Vertex interface of this Graph. + */ + interface Vertex { } - @AutoValue - public abstract static class HeadTail { - abstract Vertex getHead(); - abstract Vertex getTail(); + public abstract static class AbstractStep implements Vertex { + public abstract List getInputTags(); + public abstract List getOutputTags(); + } - public static HeadTail of(Vertex head, Vertex tail) { - return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graph_HeadTail(head, tail); - } + public abstract static class AbstractTag implements Vertex { } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java index e7e7598c1e56..1e818fa257a2 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -1,77 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; -import com.google.common.collect.ImmutableList; +import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.collect.Maps; import java.util.Map; import org.apache.beam.runners.mapreduce.MapReduceRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; /** * Pipeline translator for {@link MapReduceRunner}. */ public class GraphConverter extends Pipeline.PipelineVisitor.Defaults { + private final TranslationContext context; private final Map> pValueToTupleTag; - private final Map, Graph.Vertex> outputToProducer; - private final Graph graph; - public GraphConverter() { + public GraphConverter(TranslationContext context) { + this.context = checkNotNull(context, "context"); this.pValueToTupleTag = Maps.newHashMap(); - this.outputToProducer = Maps.newHashMap(); - this.graph = new Graph(); } @Override - public void visitPrimitiveTransform(TransformHierarchy.Node node) { - WindowingStrategy windowingStrategy = - getWindowingStrategy(node.getOutputs().values().iterator().next()); - Graph.Step step = Graph.Step.of( - node.getFullName(), - node.getTransform(), - windowingStrategy, - ImmutableList.copyOf(node.getInputs().keySet()), - ImmutableList.copyOf(node.getOutputs().keySet())); - Graph.Vertex v = graph.addVertex(step); + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { + // check if current composite transforms need to be translated. + // If not, all sub transforms will be translated in visitPrimitiveTransform. + PTransform transform = node.getTransform(); + if (transform != null) { + TransformTranslator translator = TranslatorRegistry.getTranslator(transform); - for (PValue pValue : node.getInputs().values()) { - TupleTag tag = pValueToTupleTag.get(pValue); - if (outputToProducer.containsKey(tag)) { - Graph.Vertex producer = outputToProducer.get(tag); - - PCollection pc = (PCollection) pValue; - graph.addEdge(producer, v, pc.getCoder()); + if (translator != null && applyCanTranslate(transform, node, translator)) { + applyTransform(transform, node, translator); + return CompositeBehavior.DO_NOT_ENTER_TRANSFORM; } } + return CompositeBehavior.ENTER_TRANSFORM; + } - for (Map.Entry, PValue> entry : node.getOutputs().entrySet()) { - pValueToTupleTag.put(entry.getValue(), entry.getKey()); - outputToProducer.put(entry.getKey(), v); + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + if (!node.isRootNode()) { + PTransform transform = node.getTransform(); + TransformTranslator translator = TranslatorRegistry.getTranslator(transform); + if (translator == null || !applyCanTranslate(transform, node, translator)) { + throw new UnsupportedOperationException( + "The transform " + transform + " is currently not supported."); + } + applyTransform(transform, node, translator); } } - private WindowingStrategy getWindowingStrategy(PValue pValue) { - if (pValue instanceof PCollection) { - return ((PCollection) pValue).getWindowingStrategy(); - } else if (pValue instanceof PCollectionList) { - return ((PCollectionList) pValue).get(0).getWindowingStrategy(); - } else if (pValue instanceof PCollectionTuple) { - return ((PCollectionTuple) pValue).getAll().values().iterator().next().getWindowingStrategy(); - } else if (pValue instanceof PCollectionView) { - return ((PCollectionView) pValue).getPCollection().getWindowingStrategy(); - } else { - throw new RuntimeException("Unexpected pValue type: " + pValue.getClass()); - } + private > void applyTransform( + PTransform transform, + TransformHierarchy.Node node, + TransformTranslator translator) { + @SuppressWarnings("unchecked") + T typedTransform = (T) transform; + @SuppressWarnings("unchecked") + TransformTranslator typedTranslator = (TransformTranslator) translator; + context.getUserGraphContext().setCurrentNode(node); + typedTranslator.translateNode(typedTransform, context); } - public Graph getGraph() { - return graph; + private > boolean applyCanTranslate( + PTransform transform, + TransformHierarchy.Node node, + TransformTranslator translator) { + @SuppressWarnings("unchecked") + T typedTransform = (T) transform; + @SuppressWarnings("unchecked") + TransformTranslator typedTranslator = (TransformTranslator) translator; + context.getUserGraphContext().setCurrentNode(node); + return typedTranslator.canTranslate(typedTransform, context); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 9ae836585969..be694e475899 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -1,117 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; -import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.coders.Coder; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; /** - * Created by peihe on 06/07/2017. + * Class that optimizes the initial graph to a fused graph. */ public class GraphPlanner { - public Graph plan(Graph initGraph) { - FusionVisitor fusionVisitor = new FusionVisitor(); - initGraph.accept(fusionVisitor); - return fusionVisitor.getFusedGraph(); - } - - private class FusionVisitor implements GraphVisitor { - private Graph fusedGraph; - private Graph.Vertex workingVertex; - private Graph.NodePath workingPath; - private Coder workingEdgeCoder; - - FusionVisitor() { - fusedGraph = new Graph(); - workingVertex = null; - workingPath = null; - } - - @Override - public void visitRead(Graph.Vertex read) { - if (workingVertex == null) { - // drop if read is leaf vertex. - return; - } - Graph.Vertex v = fusedGraph.addVertex(read.getStep()); - workingPath.addFirst(read.getStep()); - Graph.Edge edge = fusedGraph.addEdge(v, workingVertex, workingEdgeCoder); - edge.addPath(workingPath); - } - - @Override - public void visitParDo(Graph.Vertex parDo) { - Graph.Step step = parDo.getStep(); - checkArgument( - step.getTransform().getAdditionalInputs().isEmpty(), - "Side inputs are not " + "supported."); - checkArgument( - parDo.getIncoming().size() == 1, - "Side inputs are not supported."); - Graph.Edge inEdge = parDo.getIncoming().iterator().next(); + public GraphPlanner() { + } - if (workingVertex == null) { - // Leaf vertex - workingVertex = fusedGraph.addVertex(step); - workingPath = new Graph.NodePath(); - workingEdgeCoder = inEdge.getCoder(); - } else { - workingPath.addFirst(step); - } - processParent(inEdge.getHead()); - } + public Graphs.FusedGraph plan(Graph initGraph) { + Graphs.FusedGraph fusedGraph = new Graphs.FusedGraph(); + // Convert from the list of steps to Graphs. + for (Graphs.Step step : Lists.reverse(initGraph.getSteps())) { + Graphs.FusedStep fusedStep = new Graphs.FusedStep(); + fusedStep.addStep(step); + fusedGraph.addFusedStep(fusedStep); - @Override - public void visitFlatten(Graph.Vertex flatten) { - if (workingVertex == null) { - return; - } - Graph.NodePath basePath = workingPath; - Graph.Vertex baseVertex = workingVertex; - for (Graph.Edge e : flatten.getIncoming()) { - workingPath = new Graph.NodePath(basePath); - workingVertex = baseVertex; - workingEdgeCoder = e.getCoder(); - processParent(e.getHead()); - } + tryFuse(fusedGraph, fusedStep); } + return fusedGraph; + } - @Override - public void visitGroupByKey(Graph.Vertex groupByKey) { - if (workingVertex == null) { - return; - } - Graph.Step step = groupByKey.getStep(); - Graph.Vertex addedGroupByKey = fusedGraph.addVertex(step); - - Graph.Edge edge = fusedGraph.addEdge( - addedGroupByKey, - workingVertex, - workingEdgeCoder); - edge.addPath(workingPath); - Graph.Edge inEdge = groupByKey.getIncoming().iterator().next(); - workingVertex = addedGroupByKey; - workingPath = new Graph.NodePath(); - workingEdgeCoder = inEdge.getCoder(); - processParent(inEdge.getHead()); + private void tryFuse(Graphs.FusedGraph fusedGraph, Graphs.FusedStep fusedStep) { + if (fusedStep.getOutputTags().size() != 1) { + return; } - - public Graph getFusedGraph() { - return fusedGraph; + Graphs.Tag outTag = Iterables.getOnlyElement(fusedStep.getOutputTags()); + if (fusedGraph.getConsumers(outTag).size() != 1) { + return; } - - private void processParent(Graph.Vertex parent) { - Graph.Step step = parent.getStep(); - Graph.Vertex v = fusedGraph.getVertex(step); - if (v == null) { - parent.accept(this); - } else { - // TODO: parent is consumed more than once. - // It is duplicated in multiple outgoing path. Figure out the impact. - workingPath.addFirst(step); - fusedGraph.getEdge(v, workingVertex).addPath(workingPath); - } + Graphs.FusedStep consumer = Iterables.getOnlyElement(fusedGraph.getConsumers(outTag)); + if (fusedStep.containsGroupByKey() && consumer.containsGroupByKey()) { + return; } + fusedGraph.merge(fusedStep, consumer); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java deleted file mode 100644 index fe4a76f0ddeb..000000000000 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphVisitor.java +++ /dev/null @@ -1,11 +0,0 @@ -package org.apache.beam.runners.mapreduce.translation; - -/** - * Created by peihe on 06/07/2017. - */ -public interface GraphVisitor { - void visitRead(Graph.Vertex read); - void visitParDo(Graph.Vertex parDo); - void visitFlatten(Graph.Vertex flatten); - void visitGroupByKey(Graph.Vertex groupByKey); -} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java new file mode 100644 index 000000000000..029d425d4677 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import com.google.auto.value.AutoValue; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.values.TupleTag; + +/** + * Class that defines graph vertices. + */ +public class Graphs { + + private Graphs() {} + + public static class FusedGraph { + private final Graph graph; + + public FusedGraph() { + this.graph = new Graph<>(); + } + + public void addFusedStep(FusedStep fusedStep) { + graph.addStep(fusedStep); + } + + public void merge(FusedStep src, FusedStep dest) { + for (Step step : src.steps.getSteps()) { + dest.addStep(step); + } + for (Tag inTag : src.getInputTags()) { + graph.addEdge(inTag, dest); + } + for (Tag outTag : src.getOutputTags()) { + graph.addEdge(dest, outTag); + } + graph.removeStep(src); + } + + public FusedStep getProducer(Tag tag) { + return graph.getProducer(tag); + } + + public List getConsumers(Tag tag) { + return graph.getConsumers(tag); + } + + public List getFusedSteps() { + return graph.getSteps(); + } + } + + public static class FusedStep extends Graph.AbstractStep { + private final Graph steps; + private Step groupByKeyStep; + + public FusedStep() { + this.steps = new Graph<>(); + this.groupByKeyStep = null; + } + + @Override + public List getInputTags() { + return steps.getInputTags(); + } + + @Override + public List getOutputTags() { + return steps.getOutputTags(); + } + + public void addStep(Step step) { + steps.addStep(step); + if (step.getOperation() instanceof GroupByKeyOperation) { + groupByKeyStep = step; + } + } + + public void removeStep(Step step) { + steps.removeStep(step); + } + + public void removeTag(Tag tag) { + steps.removeTag(tag); + } + + public void addEdge(Tag inTag, Step step) { + steps.addEdge(inTag, step); + } + + public void addEdge(Step step, Tag outTag) { + steps.addEdge(step, outTag); + } + + public void removeEdge(Tag inTag, Step step) { + steps.removeEdge(inTag, step); + } + + public void removeEdge(Step step, Tag outTag) { + steps.removeEdge(step, outTag); + } + + public Step getProducer(Tag tag) { + return steps.getProducer(tag); + } + + public List getConsumers(Tag tag) { + return steps.getConsumers(tag); + } + + public List getSteps() { + return steps.getSteps(); + } + + public List getStartSteps() { + return steps.getStartSteps(); + } + + public boolean containsGroupByKey() { + return groupByKeyStep != null; + } + + @Nullable + public Step getGroupByKeyStep() { + return groupByKeyStep; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + for (Step step : steps.getSteps()) { + sb.append(step.getFullName() + "|"); + } + if (sb.length() > 0) { + sb.deleteCharAt(sb.length() - 1); + } + return sb.toString(); + } + + public String getFullName() { + return toString(); + } + } + + @AutoValue + public abstract static class Step extends Graph.AbstractStep { + abstract String getFullName(); + // TODO: remove public + public abstract Operation getOperation(); + + public static Step of( + String fullName, + Operation operation, + List inputTags, + List outputTags) { + return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graphs_Step( + inputTags, outputTags, fullName, operation); + } + } + + @AutoValue + public abstract static class Tag extends Graph.AbstractTag { + abstract TupleTag getTupleTag(); + abstract Coder getCoder(); + + public static Tag of(TupleTag tupleTag, Coder coder) { + return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graphs_Tag( + tupleTag, coder); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java index 1da8d262212f..66cf3b613efa 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java @@ -1,7 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.ImmutableList; import java.util.List; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.coders.Coder; @@ -11,7 +29,7 @@ import org.apache.beam.sdk.values.WindowingStrategy; /** - * Created by peihe on 26/07/2017. + * {@link Operation} that executes a {@link GroupAlsoByWindowsViaOutputBufferDoFn}. */ public class GroupAlsoByWindowsParDoOperation extends ParDoOperation { @@ -19,11 +37,9 @@ public class GroupAlsoByWindowsParDoOperation extends ParDoOperation { public GroupAlsoByWindowsParDoOperation( PipelineOptions options, - TupleTag mainOutputTag, - List> sideOutputTags, WindowingStrategy windowingStrategy, Coder inputCoder) { - super(options, mainOutputTag, sideOutputTags, windowingStrategy); + super(options, new TupleTag<>(), ImmutableList.>of(), windowingStrategy); this.inputCoder = checkNotNull(inputCoder, "inputCoder"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java index 8ee616d7b7ea..5ac23a50b22d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyOperation.java new file mode 100644 index 000000000000..b0be4947d88c --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyOperation.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * A GroupByKey place holder {@link Operation} during pipeline translation. + */ +public class GroupByKeyOperation extends Operation> { + + private final WindowingStrategy windowingStrategy; + private final KvCoder kvCoder; + + public GroupByKeyOperation(WindowingStrategy windowingStrategy, KvCoder kvCoder) { + super(1); + this.windowingStrategy = checkNotNull(windowingStrategy, "windowingStrategy"); + this.kvCoder = checkNotNull(kvCoder, "kvCoder"); + } + + @Override + public void process(WindowedValue elem) { + throw new IllegalStateException( + String.format("%s should not in execution graph.", this.getClass().getSimpleName())); + } + + public WindowingStrategy getWindowingStrategy() { + return windowingStrategy; + } + + public KvCoder getKvCoder() { + return kvCoder; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java new file mode 100644 index 000000000000..e87ed09e6008 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * Translates a {@link GroupByKey} to {@link Operation Operations}. + */ +class GroupByKeyTranslator extends TransformTranslator.Default> { + @Override + public void translateNode(GroupByKey transform, TranslationContext context) { + TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); + + PCollection inPCollection = (PCollection) userGraphContext.getInput(); + WindowingStrategy windowingStrategy = inPCollection.getWindowingStrategy(); + Coder inCoder = inPCollection.getCoder(); + + GroupByKeyOperation groupByKeyOperation = + new GroupByKeyOperation<>(windowingStrategy, (KvCoder) inCoder); + context.addInitStep(Graphs.Step.of( + userGraphContext.getStepName(), + groupByKeyOperation, + userGraphContext.getInputTags(), + userGraphContext.getOutputTags())); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 576c6bf56855..c336a70d7b60 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -1,27 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import com.google.common.base.Predicate; -import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import java.io.IOException; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Set; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; @@ -33,22 +42,25 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; /** - * Created by peihe on 24/07/2017. + * Class that translates a {@link Graphs.FusedStep} to a MapReduce job. */ public class JobPrototype { - public static JobPrototype create(int stageId, Graph.Vertex vertex) { - return new JobPrototype(stageId, vertex); + public static JobPrototype create( + int stageId, Graphs.FusedStep fusedStep, PipelineOptions options) { + return new JobPrototype(stageId, fusedStep, options); } private final int stageId; - private final Graph.Vertex vertex; + private final Graphs.FusedStep fusedStep; private final Set dependencies; + private final PipelineOptions options; - private JobPrototype(int stageId, Graph.Vertex vertex) { + private JobPrototype(int stageId, Graphs.FusedStep fusedStep, PipelineOptions options) { this.stageId = stageId; - this.vertex = checkNotNull(vertex, "vertex"); + this.fusedStep = checkNotNull(fusedStep, "fusedStep"); this.dependencies = Sets.newHashSet(); + this.options = checkNotNull(options, "options"); } public Job build(Class jarClass, Configuration conf) throws IOException { @@ -57,168 +69,101 @@ public Job build(Class jarClass, Configuration conf) throws IOException { job.setJarByClass(jarClass); conf.set( "io.serializations", - "org.apache.hadoop.io.serializer.WritableSerialization," + - "org.apache.hadoop.io.serializer.JavaSerialization"); + "org.apache.hadoop.io.serializer.WritableSerialization," + + "org.apache.hadoop.io.serializer.JavaSerialization"); // Setup BoundedSources in BeamInputFormat. - // TODO: support more than one in-edge - Graph.Edge inEdge = Iterables.getOnlyElement(vertex.getIncoming()); - Graph.Vertex head = inEdge.getHead(); - Graph.Step headStep = head.getStep(); - checkState(headStep.getTransform() instanceof Read.Bounded); - Read.Bounded read = (Read.Bounded) headStep.getTransform(); + // TODO: support more than one read steps by introducing a composed BeamInputFormat + // and a partition operation. + Graphs.Step readStep = Iterables.getOnlyElement(fusedStep.getStartSteps()); + checkState(readStep.getOperation() instanceof ReadOperation); + BoundedSource source = ((ReadOperation) readStep.getOperation()).getSource(); conf.set( BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(read.getSource()))); + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(source))); job.setInputFormatClass(BeamInputFormat.class); - // Setup DoFns in BeamMapper. - // TODO: support more than one in-path. - Graph.NodePath inPath = Iterables.getOnlyElement(inEdge.getPaths()); - - Operation mapperParDoRoot = chainParDosInPath(inPath); - Operation mapperParDoTail = getTailOperation(mapperParDoRoot); - Graph.Step vertexStep = vertex.getStep(); - if (vertexStep.getTransform() instanceof ParDo.SingleOutput - || vertexStep.getTransform() instanceof ParDo.MultiOutput - || vertexStep.getTransform() instanceof Window.Assign) { - // TODO: add a TailVertex type to simplify the translation. - Operation vertexParDo = translateToOperation(vertexStep); - Operation mapperWrite = new WriteOperation( - getKeyCoder(inEdge.getCoder()), - getReifyValueCoder(inEdge.getCoder(), vertexStep.getWindowingStrategy())); - mapperParDoTail.attachOutput(vertexParDo, 0); - vertexParDo.attachOutput(mapperWrite, 0); - } else if (vertexStep.getTransform() instanceof GroupByKey) { - Operation reifyOperation = new ReifyTimestampAndWindowsParDoOperation( - PipelineOptionsFactory.create(), - new TupleTag<>(), - ImmutableList.>of(), - vertexStep.getWindowingStrategy()); - Operation mapperWrite = new WriteOperation( - getKeyCoder(inEdge.getCoder()), - getReifyValueCoder(inEdge.getCoder(), vertexStep.getWindowingStrategy())); - mapperParDoTail.attachOutput(reifyOperation, 0); - reifyOperation.attachOutput(mapperWrite, 0); - } else { - throw new UnsupportedOperationException("Transform: " + vertexStep.getTransform()); - } - job.setMapOutputKeyClass(BytesWritable.class); - job.setMapOutputValueClass(byte[].class); - conf.set( - BeamMapper.BEAM_PAR_DO_OPERATION_MAPPER, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(mapperParDoRoot))); - job.setMapperClass(BeamMapper.class); + if (fusedStep.containsGroupByKey()) { + Graphs.Step groupByKey = fusedStep.getGroupByKeyStep(); + GroupByKeyOperation operation = (GroupByKeyOperation) groupByKey.getOperation(); + WindowingStrategy windowingStrategy = operation.getWindowingStrategy(); + KvCoder kvCoder = operation.getKvCoder(); + + Coder reifyValueCoder = getReifyValueCoder(kvCoder.getValueCoder(), windowingStrategy); + Graphs.Tag reifyOutputTag = Graphs.Tag.of(new TupleTag(), reifyValueCoder); + Graphs.Step reifyStep = Graphs.Step.of( + groupByKey.getFullName() + "-Reify", + new ReifyTimestampAndWindowsParDoOperation(options, operation.getWindowingStrategy()), + groupByKey.getInputTags(), + ImmutableList.of(reifyOutputTag)); + + Graphs.Step writeStep = Graphs.Step.of( + groupByKey.getFullName() + "-Write", + new WriteOperation(kvCoder.getKeyCoder(), reifyValueCoder), + ImmutableList.of(reifyOutputTag), + Collections.emptyList()); + + Graphs.Step gabwStep = Graphs.Step.of( + groupByKey.getFullName() + "-GroupAlsoByWindows", + new GroupAlsoByWindowsParDoOperation(options, windowingStrategy, kvCoder), + Collections.emptyList(), + groupByKey.getOutputTags()); + + fusedStep.addStep(reifyStep); + fusedStep.addStep(writeStep); + fusedStep.addStep(gabwStep); + fusedStep.removeStep(groupByKey); - if (vertexStep.getTransform() instanceof GroupByKey) { // Setup BeamReducer - Operation gabwOperation = new GroupAlsoByWindowsParDoOperation( - PipelineOptionsFactory.create(), - (TupleTag) vertexStep.getOutputs().iterator().next(), - ImmutableList.>of(), - vertexStep.getWindowingStrategy(), - inEdge.getCoder()); - Graph.Edge outEdge = Iterables.getOnlyElement(vertex.getOutgoing()); - Graph.NodePath outPath = Iterables.getOnlyElement(outEdge.getPaths()); - Operation reducerParDoRoot = chainParDosInPath(outPath); - Operation reducerParDoTail = getTailOperation(reducerParDoRoot); - - Operation reducerTailParDo = translateToOperation(outEdge.getTail().getStep()); - if (reducerParDoRoot == null) { - gabwOperation.attachOutput(reducerTailParDo, 0); - } else { - gabwOperation.attachOutput(reducerParDoRoot, 0); - reducerParDoTail.attachOutput(reducerTailParDo, 0); - } + Graphs.Step reducerStartStep = gabwStep; + chainOperations(reducerStartStep, fusedStep); conf.set( BeamReducer.BEAM_REDUCER_KV_CODER, Base64.encodeBase64String(SerializableUtils.serializeToByteArray( - KvCoder.of( - getKeyCoder(inEdge.getCoder()), - getReifyValueCoder(inEdge.getCoder(), vertexStep.getWindowingStrategy()))))); + KvCoder.of(kvCoder.getKeyCoder(), reifyValueCoder)))); conf.set( BeamReducer.BEAM_PAR_DO_OPERATION_REDUCER, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(gabwOperation))); + Base64.encodeBase64String( + SerializableUtils.serializeToByteArray(reducerStartStep.getOperation()))); job.setReducerClass(BeamReducer.class); } - job.setOutputFormatClass(NullOutputFormat.class); - return job; - } + // Setup DoFns in BeamMapper. + Graphs.Tag readOutputTag = Iterables.getOnlyElement(readStep.getOutputTags()); + Graphs.Step mapperStartStep = Iterables.getOnlyElement(fusedStep.getConsumers(readOutputTag)); + chainOperations(mapperStartStep, fusedStep); - private Coder getKeyCoder(Coder coder) { - KvCoder kvCoder = (KvCoder) checkNotNull(coder, "coder"); - return kvCoder.getKeyCoder(); - } + job.setMapOutputKeyClass(BytesWritable.class); + job.setMapOutputValueClass(byte[].class); + conf.set( + BeamMapper.BEAM_PAR_DO_OPERATION_MAPPER, + Base64.encodeBase64String( + SerializableUtils.serializeToByteArray(mapperStartStep.getOperation()))); + job.setMapperClass(BeamMapper.class); - private Coder getReifyValueCoder( - Coder coder, WindowingStrategy windowingStrategy) { - KvCoder kvCoder = (KvCoder) checkNotNull(coder, "coder"); - return (Coder) WindowedValue.getFullCoder( - kvCoder.getValueCoder(), windowingStrategy.getWindowFn().windowCoder()); - } + job.setOutputFormatClass(NullOutputFormat.class); - private Operation getTailOperation(@Nullable Operation operation) { - if (operation == null) { - return null; - } - if (operation.getOutputReceivers().isEmpty()) { - return operation; - } - OutputReceiver receiver = Iterables.getOnlyElement(operation.getOutputReceivers()); - if (receiver.getReceivingOperations().isEmpty()) { - return operation; - } - return getTailOperation(Iterables.getOnlyElement(receiver.getReceivingOperations())); + return job; } - private Operation chainParDosInPath(Graph.NodePath path) { - List parDos = new ArrayList<>(); - // TODO: we should not need this filter. - parDos.addAll(FluentIterable.from(path.steps()) - .filter(new Predicate() { - @Override - public boolean apply(Graph.Step input) { - PTransform transform = input.getTransform(); - return !(transform instanceof Read.Bounded); - }}) - .toList()); - - Operation root = null; - Operation prev = null; - for (Graph.Step step : parDos) { - Operation current = translateToOperation(step); - if (prev == null) { - root = current; - } else { - // TODO: set a proper outputNum for ParDo.MultiOutput instead of zero. - prev.attachOutput(current, 0); + private void chainOperations(Graphs.Step current, Graphs.FusedStep fusedStep) { + Operation operation = current.getOperation(); + List outputTags = current.getOutputTags(); + for (int index = 0; index < outputTags.size(); ++index) { + for (Graphs.Step consumer : fusedStep.getConsumers(outputTags.get(index))) { + operation.attachConsumer(index, consumer.getOperation()); + } + } + for (Graphs.Tag outTag : outputTags) { + for (Graphs.Step consumer : fusedStep.getConsumers(outTag)) { + chainOperations(consumer, fusedStep); } - prev = current; } - return root; } - private Operation translateToOperation(Graph.Step parDoStep) { - PTransform transform = parDoStep.getTransform(); - DoFn doFn; - if (transform instanceof ParDo.SingleOutput) { - return new NormalParDoOperation( - ((ParDo.SingleOutput) transform).getFn(), - PipelineOptionsFactory.create(), - (TupleTag) parDoStep.getOutputs().iterator().next(), - ImmutableList.>of(), - parDoStep.getWindowingStrategy()); - } else if (transform instanceof ParDo.MultiOutput) { - return new NormalParDoOperation( - ((ParDo.MultiOutput) transform).getFn(), - PipelineOptionsFactory.create(), - (TupleTag) parDoStep.getOutputs().iterator().next(), - ImmutableList.>of(), - parDoStep.getWindowingStrategy()); - } else if (transform instanceof Window.Assign) { - return new WindowAssignOperation<>(1, parDoStep.getWindowingStrategy().getWindowFn()); - } else { - throw new UnsupportedOperationException("Transform: " + transform); - } + private Coder getReifyValueCoder( + Coder valueCoder, WindowingStrategy windowingStrategy) { + // TODO: do we need full coder to encode windows. + return (Coder) WindowedValue.getFullCoder( + valueCoder, windowingStrategy.getWindowFn().windowCoder()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java index 1da39a9b955f..fd1b5280ddc7 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java @@ -26,16 +26,16 @@ import org.apache.beam.sdk.values.WindowingStrategy; /** - * Created by peihe on 26/07/2017. + * {@link Operation} that executes a {@link DoFn}. */ -public class NormalParDoOperation extends ParDoOperation { +public class NormalParDoOperation extends ParDoOperation { - private final DoFn doFn; + private final DoFn doFn; public NormalParDoOperation( - DoFn doFn, + DoFn doFn, PipelineOptions options, - TupleTag mainOutputTag, + TupleTag mainOutputTag, List> sideOutputTags, WindowingStrategy windowingStrategy) { super(options, mainOutputTag, sideOutputTags, windowingStrategy); @@ -43,7 +43,7 @@ public NormalParDoOperation( } @Override - DoFn getDoFn() { + DoFn getDoFn() { return doFn; } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java index 695190949d93..187ea79651a0 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java @@ -1,14 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import com.google.common.collect.ImmutableList; import java.io.Serializable; import java.util.List; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.hadoop.mapreduce.TaskInputOutputContext; /** - * Created by peihe on 26/07/2017. + * Class that processes elements and forwards outputs to consumers. */ -public abstract class Operation implements Serializable { +public abstract class Operation implements Serializable { private final OutputReceiver[] receivers; public Operation(int numOutputs) { @@ -37,7 +55,7 @@ public void start(TaskInputOutputContext taskCon /** * Processes the element. */ - public abstract void process(Object elem); + public abstract void process(WindowedValue elem); /** * Finishes this Operation's execution. @@ -62,8 +80,8 @@ public List getOutputReceivers() { /** * Adds an output to this Operation. */ - public void attachOutput(Operation output, int outputNum) { - OutputReceiver fanOut = receivers[outputNum]; - fanOut.addOutput(output); + public void attachConsumer(int outputIndex, Operation consumer) { + OutputReceiver fanOut = receivers[outputIndex]; + fanOut.addOutput(consumer); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java index 6aeefd2af228..3dab890f8201 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import org.apache.beam.sdk.util.WindowedValue; /** * OutputReceiver that forwards each input it receives to each of a list of down stream operations. @@ -42,7 +43,7 @@ public List getReceivingOperations() { /** * Processes the element. */ - public void process(Object elem) { + public void process(WindowedValue elem) { for (Operation out : receivingOperations) { if (out != null) { out.process(elem); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java index 2627d20a37b1..a76773f1011e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -36,19 +36,19 @@ /** * Operation for ParDo. */ -public abstract class ParDoOperation extends Operation { +public abstract class ParDoOperation extends Operation { private static final Logger LOG = LoggerFactory.getLogger(ParDoOperation.class); protected final SerializedPipelineOptions options; - protected final TupleTag mainOutputTag; + protected final TupleTag mainOutputTag; private final List> sideOutputTags; protected final WindowingStrategy windowingStrategy; - private DoFnRunner fnRunner; + private DoFnRunner fnRunner; public ParDoOperation( PipelineOptions options, - TupleTag mainOutputTag, + TupleTag mainOutputTag, List> sideOutputTags, WindowingStrategy windowingStrategy) { super(1 + sideOutputTags.size()); @@ -61,7 +61,7 @@ public ParDoOperation( /** * Returns a {@link DoFn} for processing inputs. */ - abstract DoFn getDoFn(); + abstract DoFn getDoFn(); @Override public void start(TaskInputOutputContext taskContext) { @@ -82,9 +82,9 @@ public void start(TaskInputOutputContext taskCon * Processes the element. */ @Override - public void process(Object elem) { + public void process(WindowedValue elem) { LOG.info("elem: {}.", elem); - fnRunner.processElement((WindowedValue) elem); + fnRunner.processElement(elem); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java new file mode 100644 index 000000000000..1a1373a0936e --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + +/** + * Translates a {@link ParDo} to a {@link Operation}. + */ +class ParDoTranslator + extends TransformTranslator.Default> { + + @Override + public void translateNode( + ParDo.MultiOutput transform, TranslationContext context) { + TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); + NormalParDoOperation operation = new NormalParDoOperation( + transform.getFn(), + userGraphContext.getOptions(), + transform.getMainOutputTag(), + transform.getAdditionalOutputTags().getAll(), + ((PCollection) userGraphContext.getInput()).getWindowingStrategy()); + + context.addInitStep(Graphs.Step.of( + userGraphContext.getStepName(), + operation, + userGraphContext.getInputTags(), + userGraphContext.getOutputTags())); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java new file mode 100644 index 000000000000..07108275239a --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.io.Read; + +/** + * Translates a {@link Read.Bounded} to a {@link ReadOperation}. + */ +class ReadBoundedTranslator extends TransformTranslator.Default> { + @Override + public void translateNode(Read.Bounded transform, TranslationContext context) { + TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); + + ReadOperation operation = new ReadOperation(transform.getSource()); + context.addInitStep(Graphs.Step.of( + userGraphContext.getStepName(), + operation, + userGraphContext.getInputTags(), + userGraphContext.getOutputTags())); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java new file mode 100644 index 000000000000..c199dc6a7175 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.util.WindowedValue; + +/** + * A Read.Bounded place holder {@link Operation} during pipeline translation. + */ +class ReadOperation extends Operation { + private final BoundedSource source; + + ReadOperation(BoundedSource source) { + super(1); + this.source = checkNotNull(source, "source"); + } + + @Override + public void process(WindowedValue elem) { + throw new IllegalStateException( + String.format("%s should not in execution graph.", this.getClass().getSimpleName())); + } + + BoundedSource getSource() { + return source; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java index ec954bb52d3f..83d1af5cdcd4 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java @@ -1,5 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; +import com.google.common.collect.ImmutableList; import java.util.List; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -10,16 +28,14 @@ import org.apache.beam.sdk.values.WindowingStrategy; /** - * Created by peihe on 27/07/2017. + * {@link Operation} that executes {@link ReifyTimestampAndWindowsDoFn}. */ public class ReifyTimestampAndWindowsParDoOperation extends ParDoOperation { public ReifyTimestampAndWindowsParDoOperation( PipelineOptions options, - TupleTag mainOutputTag, - List> sideOutputTags, WindowingStrategy windowingStrategy) { - super(options, mainOutputTag, sideOutputTags, windowingStrategy); + super(options, new TupleTag<>(), ImmutableList.>of(), windowingStrategy); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TransformTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TransformTranslator.java new file mode 100644 index 000000000000..f49537227065 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TransformTranslator.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.transforms.PTransform; + +/** + * Interface for classes capable of tranforming Beam PTransforms into Storm primitives. + */ +interface TransformTranslator> { + + void translateNode(T transform, TranslationContext context); + + /** + * Returns true if this translator can translate the given transform. + */ + boolean canTranslate(T transform, TranslationContext context); + + /** + * Default translator. + */ + class Default> implements TransformTranslator { + @Override + public void translateNode(T1 transform, TranslationContext context) { + + } + + @Override + public boolean canTranslate(T1 transform, TranslationContext context) { + return true; + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java new file mode 100644 index 000000000000..0df365e04505 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import java.util.List; +import java.util.Map; +import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; +import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; + +/** + * Class that maintains contexts during translation. + */ +public class TranslationContext { + + private final UserGraphContext userGraphContext; + private final Graph initGraph; + + public TranslationContext(MapReducePipelineOptions options) { + this.userGraphContext = new UserGraphContext(options); + this.initGraph = new Graph<>(); + } + + public UserGraphContext getUserGraphContext() { + return userGraphContext; + } + + public void addInitStep(Graphs.Step step) { + initGraph.addStep(step); + } + + /** + * Returns {@link Graphs.Step steps} in reverse topological order. + */ + public Graph getInitGraph() { + return initGraph; + } + + /** + * Context of user graph. + */ + public static class UserGraphContext { + private final MapReducePipelineOptions options; + private final Map> pValueToTupleTag; + private TransformHierarchy.Node currentNode; + + public UserGraphContext(MapReducePipelineOptions options) { + this.options = checkNotNull(options, "options"); + this.pValueToTupleTag = Maps.newHashMap(); + this.currentNode = null; + } + + public MapReducePipelineOptions getOptions() { + return options; + } + + public void setCurrentNode(TransformHierarchy.Node node) { + this.currentNode = node; + for (Map.Entry, PValue> entry : currentNode.getOutputs().entrySet()) { + pValueToTupleTag.put(entry.getValue(), entry.getKey()); + } + } + + public String getStepName() { + return currentNode.getFullName(); + } + + public PValue getInput() { + return Iterables.get(currentNode.getInputs().values(), 0); + } + + public PValue getOutput() { + return Iterables.get(currentNode.getOutputs().values(), 0); + } + + public List getInputTags() { + return FluentIterable.from(currentNode.getInputs().values()) + .transform(new Function() { + @Override + public Graphs.Tag apply(PValue pValue) { + checkState( + pValueToTupleTag.containsKey(pValue), + String.format("Failed to find TupleTag for pValue: %s.", pValue)); + return Graphs.Tag.of( + pValueToTupleTag.get(pValue), ((PCollection) pValue).getCoder()); + }}) + .toList(); + } + + public List getOutputTags() { + return FluentIterable.from(currentNode.getOutputs().entrySet()) + .transform(new Function, PValue>, Graphs.Tag>() { + @Override + public Graphs.Tag apply(Map.Entry, PValue> entry) { + return Graphs.Tag.of(entry.getKey(), ((PCollection) entry.getValue()).getCoder()); + }}) + .toList(); + } + + public TupleTag getOnlyOutputTag() { + return Iterables.getOnlyElement(currentNode.getOutputs().keySet()); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java new file mode 100644 index 000000000000..f79260ad63ff --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Lookup table mapping PTransform types to associated TransformTranslator implementations. + */ +public class TranslatorRegistry { + + private static final Logger LOG = LoggerFactory.getLogger(TranslatorRegistry.class); + + private static final Map, TransformTranslator> TRANSLATORS = + new HashMap<>(); + + static { + TRANSLATORS.put(Read.Bounded.class, new ReadBoundedTranslator()); + TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoTranslator()); + TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslator<>()); + TRANSLATORS.put(Flatten.PCollections.class, new FlattenTranslator()); + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator()); + TRANSLATORS.put(View.CreatePCollectionView.class, new ViewTranslator()); + } + + public static TransformTranslator getTranslator(PTransform transform) { + TransformTranslator translator = TRANSLATORS.get(transform.getClass()); + if (translator == null) { + LOG.warn("Unsupported operator={}", transform.getClass().getName()); + } + return translator; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java new file mode 100644 index 000000000000..093f00e2193f --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Throwables; +import java.io.ByteArrayOutputStream; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; + +/** + * {@link Operation} that materializes views. + */ +public class ViewOperation extends Operation { + + private final Coder> valueCoder; + + private transient TaskInputOutputContext taskContext; + + public ViewOperation(Coder> valueCoder) { + super(0); + this.valueCoder = checkNotNull(valueCoder, "valueCoder"); + } + + @Override + public void start(TaskInputOutputContext taskContext) { + this.taskContext = checkNotNull(taskContext, "taskContext"); + } + + @Override + public void process(WindowedValue elem) { + try { + ByteArrayOutputStream valueStream = new ByteArrayOutputStream(); + valueCoder.encode(elem, valueStream); + taskContext.write(new BytesWritable("view".getBytes()), valueStream.toByteArray()); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java new file mode 100644 index 000000000000..815ce77c39b3 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.View; + +/** + * Translates a {@link View.CreatePCollectionView} to a {@link ViewOperation}. + */ +public class ViewTranslator extends TransformTranslator.Default> { + + @Override + public void translateNode( + View.CreatePCollectionView transform, TranslationContext context) { + TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); + + ViewOperation operation = + new ViewOperation<>((Coder) transform.getView().getPCollection().getCoder()); + + context.addInitStep(Graphs.Step.of( + userGraphContext.getStepName(), + operation, + userGraphContext.getInputTags(), + userGraphContext.getOutputTags())); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java index 144ef16c6046..3279e11a46b2 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignOperation.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkArgument; @@ -12,28 +29,24 @@ import org.joda.time.Instant; /** - * Created by peihe on 27/07/2017. + * {@link Operation} that executes for assigning windows to elements. */ -public class WindowAssignOperation extends Operation { +public class WindowAssignOperation extends Operation { private final WindowFn windowFn; - public WindowAssignOperation(int numOutputs, WindowFn windowFn) { - super(numOutputs); + public WindowAssignOperation(WindowFn windowFn) { + super(1); this.windowFn = checkNotNull(windowFn, "windowFn"); } @Override - public void process(Object elem) { - WindowedValue windowedValue = (WindowedValue) elem; + public void process(WindowedValue elem) { try { - Collection windows = windowFn.assignWindows(new AssignContextInternal<>(windowFn, windowedValue)); + Collection windows = windowFn.assignWindows(new AssignContextInternal<>(windowFn, elem)); for (W window : windows) { OutputReceiver receiver = Iterables.getOnlyElement(getOutputReceivers()); receiver.process(WindowedValue.of( - windowedValue.getValue(), - windowedValue.getTimestamp(), - window, - windowedValue.getPane())); + elem.getValue(), elem.getTimestamp(), window, elem.getPane())); } } catch (Exception e) { Throwables.throwIfUnchecked(e); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java new file mode 100644 index 000000000000..367c375b6129 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.transforms.windowing.Window; + +/** + * Translates a {@link Window.Assign} to a {@link WindowAssignOperation}. + */ +public class WindowAssignTranslator extends TransformTranslator.Default> { + + @Override + public void translateNode(Window.Assign transform, TranslationContext context) { + TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); + + WindowAssignOperation operation = new WindowAssignOperation<>(transform.getWindowFn()); + context.addInitStep(Graphs.Step.of( + userGraphContext.getStepName(), + operation, + userGraphContext.getInputTags(), + userGraphContext.getOutputTags())); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java index 0585032a8a76..2eb4684f2c8c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; @@ -5,19 +22,15 @@ import com.google.common.base.Throwables; import java.io.ByteArrayOutputStream; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.NullableCoder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapreduce.TaskInputOutputContext; /** - * Created by peihe on 26/07/2017. + * {@link Operation} that materializes input for group by key. */ -public class WriteOperation extends Operation { +public class WriteOperation extends Operation { private final Coder keyCoder; private final Coder valueCoder; @@ -36,14 +49,14 @@ public void start(TaskInputOutputContext taskCon } @Override - public void process(Object elem) { - WindowedValue> windowedElem = (WindowedValue>) elem; + public void process(WindowedValue elem) { + KV kv = (KV) elem.getValue(); try { ByteArrayOutputStream keyStream = new ByteArrayOutputStream(); - keyCoder.encode(windowedElem.getValue().getKey(), keyStream); + keyCoder.encode(kv.getKey(), keyStream); ByteArrayOutputStream valueStream = new ByteArrayOutputStream(); - valueCoder.encode(windowedElem.getValue().getValue(), valueStream); + valueCoder.encode(kv.getValue(), valueStream); taskContext.write(new BytesWritable(keyStream.toByteArray()), valueStream.toByteArray()); } catch (Exception e) { Throwables.throwIfUnchecked(e); diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java index a548ba7fea17..363ba0124dfb 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce; import org.apache.beam.sdk.Pipeline; @@ -10,7 +27,7 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.log4j.BasicConfigurator; @@ -75,11 +92,11 @@ public void testWordCount() { // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the // static FormatAsTextFn() to the ParDo transform. p.apply("ReadLines", TextIO.read().from(input)) - .apply(Window.into(SlidingWindows.of(Duration.millis(100)))) + .apply(Window.into(FixedWindows.of(Duration.millis(1000)))) .apply(ParDo.of(new ExtractWordsFn())) .apply(Count.perElement()) - .apply(MapElements.via(new FormatAsTextFn())); - //.apply("WriteCounts", TextIO.write().to(output)); + .apply(MapElements.via(new FormatAsTextFn())) + .apply("WriteCounts", TextIO.write().to(output)); p.run(); } diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java index 4f0c28340650..76c8311a44d1 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphConverterTest.java @@ -1,12 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static org.junit.Assert.assertEquals; import com.google.common.collect.Iterables; +import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; @@ -23,17 +43,18 @@ public class GraphConverterTest { @Test public void testCombine() throws Exception { - Pipeline p = Pipeline.create(); + MapReducePipelineOptions options = PipelineOptionsFactory.as(MapReducePipelineOptions.class); + options.setRunner(CrashingRunner.class); + Pipeline p = Pipeline.create(options); PCollection> input = p .apply(Create.empty(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) .apply(Sum.integersPerKey()); - GraphConverter graphConverter = new GraphConverter(); + TranslationContext context = new TranslationContext(options); + GraphConverter graphConverter = new GraphConverter(context); p.traverseTopologically(graphConverter); - Graph graph = graphConverter.getGraph(); + Graph initGraph = context.getInitGraph(); - assertEquals(3, Iterables.size(graph.getAllVertices())); - assertEquals(2, Iterables.size(graph.getAllEdges())); - assertEquals(1, Iterables.size(graph.getLeafVertices())); + assertEquals(3, Iterables.size(initGraph.getSteps())); } } diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java index c98f81789a96..cf5262f82347 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java @@ -1,12 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.runners.mapreduce.translation; import static org.junit.Assert.assertEquals; import com.google.common.collect.Iterables; +import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; @@ -23,20 +43,21 @@ public class GraphPlannerTest { @Test public void testCombine() throws Exception { - Pipeline p = Pipeline.create(); + MapReducePipelineOptions options = PipelineOptionsFactory.as(MapReducePipelineOptions.class); + options.setRunner(CrashingRunner.class); + Pipeline p = Pipeline.create(options); PCollection> input = p .apply(Create.empty(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) .apply(Sum.integersPerKey()); - GraphConverter graphConverter = new GraphConverter(); - p.traverseTopologically(graphConverter); - Graph graph = graphConverter.getGraph(); + TranslationContext context = new TranslationContext(options); + GraphConverter graphConverter = new GraphConverter(context); + p.traverseTopologically(graphConverter); GraphPlanner planner = new GraphPlanner(); - Graph fusedGraph = planner.plan(graph); + Graphs.FusedGraph fusedGraph = planner.plan(context.getInitGraph()); - assertEquals(3, Iterables.size(fusedGraph.getAllVertices())); - assertEquals(2, Iterables.size(fusedGraph.getAllEdges())); - assertEquals(1, Iterables.size(fusedGraph.getLeafVertices())); + assertEquals(1, Iterables.size(fusedGraph.getFusedSteps())); + assertEquals(3, Iterables.getOnlyElement(fusedGraph.getFusedSteps()).getSteps().size()); } } From 98da2a2ac88c544dc3623b4f6bbe1cbbfaf569b2 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 2 Aug 2017 19:19:14 +0800 Subject: [PATCH 12/34] mr-runner: support graph visualization with dotfiles. --- .../mapreduce/MapReducePipelineOptions.java | 1 - .../runners/mapreduce/MapReduceRunner.java | 12 ++- .../mapreduce/translation/DotfileWriter.java | 54 ++++++++++++ .../mapreduce/translation/GraphConverter.java | 88 ++++++++++++++++++- .../runners/mapreduce/translation/Graphs.java | 19 ++-- .../mapreduce/translation/JobPrototype.java | 6 +- .../translation/TranslationContext.java | 6 +- 7 files changed, 171 insertions(+), 15 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java index 73c7d4711ff9..c37da586e29f 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -38,7 +38,6 @@ public interface MapReducePipelineOptions extends PipelineOptions { PipelineOptionsFactory.Builder.class.getName(), "org.apache.beam.sdk.options.ProxyInvocationHandler"); - @Description("The jar class of the user Beam program.") @Default.InstanceFactory(JarClassInstanceFactory.class) Class getJarClass(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index b6a82d1bc237..c5626a4de83f 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -20,9 +20,10 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.base.Throwables; -import org.apache.beam.runners.mapreduce.translation.Graphs; +import org.apache.beam.runners.mapreduce.translation.DotfileWriter; import org.apache.beam.runners.mapreduce.translation.GraphConverter; import org.apache.beam.runners.mapreduce.translation.GraphPlanner; +import org.apache.beam.runners.mapreduce.translation.Graphs; import org.apache.beam.runners.mapreduce.translation.JobPrototype; import org.apache.beam.runners.mapreduce.translation.TranslationContext; import org.apache.beam.sdk.Pipeline; @@ -31,12 +32,16 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Job; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * {@link PipelineRunner} for MapReduce. */ public class MapReduceRunner extends PipelineRunner { + private static final Logger LOG = LoggerFactory.getLogger(MapReduceRunner.class); + /** * Construct a runner from the provided options. * @@ -59,8 +64,13 @@ public PipelineResult run(Pipeline pipeline) { GraphConverter graphConverter = new GraphConverter(context); pipeline.traverseTopologically(graphConverter); + LOG.info(graphConverter.getDotfile()); + GraphPlanner planner = new GraphPlanner(); Graphs.FusedGraph fusedGraph = planner.plan(context.getInitGraph()); + + LOG.info(DotfileWriter.toDotfile(fusedGraph)); + int stageId = 0; for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { JobPrototype jobPrototype = JobPrototype.create(stageId++, fusedStep, options); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java new file mode 100644 index 000000000000..5b0fcd86f818 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +/** + * Class that outputs {@link Graph} to dot file. + */ +public class DotfileWriter { + + public static , TagT extends Graph.AbstractTag> + String toDotfile(Graphs.FusedGraph fusedGraph) { + StringBuilder sb = new StringBuilder(); + sb.append("\ndigraph G {\n"); + + int i = 0; + for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + sb.append(String.format(" subgraph \"cluster_%d\" {\n", i++)); + for (Graphs.Step step : fusedStep.getSteps()) { + sb.append(String.format(" \"%s\" [shape=box];\n", step.getFullName())); + for (Graph.AbstractTag outTag : step.getOutputTags()) { + sb.append(String.format(" \"%s\" [shape=ellipse];\n", outTag)); + } + } + sb.append(String.format(" }")); + } + for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + for (Graphs.Step step : fusedStep.getSteps()) { + for (Graph.AbstractTag inTag : step.getInputTags()) { + sb.append(String.format(" \"%s\" -> \"%s\";\n", inTag, step)); + } + for (Graph.AbstractTag outTag : step.getOutputTags()) { + sb.append(String.format(" \"%s\" -> \"%s\";\n", step, outTag)); + } + } + } + sb.append("}\n"); + return sb.toString(); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java index 1e818fa257a2..de1c80b758a3 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -18,14 +18,15 @@ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; + import com.google.common.collect.Maps; import java.util.Map; +import java.util.Stack; import org.apache.beam.runners.mapreduce.MapReduceRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TupleTag; /** * Pipeline translator for {@link MapReduceRunner}. @@ -33,11 +34,18 @@ public class GraphConverter extends Pipeline.PipelineVisitor.Defaults { private final TranslationContext context; - private final Map> pValueToTupleTag; + private final Stack dotfileNodesBuilders; + private final Map enclosedTransformCounts; + private final StringBuilder dotfileEdgesBuilder; + + private int indent; public GraphConverter(TranslationContext context) { this.context = checkNotNull(context, "context"); - this.pValueToTupleTag = Maps.newHashMap(); + this.enclosedTransformCounts = Maps.newHashMap(); + this.dotfileNodesBuilders = new Stack<>(); + this.dotfileEdgesBuilder = new StringBuilder(); + this.indent = 0; } @Override @@ -45,20 +53,47 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { // check if current composite transforms need to be translated. // If not, all sub transforms will be translated in visitPrimitiveTransform. PTransform transform = node.getTransform(); + dotfileNodesBuilders.push(new StringBuilder()); if (transform != null) { + markEnclosedTransformCounts(node); TransformTranslator translator = TranslatorRegistry.getTranslator(transform); if (translator != null && applyCanTranslate(transform, node, translator)) { applyTransform(transform, node, translator); return CompositeBehavior.DO_NOT_ENTER_TRANSFORM; } + indent += 2; } return CompositeBehavior.ENTER_TRANSFORM; } + @Override + public void leaveCompositeTransform(TransformHierarchy.Node node) { + if (node.getTransform() != null) { + if (enclosedTransformCounts.get(node) > 1) { + dotfileNodesBuilders.peek().insert(0, new StringBuilder() + .append(getIndent()).append( + String.format("subgraph \"cluster_%s\" {", node.getFullName())) + .append('\n') + .append(getIndent()).append( + String.format(" label=\"%s\";", node.getFullName())) + .append('\n') + .toString()); + dotfileNodesBuilders.peek().append(new StringBuilder() + .append(getIndent()).append("}").append('\n') + .toString()); + } + StringBuilder top = dotfileNodesBuilders.pop(); + dotfileNodesBuilders.peek().append(top.toString()); + indent -= 2; + } + } + @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (!node.isRootNode()) { + markEnclosedTransformCounts(node); + PTransform transform = node.getTransform(); TransformTranslator translator = TranslatorRegistry.getTranslator(transform); if (translator == null || !applyCanTranslate(transform, node, translator)) { @@ -69,10 +104,48 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { } } + @Override + public void visitValue(PValue value, TransformHierarchy.Node producer) { + dotfileNodesBuilders.peek().append(getIndent()) + .append(String.format("\"%s\" [shape=ellipse];", value.getName())) + .append('\n'); + } + + private void markEnclosedTransformCounts(TransformHierarchy.Node node) { + TransformHierarchy.Node parent = node.getEnclosingNode(); + Integer primitiveCount = enclosedTransformCounts.get(parent); + if (primitiveCount == null) { + primitiveCount = 0; + } + enclosedTransformCounts.put(parent, primitiveCount + 1); + } + + public String getDotfile() { + return String.format( + "\ndigraph G {\n%s%s}\n", + dotfileNodesBuilders.peek().toString(), + dotfileEdgesBuilder.toString()); + } + private > void applyTransform( PTransform transform, TransformHierarchy.Node node, TransformTranslator translator) { + dotfileNodesBuilders.peek() + .append(getIndent()) + .append(String.format("\"%s\" [shape=box];", node.getFullName())) + .append('\n'); + for (PValue input : node.getInputs().values()) { + dotfileEdgesBuilder + .append(String.format(" \"%s\" -> \"%s\";", input.getName(), node.getFullName())) + .append('\n'); + } + for (PValue output : node.getOutputs().values()) { + dotfileEdgesBuilder + .append(String.format(" \"%s\" -> \"%s\";", node.getFullName(), output.getName())) + .append('\n'); + } + @SuppressWarnings("unchecked") T typedTransform = (T) transform; @SuppressWarnings("unchecked") @@ -92,4 +165,13 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { context.getUserGraphContext().setCurrentNode(node); return typedTranslator.canTranslate(typedTransform, context); } + + private String getIndent() { + StringBuilder ret = new StringBuilder(); + for (int i = 0; i < indent; ++i) { + ret.append(' '); + } + return ret.toString(); + } } + diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java index 029d425d4677..cef5afcd0371 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java @@ -153,10 +153,6 @@ public String toString() { } return sb.toString(); } - - public String getFullName() { - return toString(); - } } @AutoValue @@ -173,16 +169,27 @@ public static Step of( return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graphs_Step( inputTags, outputTags, fullName, operation); } + + @Override + public String toString() { + return getFullName(); + } } @AutoValue public abstract static class Tag extends Graph.AbstractTag { + abstract String getName(); abstract TupleTag getTupleTag(); abstract Coder getCoder(); - public static Tag of(TupleTag tupleTag, Coder coder) { + @Override + public String toString() { + return getName(); + } + + public static Tag of(String name, TupleTag tupleTag, Coder coder) { return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graphs_Tag( - tupleTag, coder); + name, tupleTag, coder); } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index c336a70d7b60..24feebd284ac 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -89,10 +89,12 @@ public Job build(Class jarClass, Configuration conf) throws IOException { WindowingStrategy windowingStrategy = operation.getWindowingStrategy(); KvCoder kvCoder = operation.getKvCoder(); + String reifyStepName = groupByKey.getFullName() + "-Reify"; Coder reifyValueCoder = getReifyValueCoder(kvCoder.getValueCoder(), windowingStrategy); - Graphs.Tag reifyOutputTag = Graphs.Tag.of(new TupleTag(), reifyValueCoder); + Graphs.Tag reifyOutputTag = Graphs.Tag.of( + reifyStepName + ".out", new TupleTag(), reifyValueCoder); Graphs.Step reifyStep = Graphs.Step.of( - groupByKey.getFullName() + "-Reify", + reifyStepName, new ReifyTimestampAndWindowsParDoOperation(options, operation.getWindowingStrategy()), groupByKey.getInputTags(), ImmutableList.of(reifyOutputTag)); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java index 0df365e04505..2b51df5d4020 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java @@ -105,8 +105,9 @@ public Graphs.Tag apply(PValue pValue) { checkState( pValueToTupleTag.containsKey(pValue), String.format("Failed to find TupleTag for pValue: %s.", pValue)); + PCollection pc = (PCollection) pValue; return Graphs.Tag.of( - pValueToTupleTag.get(pValue), ((PCollection) pValue).getCoder()); + pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder()); }}) .toList(); } @@ -116,7 +117,8 @@ public List getOutputTags() { .transform(new Function, PValue>, Graphs.Tag>() { @Override public Graphs.Tag apply(Map.Entry, PValue> entry) { - return Graphs.Tag.of(entry.getKey(), ((PCollection) entry.getValue()).getCoder()); + PCollection pc = (PCollection) entry.getValue(); + return Graphs.Tag.of(pc.getName(), entry.getKey(), pc.getCoder()); }}) .toList(); } From 5905efd3364f2cd27567126508576aac887a1f63 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 2 Aug 2017 21:59:21 +0800 Subject: [PATCH 13/34] mr-runner: hack to get around that ViewAsXXX.expand() return wrong output PValue. --- .../translation/TranslationContext.java | 54 ++++++++++++++----- 1 file changed, 42 insertions(+), 12 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java index 2b51df5d4020..365bdc0312b5 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java @@ -22,13 +22,17 @@ import com.google.common.base.Function; import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import java.util.List; import java.util.Map; import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; @@ -82,6 +86,11 @@ public void setCurrentNode(TransformHierarchy.Node node) { this.currentNode = node; for (Map.Entry, PValue> entry : currentNode.getOutputs().entrySet()) { pValueToTupleTag.put(entry.getValue(), entry.getKey()); + // TODO: this is a hack to get around that ViewAsXXX.expand() return wrong output PValue. + if (node.getTransform() instanceof View.CreatePCollectionView) { + View.CreatePCollectionView view = (View.CreatePCollectionView) node.getTransform(); + pValueToTupleTag.put(view.getView(), view.getView().getTagInternal()); + } } } @@ -98,29 +107,50 @@ public PValue getOutput() { } public List getInputTags() { - return FluentIterable.from(currentNode.getInputs().values()) + Iterable inputs; + if (currentNode.getTransform() instanceof ParDo.MultiOutput) { + ParDo.MultiOutput parDo = (ParDo.MultiOutput) currentNode.getTransform(); + inputs = ImmutableList.builder() + .add(getInput()).addAll(parDo.getSideInputs()).build(); + } else { + inputs = currentNode.getInputs().values(); + } + return FluentIterable.from(inputs) .transform(new Function() { @Override public Graphs.Tag apply(PValue pValue) { checkState( pValueToTupleTag.containsKey(pValue), String.format("Failed to find TupleTag for pValue: %s.", pValue)); - PCollection pc = (PCollection) pValue; - return Graphs.Tag.of( - pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder()); + if (pValue instanceof PCollection) { + PCollection pc = (PCollection) pValue; + return Graphs.Tag.of( + pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder()); + } else { + return Graphs.Tag.of( + pValue.getName(), + pValueToTupleTag.get(pValue), + ((PCollectionView) pValue).getCoderInternal()); + } }}) .toList(); } public List getOutputTags() { - return FluentIterable.from(currentNode.getOutputs().entrySet()) - .transform(new Function, PValue>, Graphs.Tag>() { - @Override - public Graphs.Tag apply(Map.Entry, PValue> entry) { - PCollection pc = (PCollection) entry.getValue(); - return Graphs.Tag.of(pc.getName(), entry.getKey(), pc.getCoder()); - }}) - .toList(); + if (currentNode.getTransform() instanceof View.CreatePCollectionView) { + PCollectionView view = ((View.CreatePCollectionView) currentNode.getTransform()).getView(); + return ImmutableList.of( + Graphs.Tag.of(view.getName(), view.getTagInternal(), view.getCoderInternal())); + } else { + return FluentIterable.from(currentNode.getOutputs().entrySet()) + .transform(new Function, PValue>, Graphs.Tag>() { + @Override + public Graphs.Tag apply(Map.Entry, PValue> entry) { + PCollection pc = (PCollection) entry.getValue(); + return Graphs.Tag.of(pc.getName(), entry.getKey(), pc.getCoder()); + }}) + .toList(); + } } public TupleTag getOnlyOutputTag() { From 40396d758ad21e4938d395007583bf7c61ebdd97 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 8 Aug 2017 11:30:29 +0800 Subject: [PATCH 14/34] mr-runner: support PCollections materialization with multiple MR jobs. --- .../mapreduce/MapReducePipelineOptions.java | 4 + .../runners/mapreduce/MapReduceRunner.java | 15 +- .../translation/BeamInputFormat.java | 85 ++++++--- .../mapreduce/translation/BeamMapper.java | 8 +- .../mapreduce/translation/BeamReducer.java | 20 +-- .../mapreduce/translation/DotfileWriter.java | 53 +++++- .../translation/FileReadOperation.java | 165 ++++++++++++++++++ .../translation/FileWriteOperation.java | 77 ++++++++ .../translation/FlattenTranslator.java | 7 +- .../runners/mapreduce/translation/Graph.java | 79 ++++++--- .../mapreduce/translation/GraphPlanner.java | 67 ++++--- .../runners/mapreduce/translation/Graphs.java | 106 ++++++++--- .../GroupAlsoByWindowsParDoOperation.java | 5 +- .../translation/GroupByKeyTranslator.java | 7 +- .../mapreduce/translation/JobPrototype.java | 93 ++++++---- .../mapreduce/translation/Operation.java | 11 +- .../mapreduce/translation/OutputReceiver.java | 9 +- .../mapreduce/translation/ParDoOperation.java | 43 +++-- .../translation/ParDoTranslator.java | 8 +- .../translation/ReadBoundedTranslator.java | 11 +- ...eifyTimestampAndWindowsParDoOperation.java | 5 +- ...ration.java => ShuffleWriteOperation.java} | 22 +-- ...eadOperation.java => SourceOperation.java} | 4 +- .../translation/TranslationContext.java | 4 +- .../translation/TranslatorRegistry.java | 11 +- .../mapreduce/translation/ViewTranslator.java | 8 +- .../translation/WindowAssignTranslator.java | 7 +- .../translation/GraphPlannerTest.java | 3 +- 28 files changed, 705 insertions(+), 232 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java rename runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/{WriteOperation.java => ShuffleWriteOperation.java} (74%) rename runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/{ReadOperation.java => SourceOperation.java} (94%) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java index c37da586e29f..9224eb606930 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -43,6 +43,10 @@ public interface MapReducePipelineOptions extends PipelineOptions { Class getJarClass(); void setJarClass(Class jarClass); + @Description("The jar class of the user Beam program.") + String getTmpDir(); + void setTmpDir(String tmpDir); + class JarClassInstanceFactory implements DefaultValueFactory> { @Override public Class create(PipelineOptions options) { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index c5626a4de83f..a7e75bbb5e17 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -66,19 +66,30 @@ public PipelineResult run(Pipeline pipeline) { LOG.info(graphConverter.getDotfile()); + Graphs.FusedGraph fusedGraph = new Graphs.FusedGraph(context.getInitGraph()); + LOG.info(DotfileWriter.toDotfile(fusedGraph)); + GraphPlanner planner = new GraphPlanner(); - Graphs.FusedGraph fusedGraph = planner.plan(context.getInitGraph()); + fusedGraph = planner.plan(fusedGraph); LOG.info(DotfileWriter.toDotfile(fusedGraph)); + Configuration config = new Configuration(); + config.set("keep.failed.task.files", "true"); + + fusedGraph.getFusedSteps(); + int stageId = 0; for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { JobPrototype jobPrototype = JobPrototype.create(stageId++, fusedStep, options); + LOG.info("Running job-{}.", stageId); + LOG.info(DotfileWriter.toDotfile(fusedStep)); try { - Job job = jobPrototype.build(options.getJarClass(), new Configuration()); + Job job = jobPrototype.build(options.getJarClass(), config); job.waitForCompletion(true); } catch (Exception e) { Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); } } return null; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java index 8a27a85a1cca..03a88aa32b4c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -21,14 +21,17 @@ import com.google.common.base.Function; import com.google.common.base.Strings; +import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.List; +import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.commons.codec.binary.Base64; @@ -45,10 +48,12 @@ public class BeamInputFormat extends InputFormat { public static final String BEAM_SERIALIZED_BOUNDED_SOURCE = "beam-serialized-bounded-source"; + public static final String BEAM_SERIALIZED_PIPELINE_OPTIONS = "beam-serialized-pipeline-options"; + private static final long DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES = 5 * 1000 * 1000; - private BoundedSource source; - private PipelineOptions options; + private List> sources; + private SerializedPipelineOptions options; public BeamInputFormat() { } @@ -56,21 +61,36 @@ public BeamInputFormat() { @Override public List getSplits(JobContext context) throws IOException, InterruptedException { String serializedBoundedSource = context.getConfiguration().get(BEAM_SERIALIZED_BOUNDED_SOURCE); - if (Strings.isNullOrEmpty(serializedBoundedSource)) { + String serializedPipelineOptions = + context.getConfiguration().get(BEAM_SERIALIZED_PIPELINE_OPTIONS); + if (Strings.isNullOrEmpty(serializedBoundedSource) + || Strings.isNullOrEmpty(serializedPipelineOptions)) { return ImmutableList.of(); } - source = (BoundedSource) SerializableUtils.deserializeFromByteArray( + sources = (List>) SerializableUtils.deserializeFromByteArray( Base64.decodeBase64(serializedBoundedSource), "BoundedSource"); + options = ((SerializedPipelineOptions) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(serializedPipelineOptions), "SerializedPipelineOptions")); + try { - return FluentIterable.from(source.split(DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES, options)) - .transform(new Function, InputSplit>() { + + return FluentIterable.from(sources) + .transformAndConcat(new Function, Iterable>>() { @Override - public InputSplit apply(BoundedSource source) { + public Iterable> apply(BoundedSource input) { try { - return new BeamInputSplit(source.getEstimatedSizeBytes(options)); + return (Iterable>) input.split( + DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES, options.getPipelineOptions()); } catch (Exception e) { + Throwables.throwIfUnchecked(e); throw new RuntimeException(e); } + } + }) + .transform(new Function, InputSplit>() { + @Override + public InputSplit apply(BoundedSource source) { + return new BeamInputSplit(source, options); }}) .toList(); } catch (Exception e) { @@ -81,26 +101,35 @@ public InputSplit apply(BoundedSource source) { @Override public RecordReader createRecordReader( InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { - // TODO: it should initiates from InputSplit. - source = (BoundedSource) SerializableUtils.deserializeFromByteArray( - Base64.decodeBase64(context.getConfiguration().get(BEAM_SERIALIZED_BOUNDED_SOURCE)), - ""); - return new BeamRecordReader<>(source.createReader(options)); + return ((BeamInputSplit) split).createReader(); } - public static class BeamInputSplit extends InputSplit implements Writable { - private long estimatedSizeBytes; + public static class BeamInputSplit extends InputSplit implements Writable { + private BoundedSource boundedSource; + private SerializedPipelineOptions options; public BeamInputSplit() { } - BeamInputSplit(long estimatedSizeBytes) { - this.estimatedSizeBytes = estimatedSizeBytes; + public BeamInputSplit(BoundedSource boundedSource, SerializedPipelineOptions options) { + this.boundedSource = checkNotNull(boundedSource, "boundedSources"); + this.options = checkNotNull(options, "options"); + } + + public BeamRecordReader createReader() throws IOException { + return new BeamRecordReader<>(boundedSource.createReader(options.getPipelineOptions())); } @Override public long getLength() throws IOException, InterruptedException { - return estimatedSizeBytes; + try { + return boundedSource.getEstimatedSizeBytes(options.getPipelineOptions()); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + Throwables.throwIfInstanceOf(e, IOException.class); + Throwables.throwIfInstanceOf(e, InterruptedException.class); + throw new RuntimeException(e); + } } @Override @@ -110,16 +139,28 @@ public String[] getLocations() throws IOException, InterruptedException { @Override public void write(DataOutput out) throws IOException { - out.writeLong(estimatedSizeBytes); + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + SerializableCoder.of(BoundedSource.class).encode(boundedSource, stream); + SerializableCoder.of(SerializedPipelineOptions.class).encode(options, stream); + + byte[] bytes = stream.toByteArray(); + out.writeInt(bytes.length); + out.write(bytes); } @Override public void readFields(DataInput in) throws IOException { - estimatedSizeBytes = in.readLong(); + int length = in.readInt(); + byte[] bytes = new byte[length]; + in.readFully(bytes); + + ByteArrayInputStream inStream = new ByteArrayInputStream(bytes); + boundedSource = SerializableCoder.of(BoundedSource.class).decode(inStream); + options = SerializableCoder.of(SerializedPipelineOptions.class).decode(inStream); } } - private class BeamRecordReader extends RecordReader { + private static class BeamRecordReader extends RecordReader { private final BoundedSource.BoundedReader reader; private boolean started; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index bc52967751a5..d3ebb5c5f133 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -19,17 +19,21 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.TaskInputOutputContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Adapter for executing Beam transforms in {@link Mapper}. */ public class BeamMapper extends Mapper, Object, WindowedValue> { + private static final Logger LOG = LoggerFactory.getLogger(Mapper.class); public static final String BEAM_PAR_DO_OPERATION_MAPPER = "beam-par-do-op-mapper"; @@ -50,7 +54,9 @@ protected void setup( protected void map( Object key, WindowedValue value, - Mapper, Object, WindowedValue>.Context context) { + Mapper, Object, WindowedValue>.Context context) + throws IOException, InterruptedException { + LOG.info("key: {} value: {}.", key, value); operation.process(value); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java index 3490b3b1b95c..a38290427f43 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java @@ -33,8 +33,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.TaskInputOutputContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -42,6 +45,7 @@ */ public class BeamReducer extends Reducer> { + private static final Logger LOG = LoggerFactory.getLogger(Reducer.class); public static final String BEAM_REDUCER_KV_CODER = "beam-reducer-kv-coder"; public static final String BEAM_PAR_DO_OPERATION_REDUCER = "beam-par-do-op-reducer"; @@ -72,7 +76,8 @@ protected void setup( protected void reduce( BytesWritable key, Iterable values, - Reducer>.Context context) { + Reducer>.Context context) + throws InterruptedException, IOException { List decodedValues = Lists.newArrayList(FluentIterable.from(values) .transform(new Function() { @Override @@ -85,15 +90,10 @@ public Object apply(byte[] input) { throw new RuntimeException(e); } }})); - - try { - operation.process( - WindowedValue.valueInGlobalWindow( - KV.of(keyCoder.decode(new ByteArrayInputStream(key.getBytes())), decodedValues))); - } catch (IOException e) { - Throwables.throwIfUnchecked(e); - throw new RuntimeException(e); - } + Object decodedKey = keyCoder.decode(new ByteArrayInputStream(key.getBytes())); + LOG.info("key: {} value: {}.", decodedKey, decodedValues); + operation.process( + WindowedValue.valueInGlobalWindow(KV.of(decodedKey, decodedValues))); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java index 5b0fcd86f818..863c4c96b3a3 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java @@ -17,36 +17,75 @@ */ package org.apache.beam.runners.mapreduce.translation; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.util.Map; +import java.util.Set; + /** * Class that outputs {@link Graph} to dot file. */ public class DotfileWriter { - public static , TagT extends Graph.AbstractTag> + public static String toDotfile(Graphs.FusedGraph fusedGraph) { StringBuilder sb = new StringBuilder(); sb.append("\ndigraph G {\n"); + Map fusedStepToId = Maps.newHashMap(); int i = 0; for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { - sb.append(String.format(" subgraph \"cluster_%d\" {\n", i++)); + String clusterId = String.format("cluster_%d", i++); + sb.append(String.format(" subgraph \"%s\" {\n", clusterId)); + sb.append(String.format(" \"%s\" [shape=point style=invis];\n", clusterId)); + fusedStepToId.put(fusedStep, clusterId); + + Set nodeDefines = Sets.newHashSet(); for (Graphs.Step step : fusedStep.getSteps()) { - sb.append(String.format(" \"%s\" [shape=box];\n", step.getFullName())); - for (Graph.AbstractTag outTag : step.getOutputTags()) { - sb.append(String.format(" \"%s\" [shape=ellipse];\n", outTag)); + nodeDefines.add(String.format(" \"%s\" [shape=box];\n", step.getFullName())); + for (Graph.AbstractTag inTag : fusedStep.getInputTags(step)) { + nodeDefines.add(String.format(" \"%s\" [shape=ellipse];\n", inTag)); } + for (Graph.AbstractTag outTag : fusedStep.getOutputTags(step)) { + nodeDefines.add(String.format(" \"%s\" [shape=ellipse];\n", outTag)); + } + } + for (String str : nodeDefines) { + sb.append(str); } sb.append(String.format(" }")); } for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + // Edges within fused steps. for (Graphs.Step step : fusedStep.getSteps()) { - for (Graph.AbstractTag inTag : step.getInputTags()) { + for (Graph.AbstractTag inTag : fusedStep.getInputTags(step)) { sb.append(String.format(" \"%s\" -> \"%s\";\n", inTag, step)); } - for (Graph.AbstractTag outTag : step.getOutputTags()) { + for (Graph.AbstractTag outTag : fusedStep.getOutputTags(step)) { sb.append(String.format(" \"%s\" -> \"%s\";\n", step, outTag)); } } + + // Edges between sub-graphs. + for (Graphs.Tag inTag : fusedGraph.getInputTags(fusedStep)) { + sb.append(String.format(" \"%s\" -> \"%s\";\n", inTag, fusedStepToId.get(fusedStep))); + } + } + sb.append("}\n"); + return sb.toString(); + } + + public static String toDotfile(Graphs.FusedStep fusedStep) { + StringBuilder sb = new StringBuilder(); + sb.append("\ndigraph G {\n"); + for (Graphs.Step step : fusedStep.getSteps()) { + sb.append(String.format(" \"%s\" [shape=box];\n", step.getFullName())); + for (Graph.AbstractTag inTag : fusedStep.getInputTags(step)) { + sb.append(String.format(" \"%s\" -> \"%s\";\n", inTag, step)); + } + for (Graph.AbstractTag outTag : fusedStep.getOutputTags(step)) { + sb.append(String.format(" \"%s\" -> \"%s\";\n", step, outTag)); + } } sb.append("}\n"); return sb.toString(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java new file mode 100644 index 000000000000..674e30a4112f --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Queue; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; + +/** + * Operation that reads from files. + */ +public class FileReadOperation extends SourceOperation> { + + public FileReadOperation(int producerStageId, String fileName, Coder coder) { + super(new FileBoundedSource<>(producerStageId, fileName, coder)); + } + + private static class FileBoundedSource extends BoundedSource> { + + private final int producerStageId; + private final String fileName; + private final Coder> coder; + + FileBoundedSource(int producerStageId, String fileName, Coder coder) { + this.producerStageId = producerStageId; + this.fileName = checkNotNull(fileName, "fileName"); + checkNotNull(coder, "coder"); + this.coder = WindowedValue.getFullCoder( + coder, WindowingStrategy.globalDefault().getWindowFn().windowCoder()); + + } + + @Override + public List>> split( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + // TODO: support split. + return ImmutableList.of(this); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 0; + } + + @Override + public BoundedReader> createReader(PipelineOptions options) + throws IOException { + Path pattern = new Path(String.format("/tmp/mapreduce/stage-2/%s*", fileName)); + // TODO: use config from the job. + Configuration conf = new Configuration(); + conf.set( + "io.serializations", + "org.apache.hadoop.io.serializer.WritableSerialization," + + "org.apache.hadoop.io.serializer.JavaSerialization"); + FileSystem fs = pattern.getFileSystem(conf); + FileStatus[] files = fs.globStatus(pattern); + Queue readers = new LinkedList<>(); + for (FileStatus f : files) { + readers.add(new SequenceFile.Reader(fs, files[0].getPath(), conf)); + } + return new Reader<>(this, readers, coder); + } + + @Override + public void validate() { + } + + @Override + public Coder> getDefaultOutputCoder() { + return coder; + } + + private static class Reader extends BoundedReader> { + + private final BoundedSource> boundedSource; + private final Queue readers; + private final Coder> coder; + private final BytesWritable value = new BytesWritable(); + + Reader( + BoundedSource> boundedSource, + Queue readers, + Coder> coder) { + this.boundedSource = checkNotNull(boundedSource, "boundedSource"); + this.readers = checkNotNull(readers, "readers"); + this.coder = checkNotNull(coder, "coder"); + } + + @Override + public boolean start() throws IOException { + return advance(); + } + + @Override + public boolean advance() throws IOException { + SequenceFile.Reader reader = readers.peek(); + if (reader == null) { + return false; + } + boolean hasNext = reader.next(NullWritable.get(), value); + if (hasNext) { + return true; + } else { + reader.close(); + readers.remove(reader); + return advance(); + } + } + + @Override + public WindowedValue getCurrent() throws NoSuchElementException { + ByteArrayInputStream inStream = new ByteArrayInputStream(value.getBytes()); + try { + return coder.decode(inStream); + } catch (IOException e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + + @Override + public void close() throws IOException { + } + + @Override + public BoundedSource> getCurrentSource() { + return boundedSource; + } + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java new file mode 100644 index 000000000000..468856ac010e --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.base.Throwables; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; +import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs; + +/** + * Operation that writes to files. + */ +public class FileWriteOperation extends Operation { + + private final String fileName; + private final Coder> coder; + private transient MultipleOutputs mos; + + public FileWriteOperation(String fileName, Coder coder) { + super(0); + this.fileName = checkNotNull(fileName, "fileName"); + checkNotNull(coder, "coder"); + // TODO: should not hard-code windows coder. + this.coder = WindowedValue.getFullCoder( + coder, WindowingStrategy.globalDefault().getWindowFn().windowCoder()); + } + + @Override + public void start(TaskInputOutputContext taskContext) { + this.mos = new MultipleOutputs(taskContext); + } + + @Override + public void process(WindowedValue elem) throws IOException, InterruptedException { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + coder.encode(elem, stream); + + mos.write(fileName, NullWritable.get(), new BytesWritable(stream.toByteArray())); + } + + @Override + public void finish() { + try { + mos.close(); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + + public String getFileName() { + return fileName; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java index 8860caf19d75..b966f2a5b356 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java @@ -28,10 +28,9 @@ public void translateNode(Flatten.PCollections transform, TranslationContext TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); Operation operation = new FlattenOperation(); - context.addInitStep(Graphs.Step.of( - userGraphContext.getStepName(), - operation, + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), operation), userGraphContext.getInputTags(), - userGraphContext.getOutputTags())); + userGraphContext.getOutputTags()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index b6900ccbac19..66e573f60fc4 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -21,19 +21,23 @@ import com.google.common.base.Predicate; import com.google.common.collect.FluentIterable; import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; import com.google.common.graph.ElementOrder; import com.google.common.graph.GraphBuilder; import com.google.common.graph.MutableGraph; +import java.util.Collections; +import java.util.Comparator; import java.util.List; +import java.util.ListIterator; import java.util.Objects; import java.util.Set; /** * Graph that represents a Beam DAG. */ -public class Graph, TagT extends Graph.AbstractTag> { +public class Graph { - private final MutableGraph graph; + public final MutableGraph graph; public Graph() { this.graph = GraphBuilder.directed() @@ -45,16 +49,16 @@ public Graph() { /** * Adds {@link StepT} to this {@link Graph}. */ - public void addStep(StepT step) { + public void addStep(StepT step, List inTags, List outTags) { graph.addNode(step); Set nodes = graph.nodes(); - for (TagT tag : step.getInputTags()) { + for (TagT tag : inTags) { if (!nodes.contains(tag)) { graph.addNode(tag); } graph.putEdge(tag, step); } - for (TagT tag : step.getOutputTags()) { + for (TagT tag : outTags) { if (!nodes.contains(tag)) { graph.addNode(tag); } @@ -93,7 +97,18 @@ public List getSteps() { public boolean apply(Vertex input) { return input instanceof AbstractStep; }})) - .toList(); + .toSortedList(new Comparator() { + @Override + public int compare(StepT left, StepT right) { + if (left.equals(right)) { + return 0; + } else if (com.google.common.graph.Graphs.reachableNodes(graph, left).contains(right)) { + return -1; + } else { + return 1; + } + } + }); } public List getStartSteps() { @@ -106,32 +121,40 @@ public boolean apply(Vertex input) { .toList(); } - public List getInputTags() { - return castToTagList(FluentIterable.from(graph.nodes()) - .filter(new Predicate() { - @Override - public boolean apply(Vertex input) { - return input instanceof AbstractTag && graph.inDegree(input) == 0; - }})) - .toList(); + public StepT getProducer(TagT tag) { + if (contains(tag)) { + return (StepT) Iterables.getOnlyElement(graph.predecessors(tag)); + } else { + return null; + } } - public List getOutputTags() { - return castToTagList(FluentIterable.from(graph.nodes()) - .filter(new Predicate() { - @Override - public boolean apply(Vertex input) { - return input instanceof AbstractTag && graph.outDegree(input) == 0; - }})) - .toList(); + public List getConsumers(TagT tag) { + if (contains(tag)) { + return castToStepList(graph.successors(tag)).toList(); + } else { + return Collections.emptyList(); + } } - public StepT getProducer(TagT tag) { - return (StepT) Iterables.getOnlyElement(graph.predecessors(tag)); + public List getInputTags(StepT step) { + if (contains(step)) { + return castToTagList(graph.predecessors(step)).toList(); + } else { + return Collections.emptyList(); + } } - public List getConsumers(TagT tag) { - return castToStepList(graph.successors(tag)).toList(); + public List getOutputTags(StepT step) { + if (contains(step)) { + return castToTagList(graph.successors(step)).toList(); + } else { + return Collections.emptyList(); + } + } + + private boolean contains(Vertex node) { + return graph.nodes().contains(node); } private FluentIterable castToStepList(Iterable vertices) { @@ -175,9 +198,7 @@ public int hashCode() { interface Vertex { } - public abstract static class AbstractStep implements Vertex { - public abstract List getInputTags(); - public abstract List getOutputTags(); + public abstract static class AbstractStep implements Vertex { } public abstract static class AbstractTag implements Vertex { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index be694e475899..13d215f159db 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -17,8 +17,9 @@ */ package org.apache.beam.runners.mapreduce.translation; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; +import com.google.common.collect.ImmutableList; +import java.util.List; +import org.apache.beam.sdk.values.TupleTag; /** * Class that optimizes the initial graph to a fused graph. @@ -29,31 +30,47 @@ public class GraphPlanner { public GraphPlanner() { } - public Graphs.FusedGraph plan(Graph initGraph) { - Graphs.FusedGraph fusedGraph = new Graphs.FusedGraph(); - // Convert from the list of steps to Graphs. - for (Graphs.Step step : Lists.reverse(initGraph.getSteps())) { - Graphs.FusedStep fusedStep = new Graphs.FusedStep(); - fusedStep.addStep(step); - fusedGraph.addFusedStep(fusedStep); + public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { + // Attach writes/reads on fusion boundaries. + for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + for (Graphs.Tag tag : fusedGraph.getOutputTags(fusedStep)) { + List consumers = fusedGraph.getConsumers(tag); + if (consumers.isEmpty()) { + continue; + } + Graphs.Step producer = fusedStep.getProducer(tag); + if (producer.getOperation() instanceof ViewOperation) { + continue; + } + String tagName = tag.getName(); + String fileName = tagName.replaceAll("[^A-Za-z0-9]", "0"); + fusedStep.addStep( + Graphs.Step.of( + tagName + "/Write", + new FileWriteOperation(fileName, tag.getCoder())), + ImmutableList.of(tag), + ImmutableList.of()); - tryFuse(fusedGraph, fusedStep); + String readStepName = tagName + "/Read"; + Graphs.Tag readOutput = Graphs.Tag.of( + readStepName + ".out", new TupleTag<>(), tag.getCoder()); + for (Graphs.FusedStep consumer : consumers) { + // Re-direct tag to readOutput. + List receivers = consumer.getConsumers(tag); + for (Graphs.Step step : receivers) { + consumer.addEdge(readOutput, step); + } + consumer.removeTag(tag); + consumer.addStep( + Graphs.Step.of( + readStepName, + new FileReadOperation(fusedStep.getStageId(), fileName, tag.getCoder())), + ImmutableList.of(), + ImmutableList.of(readOutput)); + } + } } - return fusedGraph; - } - private void tryFuse(Graphs.FusedGraph fusedGraph, Graphs.FusedStep fusedStep) { - if (fusedStep.getOutputTags().size() != 1) { - return; - } - Graphs.Tag outTag = Iterables.getOnlyElement(fusedStep.getOutputTags()); - if (fusedGraph.getConsumers(outTag).size() != 1) { - return; - } - Graphs.FusedStep consumer = Iterables.getOnlyElement(fusedGraph.getConsumers(outTag)); - if (fusedStep.containsGroupByKey() && consumer.containsGroupByKey()) { - return; - } - fusedGraph.merge(fusedStep, consumer); + return fusedGraph; } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java index cef5afcd0371..97b5441ba52b 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.mapreduce.translation; import com.google.auto.value.AutoValue; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; @@ -32,26 +34,68 @@ private Graphs() {} public static class FusedGraph { private final Graph graph; + private int stageId = 0; public FusedGraph() { this.graph = new Graph<>(); } - public void addFusedStep(FusedStep fusedStep) { - graph.addStep(fusedStep); + public FusedGraph(Graph initGraph) { + this.graph = new Graph<>(); + + // Convert from the list of steps to Graphs. + for (Graphs.Step step : Lists.reverse(initGraph.getSteps())) { + tryFuse(step, initGraph.getInputTags(step), initGraph.getOutputTags(step)); + } + // Remove unused external tags. + for (FusedStep fusedStep : graph.getSteps()) { + for (Tag outTag : graph.getOutputTags(fusedStep)) { + if (graph.getConsumers(outTag).isEmpty()) { + graph.removeTag(outTag); + } + } + } } - public void merge(FusedStep src, FusedStep dest) { - for (Step step : src.steps.getSteps()) { - dest.addStep(step); + public void tryFuse( + Graphs.Step step, + List inTags, + List outTags) { + if (canFuse(step, inTags, outTags)) { + Graphs.Tag outTag = Iterables.getOnlyElement(outTags); + Graphs.FusedStep consumer = Iterables.getOnlyElement(graph.getConsumers(outTag)); + consumer.addStep(step, inTags, outTags); + for (Graphs.Tag in : inTags) { + graph.addEdge(in, consumer); + } + graph.removeTag(outTag); + graph.addEdge(consumer, outTag); + } else { + Graphs.FusedStep newFusedStep = new Graphs.FusedStep(stageId++); + newFusedStep.addStep(step, inTags, outTags); + graph.addStep(newFusedStep, inTags, outTags); + } + } + + private boolean canFuse( + Graphs.Step step, + List inTags, + List outTags) { + if (step.getOperation() instanceof ViewOperation) { + return false; + } + if (outTags.size() != 1) { + return false; } - for (Tag inTag : src.getInputTags()) { - graph.addEdge(inTag, dest); + Graphs.Tag outTag = Iterables.getOnlyElement(outTags); + if (graph.getConsumers(outTag).size() != 1) { + return false; } - for (Tag outTag : src.getOutputTags()) { - graph.addEdge(dest, outTag); + Graphs.FusedStep consumer = Iterables.getOnlyElement(graph.getConsumers(outTag)); + if (consumer.containsGroupByKey() && step.getOperation() instanceof GroupByKeyOperation) { + return false; } - graph.removeStep(src); + return true; } public FusedStep getProducer(Tag tag) { @@ -65,29 +109,41 @@ public List getConsumers(Tag tag) { public List getFusedSteps() { return graph.getSteps(); } + + public List getInputTags(FusedStep fusedStep) { + return graph.getInputTags(fusedStep); + } + + public List getOutputTags(FusedStep fusedStep) { + return graph.getOutputTags(fusedStep); + } } - public static class FusedStep extends Graph.AbstractStep { + public static class FusedStep extends Graph.AbstractStep { + private final int stageId; private final Graph steps; private Step groupByKeyStep; - public FusedStep() { + public FusedStep(int stageid) { + this.stageId = stageid; this.steps = new Graph<>(); this.groupByKeyStep = null; } - @Override - public List getInputTags() { - return steps.getInputTags(); + public int getStageId() { + return stageId; } - @Override - public List getOutputTags() { - return steps.getOutputTags(); + public List getInputTags(Step step) { + return steps.getInputTags(step); + } + + public List getOutputTags(Step step) { + return steps.getOutputTags(step); } - public void addStep(Step step) { - steps.addStep(step); + public void addStep(Step step, List inTags, List outTags) { + steps.addStep(step, inTags, outTags); if (step.getOperation() instanceof GroupByKeyOperation) { groupByKeyStep = step; } @@ -156,18 +212,14 @@ public String toString() { } @AutoValue - public abstract static class Step extends Graph.AbstractStep { + public abstract static class Step extends Graph.AbstractStep { abstract String getFullName(); // TODO: remove public public abstract Operation getOperation(); - public static Step of( - String fullName, - Operation operation, - List inputTags, - List outputTags) { + public static Step of(String fullName, Operation operation) { return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graphs_Step( - inputTags, outputTags, fullName, operation); + fullName, operation); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java index 66cf3b613efa..1ae38da726c1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java @@ -38,8 +38,9 @@ public class GroupAlsoByWindowsParDoOperation extends ParDoOperation { public GroupAlsoByWindowsParDoOperation( PipelineOptions options, WindowingStrategy windowingStrategy, - Coder inputCoder) { - super(options, new TupleTag<>(), ImmutableList.>of(), windowingStrategy); + Coder inputCoder, + Graphs.Tag outTag) { + super(options, outTag.getTupleTag(), ImmutableList.>of(), windowingStrategy); this.inputCoder = checkNotNull(inputCoder, "inputCoder"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java index e87ed09e6008..4c627d70cccc 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupByKeyTranslator.java @@ -37,10 +37,9 @@ public void translateNode(GroupByKey transform, TranslationContext context GroupByKeyOperation groupByKeyOperation = new GroupByKeyOperation<>(windowingStrategy, (KvCoder) inCoder); - context.addInitStep(Graphs.Step.of( - userGraphContext.getStepName(), - groupByKeyOperation, + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), groupByKeyOperation), userGraphContext.getInputTags(), - userGraphContext.getOutputTags())); + userGraphContext.getOutputTags()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 24feebd284ac..1016e22fb97e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -20,13 +20,14 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Set; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.io.BoundedSource; @@ -38,8 +39,12 @@ import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs; +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; /** * Class that translates a {@link Graphs.FusedStep} to a MapReduce job. @@ -53,13 +58,11 @@ public static JobPrototype create( private final int stageId; private final Graphs.FusedStep fusedStep; - private final Set dependencies; private final PipelineOptions options; private JobPrototype(int stageId, Graphs.FusedStep fusedStep, PipelineOptions options) { this.stageId = stageId; this.fusedStep = checkNotNull(fusedStep, "fusedStep"); - this.dependencies = Sets.newHashSet(); this.options = checkNotNull(options, "options"); } @@ -72,19 +75,38 @@ public Job build(Class jarClass, Configuration conf) throws IOException { "org.apache.hadoop.io.serializer.WritableSerialization," + "org.apache.hadoop.io.serializer.JavaSerialization"); + //TODO: config out dir with PipelineOptions. + conf.set( + FileOutputFormat.OUTDIR, + String.format("/tmp/mapreduce/stage-%d", fusedStep.getStageId())); + // Setup BoundedSources in BeamInputFormat. // TODO: support more than one read steps by introducing a composed BeamInputFormat // and a partition operation. - Graphs.Step readStep = Iterables.getOnlyElement(fusedStep.getStartSteps()); - checkState(readStep.getOperation() instanceof ReadOperation); - BoundedSource source = ((ReadOperation) readStep.getOperation()).getSource(); + List readSteps = fusedStep.getStartSteps(); + ArrayList> sources = new ArrayList<>(); + sources.addAll( + FluentIterable.from(readSteps) + .transform(new Function>() { + @Override + public BoundedSource apply(Graphs.Step step) { + checkState(step.getOperation() instanceof SourceOperation); + return ((SourceOperation) step.getOperation()).getSource(); + }}) + .toList()); + conf.set( BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(source))); + Base64.encodeBase64String(SerializableUtils.serializeToByteArray(sources))); + conf.set( + BeamInputFormat.BEAM_SERIALIZED_PIPELINE_OPTIONS, + Base64.encodeBase64String(SerializableUtils.serializeToByteArray( + new SerializedPipelineOptions(options)))); job.setInputFormatClass(BeamInputFormat.class); if (fusedStep.containsGroupByKey()) { Graphs.Step groupByKey = fusedStep.getGroupByKeyStep(); + Graphs.Tag gbkOutTag = Iterables.getOnlyElement(fusedStep.getOutputTags(groupByKey)); GroupByKeyOperation operation = (GroupByKeyOperation) groupByKey.getOperation(); WindowingStrategy windowingStrategy = operation.getWindowingStrategy(); KvCoder kvCoder = operation.getKvCoder(); @@ -92,28 +114,26 @@ public Job build(Class jarClass, Configuration conf) throws IOException { String reifyStepName = groupByKey.getFullName() + "-Reify"; Coder reifyValueCoder = getReifyValueCoder(kvCoder.getValueCoder(), windowingStrategy); Graphs.Tag reifyOutputTag = Graphs.Tag.of( - reifyStepName + ".out", new TupleTag(), reifyValueCoder); + reifyStepName + ".out", new TupleTag<>(), reifyValueCoder); Graphs.Step reifyStep = Graphs.Step.of( reifyStepName, - new ReifyTimestampAndWindowsParDoOperation(options, operation.getWindowingStrategy()), - groupByKey.getInputTags(), - ImmutableList.of(reifyOutputTag)); + new ReifyTimestampAndWindowsParDoOperation( + options, operation.getWindowingStrategy(), reifyOutputTag)); Graphs.Step writeStep = Graphs.Step.of( groupByKey.getFullName() + "-Write", - new WriteOperation(kvCoder.getKeyCoder(), reifyValueCoder), - ImmutableList.of(reifyOutputTag), - Collections.emptyList()); + new ShuffleWriteOperation(kvCoder.getKeyCoder(), reifyValueCoder)); Graphs.Step gabwStep = Graphs.Step.of( groupByKey.getFullName() + "-GroupAlsoByWindows", - new GroupAlsoByWindowsParDoOperation(options, windowingStrategy, kvCoder), - Collections.emptyList(), - groupByKey.getOutputTags()); - - fusedStep.addStep(reifyStep); - fusedStep.addStep(writeStep); - fusedStep.addStep(gabwStep); + new GroupAlsoByWindowsParDoOperation(options, windowingStrategy, kvCoder, gbkOutTag)); + + fusedStep.addStep( + reifyStep, fusedStep.getInputTags(groupByKey), ImmutableList.of(reifyOutputTag)); + fusedStep.addStep( + writeStep, ImmutableList.of(reifyOutputTag), Collections.emptyList()); + fusedStep.addStep( + gabwStep, Collections.emptyList(), ImmutableList.of(gbkOutTag)); fusedStep.removeStep(groupByKey); // Setup BeamReducer @@ -129,8 +149,9 @@ public Job build(Class jarClass, Configuration conf) throws IOException { SerializableUtils.serializeToByteArray(reducerStartStep.getOperation()))); job.setReducerClass(BeamReducer.class); } + // Setup DoFns in BeamMapper. - Graphs.Tag readOutputTag = Iterables.getOnlyElement(readStep.getOutputTags()); + Graphs.Tag readOutputTag = Iterables.getOnlyElement(fusedStep.getOutputTags(readSteps.get(0))); Graphs.Step mapperStartStep = Iterables.getOnlyElement(fusedStep.getConsumers(readOutputTag)); chainOperations(mapperStartStep, fusedStep); @@ -141,18 +162,28 @@ public Job build(Class jarClass, Configuration conf) throws IOException { Base64.encodeBase64String( SerializableUtils.serializeToByteArray(mapperStartStep.getOperation()))); job.setMapperClass(BeamMapper.class); - - job.setOutputFormatClass(NullOutputFormat.class); - + job.setOutputFormatClass(TextOutputFormat.class); + + for (Graphs.Step step : fusedStep.getSteps()) { + if (step.getOperation() instanceof FileWriteOperation) { + FileWriteOperation writeOperation = (FileWriteOperation) step.getOperation(); + //SequenceFileOutputFormat.setOutputPath(job, new Path("/tmp/mapreduce/")); + MultipleOutputs.addNamedOutput( + job, + writeOperation.getFileName(), + SequenceFileOutputFormat.class, + NullWritable.class, BytesWritable.class); + } + } return job; } private void chainOperations(Graphs.Step current, Graphs.FusedStep fusedStep) { Operation operation = current.getOperation(); - List outputTags = current.getOutputTags(); - for (int index = 0; index < outputTags.size(); ++index) { - for (Graphs.Step consumer : fusedStep.getConsumers(outputTags.get(index))) { - operation.attachConsumer(index, consumer.getOperation()); + List outputTags = fusedStep.getOutputTags(current); + for (Graphs.Tag outTag : outputTags) { + for (Graphs.Step consumer : fusedStep.getConsumers(outTag)) { + operation.attachConsumer(outTag.getTupleTag(), consumer.getOperation()); } } for (Graphs.Tag outTag : outputTags) { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java index 187ea79651a0..574f152f780e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java @@ -18,9 +18,11 @@ package org.apache.beam.runners.mapreduce.translation; import com.google.common.collect.ImmutableList; +import java.io.IOException; import java.io.Serializable; import java.util.List; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; import org.apache.hadoop.mapreduce.TaskInputOutputContext; /** @@ -55,7 +57,7 @@ public void start(TaskInputOutputContext taskCon /** * Processes the element. */ - public abstract void process(WindowedValue elem); + public abstract void process(WindowedValue elem) throws IOException, InterruptedException; /** * Finishes this Operation's execution. @@ -80,8 +82,13 @@ public List getOutputReceivers() { /** * Adds an output to this Operation. */ - public void attachConsumer(int outputIndex, Operation consumer) { + public void attachConsumer(TupleTag tupleTag, Operation consumer) { + int outputIndex = getOutputIndex(tupleTag); OutputReceiver fanOut = receivers[outputIndex]; fanOut.addOutput(consumer); } + + protected int getOutputIndex(TupleTag tupleTag) { + return 0; + } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java index 3dab890f8201..b2f1b6d83612 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/OutputReceiver.java @@ -17,7 +17,9 @@ */ package org.apache.beam.runners.mapreduce.translation; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.List; @@ -46,7 +48,12 @@ public List getReceivingOperations() { public void process(WindowedValue elem) { for (Operation out : receivingOperations) { if (out != null) { - out.process(elem); + try { + out.process(elem); + } catch (IOException | InterruptedException e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java index a76773f1011e..c6bf49cbee7c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.mapreduce.translation; import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import java.util.List; import javax.annotation.Nullable; @@ -26,24 +27,23 @@ import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.hadoop.mapreduce.TaskInputOutputContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Operation for ParDo. */ public abstract class ParDoOperation extends Operation { - private static final Logger LOG = LoggerFactory.getLogger(ParDoOperation.class); - protected final SerializedPipelineOptions options; protected final TupleTag mainOutputTag; private final List> sideOutputTags; protected final WindowingStrategy windowingStrategy; + protected DoFnInvoker doFnInvoker; private DoFnRunner fnRunner; public ParDoOperation( @@ -65,6 +65,12 @@ public ParDoOperation( @Override public void start(TaskInputOutputContext taskContext) { + super.start(taskContext); + DoFn doFn = getDoFn(); + // Process user's setup + doFnInvoker = DoFnInvokers.invokerFor(doFn); + doFnInvoker.invokeSetup(); + fnRunner = DoFnRunners.simpleRunner( options.getPipelineOptions(), getDoFn(), @@ -75,7 +81,6 @@ public void start(TaskInputOutputContext taskCon null, windowingStrategy); fnRunner.startBundle(); - super.start(taskContext); } /** @@ -83,14 +88,27 @@ public void start(TaskInputOutputContext taskCon */ @Override public void process(WindowedValue elem) { - LOG.info("elem: {}.", elem); fnRunner.processElement(elem); } @Override public void finish() { - super.finish(); fnRunner.finishBundle(); + doFnInvoker.invokeTeardown(); + super.finish(); + } + + @Override + protected int getOutputIndex(TupleTag tupleTag) { + if (tupleTag == mainOutputTag) { + return 0; + } else { + int sideIndex = sideOutputTags.indexOf(tupleTag); + checkState( + sideIndex >= 0, + String.format("Cannot find index for tuple tag: %s.", tupleTag)); + return sideIndex + 1; + } } protected DoFnRunners.OutputManager createOutputManager() { @@ -100,15 +118,10 @@ protected DoFnRunners.OutputManager createOutputManager() { private class ParDoOutputManager implements DoFnRunners.OutputManager { @Nullable - private OutputReceiver getReceiverOrNull(TupleTag tag) { + private OutputReceiver getReceiverOrNull(TupleTag tupleTag) { List receivers = getOutputReceivers(); - if (tag.equals(mainOutputTag)) { - return receivers.get(0); - } else if (sideOutputTags.contains(tag)) { - return receivers.get(sideOutputTags.indexOf(tag) + 1); - } else { - return null; - } + int outputIndex = getOutputIndex(tupleTag); + return receivers.get(outputIndex); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java index 1a1373a0936e..9bd89fd0af52 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java @@ -36,11 +36,9 @@ public void translateNode( transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), ((PCollection) userGraphContext.getInput()).getWindowingStrategy()); - - context.addInitStep(Graphs.Step.of( - userGraphContext.getStepName(), - operation, + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), operation), userGraphContext.getInputTags(), - userGraphContext.getOutputTags())); + userGraphContext.getOutputTags()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java index 07108275239a..86ee78aacc9d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java @@ -20,18 +20,17 @@ import org.apache.beam.sdk.io.Read; /** - * Translates a {@link Read.Bounded} to a {@link ReadOperation}. + * Translates a {@link Read.Bounded} to a {@link SourceOperation}. */ class ReadBoundedTranslator extends TransformTranslator.Default> { @Override public void translateNode(Read.Bounded transform, TranslationContext context) { TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); - ReadOperation operation = new ReadOperation(transform.getSource()); - context.addInitStep(Graphs.Step.of( - userGraphContext.getStepName(), - operation, + SourceOperation operation = new SourceOperation(transform.getSource()); + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), operation), userGraphContext.getInputTags(), - userGraphContext.getOutputTags())); + userGraphContext.getOutputTags()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java index 83d1af5cdcd4..251828e7f133 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java @@ -34,8 +34,9 @@ public class ReifyTimestampAndWindowsParDoOperation extends ParDoOperation { public ReifyTimestampAndWindowsParDoOperation( PipelineOptions options, - WindowingStrategy windowingStrategy) { - super(options, new TupleTag<>(), ImmutableList.>of(), windowingStrategy); + WindowingStrategy windowingStrategy, + Graphs.Tag outTag) { + super(options, outTag.getTupleTag(), ImmutableList.>of(), windowingStrategy); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ShuffleWriteOperation.java similarity index 74% rename from runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java rename to runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ShuffleWriteOperation.java index 2eb4684f2c8c..782cfef9b444 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WriteOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ShuffleWriteOperation.java @@ -21,6 +21,7 @@ import com.google.common.base.Throwables; import java.io.ByteArrayOutputStream; +import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -30,14 +31,14 @@ /** * {@link Operation} that materializes input for group by key. */ -public class WriteOperation extends Operation { +public class ShuffleWriteOperation extends Operation { private final Coder keyCoder; private final Coder valueCoder; private transient TaskInputOutputContext taskContext; - public WriteOperation(Coder keyCoder, Coder valueCoder) { + public ShuffleWriteOperation(Coder keyCoder, Coder valueCoder) { super(0); this.keyCoder = checkNotNull(keyCoder, "keyCoder"); this.valueCoder = checkNotNull(valueCoder, "valueCoder"); @@ -49,18 +50,13 @@ public void start(TaskInputOutputContext taskCon } @Override - public void process(WindowedValue elem) { + public void process(WindowedValue elem) throws IOException, InterruptedException { KV kv = (KV) elem.getValue(); - try { - ByteArrayOutputStream keyStream = new ByteArrayOutputStream(); - keyCoder.encode(kv.getKey(), keyStream); + ByteArrayOutputStream keyStream = new ByteArrayOutputStream(); + keyCoder.encode(kv.getKey(), keyStream); - ByteArrayOutputStream valueStream = new ByteArrayOutputStream(); - valueCoder.encode(kv.getValue(), valueStream); - taskContext.write(new BytesWritable(keyStream.toByteArray()), valueStream.toByteArray()); - } catch (Exception e) { - Throwables.throwIfUnchecked(e); - throw new RuntimeException(e); - } + ByteArrayOutputStream valueStream = new ByteArrayOutputStream(); + valueCoder.encode(kv.getValue(), valueStream); + taskContext.write(new BytesWritable(keyStream.toByteArray()), valueStream.toByteArray()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceOperation.java similarity index 94% rename from runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java rename to runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceOperation.java index c199dc6a7175..2163f34fc4df 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceOperation.java @@ -25,10 +25,10 @@ /** * A Read.Bounded place holder {@link Operation} during pipeline translation. */ -class ReadOperation extends Operation { +class SourceOperation extends Operation { private final BoundedSource source; - ReadOperation(BoundedSource source) { + SourceOperation(BoundedSource source) { super(1); this.source = checkNotNull(source, "source"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java index 365bdc0312b5..da8ebff2e472 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java @@ -53,8 +53,8 @@ public UserGraphContext getUserGraphContext() { return userGraphContext; } - public void addInitStep(Graphs.Step step) { - initGraph.addStep(step); + public void addInitStep(Graphs.Step step, List inTags, List outTags) { + initGraph.addStep(step, inTags, outTags); } /** diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java index f79260ad63ff..e51d3920fb98 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslatorRegistry.java @@ -26,16 +26,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.Window; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Lookup table mapping PTransform types to associated TransformTranslator implementations. */ public class TranslatorRegistry { - - private static final Logger LOG = LoggerFactory.getLogger(TranslatorRegistry.class); - private static final Map, TransformTranslator> TRANSLATORS = new HashMap<>(); @@ -49,10 +44,6 @@ public class TranslatorRegistry { } public static TransformTranslator getTranslator(PTransform transform) { - TransformTranslator translator = TRANSLATORS.get(transform.getClass()); - if (translator == null) { - LOG.warn("Unsupported operator={}", transform.getClass().getName()); - } - return translator; + return TRANSLATORS.get(transform.getClass()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java index 815ce77c39b3..d5eac73a05eb 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java @@ -32,11 +32,9 @@ public void translateNode( ViewOperation operation = new ViewOperation<>((Coder) transform.getView().getPCollection().getCoder()); - - context.addInitStep(Graphs.Step.of( - userGraphContext.getStepName(), - operation, + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), operation), userGraphContext.getInputTags(), - userGraphContext.getOutputTags())); + userGraphContext.getOutputTags()); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java index 367c375b6129..3908870790c1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/WindowAssignTranslator.java @@ -29,10 +29,9 @@ public void translateNode(Window.Assign transform, TranslationContext context TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); WindowAssignOperation operation = new WindowAssignOperation<>(transform.getWindowFn()); - context.addInitStep(Graphs.Step.of( - userGraphContext.getStepName(), - operation, + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), operation), userGraphContext.getInputTags(), - userGraphContext.getOutputTags())); + userGraphContext.getOutputTags()); } } diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java index cf5262f82347..ac965cb0e938 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java @@ -55,7 +55,8 @@ public void testCombine() throws Exception { p.traverseTopologically(graphConverter); GraphPlanner planner = new GraphPlanner(); - Graphs.FusedGraph fusedGraph = planner.plan(context.getInitGraph()); + Graphs.FusedGraph fusedGraph = new Graphs.FusedGraph(context.getInitGraph()); + fusedGraph = planner.plan(fusedGraph); assertEquals(1, Iterables.size(fusedGraph.getFusedSteps())); assertEquals(3, Iterables.getOnlyElement(fusedGraph.getFusedSteps()).getSteps().size()); From e562a4432d407759876e147fdeb132518a1c9637 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 8 Aug 2017 15:49:04 +0800 Subject: [PATCH 15/34] mr-runner: support multiple SourceOperations by composing and partitioning. --- .../translation/BeamInputFormat.java | 72 ++++++++++++------- .../mapreduce/translation/BeamMapper.java | 3 +- .../translation/FileReadOperation.java | 9 ++- .../translation/FileWriteOperation.java | 8 +-- .../mapreduce/translation/GraphPlanner.java | 51 +++++++++++-- .../runners/mapreduce/translation/Graphs.java | 2 +- .../mapreduce/translation/JobPrototype.java | 28 ++------ .../mapreduce/translation/Operation.java | 1 + .../translation/PartitionOperation.java | 72 +++++++++++++++++++ .../translation/ReadBoundedTranslator.java | 3 +- .../translation/SourceOperation.java | 24 +++++-- .../mapreduce/translation/ViewOperation.java | 59 --------------- .../mapreduce/translation/ViewTranslator.java | 19 +++-- 13 files changed, 224 insertions(+), 127 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java delete mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java index 03a88aa32b4c..23534de17c14 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; @@ -52,7 +53,7 @@ public class BeamInputFormat extends InputFormat { private static final long DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES = 5 * 1000 * 1000; - private List> sources; + private List sources; private SerializedPipelineOptions options; public BeamInputFormat() { @@ -67,30 +68,37 @@ public List getSplits(JobContext context) throws IOException, Interr || Strings.isNullOrEmpty(serializedPipelineOptions)) { return ImmutableList.of(); } - sources = (List>) SerializableUtils.deserializeFromByteArray( - Base64.decodeBase64(serializedBoundedSource), "BoundedSource"); + sources = (List) SerializableUtils.deserializeFromByteArray( + Base64.decodeBase64(serializedBoundedSource), "TaggedSources"); options = ((SerializedPipelineOptions) SerializableUtils.deserializeFromByteArray( Base64.decodeBase64(serializedPipelineOptions), "SerializedPipelineOptions")); try { return FluentIterable.from(sources) - .transformAndConcat(new Function, Iterable>>() { + .transformAndConcat( + new Function>() { + @Override + public Iterable apply( + final SourceOperation.TaggedSource taggedSource) { + try { + return FluentIterable.from(taggedSource.getSource().split( + DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES, options.getPipelineOptions())) + .transform(new Function, SourceOperation.TaggedSource>() { + @Override + public SourceOperation.TaggedSource apply(BoundedSource input) { + return SourceOperation.TaggedSource.of(input, taggedSource.getTag()); + }}); + } catch (Exception e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + }) + .transform(new Function() { @Override - public Iterable> apply(BoundedSource input) { - try { - return (Iterable>) input.split( - DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES, options.getPipelineOptions()); - } catch (Exception e) { - Throwables.throwIfUnchecked(e); - throw new RuntimeException(e); - } - } - }) - .transform(new Function, InputSplit>() { - @Override - public InputSplit apply(BoundedSource source) { - return new BeamInputSplit(source, options); + public InputSplit apply(SourceOperation.TaggedSource taggedSource) { + return new BeamInputSplit(taggedSource.getSource(), options, taggedSource.getTag()); }}) .toList(); } catch (Exception e) { @@ -107,17 +115,23 @@ public RecordReader createRecordReader( public static class BeamInputSplit extends InputSplit implements Writable { private BoundedSource boundedSource; private SerializedPipelineOptions options; + private TupleTag tupleTag; public BeamInputSplit() { } - public BeamInputSplit(BoundedSource boundedSource, SerializedPipelineOptions options) { + public BeamInputSplit( + BoundedSource boundedSource, + SerializedPipelineOptions options, + TupleTag tupleTag) { this.boundedSource = checkNotNull(boundedSource, "boundedSources"); this.options = checkNotNull(options, "options"); + this.tupleTag = checkNotNull(tupleTag, "tupleTag"); } public BeamRecordReader createReader() throws IOException { - return new BeamRecordReader<>(boundedSource.createReader(options.getPipelineOptions())); + return new BeamRecordReader<>( + boundedSource.createReader(options.getPipelineOptions()), tupleTag); } @Override @@ -142,6 +156,7 @@ public void write(DataOutput out) throws IOException { ByteArrayOutputStream stream = new ByteArrayOutputStream(); SerializableCoder.of(BoundedSource.class).encode(boundedSource, stream); SerializableCoder.of(SerializedPipelineOptions.class).encode(options, stream); + SerializableCoder.of(TupleTag.class).encode(tupleTag, stream); byte[] bytes = stream.toByteArray(); out.writeInt(bytes.length); @@ -157,16 +172,19 @@ public void readFields(DataInput in) throws IOException { ByteArrayInputStream inStream = new ByteArrayInputStream(bytes); boundedSource = SerializableCoder.of(BoundedSource.class).decode(inStream); options = SerializableCoder.of(SerializedPipelineOptions.class).decode(inStream); + tupleTag = SerializableCoder.of(TupleTag.class).decode(inStream); } } private static class BeamRecordReader extends RecordReader { private final BoundedSource.BoundedReader reader; + private TupleTag tupleTag; private boolean started; - public BeamRecordReader(BoundedSource.BoundedReader reader) { + public BeamRecordReader(BoundedSource.BoundedReader reader, TupleTag tupleTag) { this.reader = checkNotNull(reader, "reader"); + this.tupleTag = checkNotNull(tupleTag, "tupleTag"); this.started = false; } @@ -187,13 +205,19 @@ public boolean nextKeyValue() throws IOException, InterruptedException { @Override public Object getCurrentKey() throws IOException, InterruptedException { - return "global"; + return tupleTag; } @Override public Object getCurrentValue() throws IOException, InterruptedException { - return WindowedValue.timestampedValueInGlobalWindow( - reader.getCurrent(), reader.getCurrentTimestamp()); + // TODO: this is a hack to handle that reads from materialized PCollections + // already return WindowedValue. + if (reader.getCurrent() instanceof WindowedValue) { + return reader.getCurrent(); + } else { + return WindowedValue.timestampedValueInGlobalWindow( + reader.getCurrent(), reader.getCurrentTimestamp()); + } } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index d3ebb5c5f133..b03236fd02bd 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.TaskInputOutputContext; @@ -57,7 +58,7 @@ protected void map( Mapper, Object, WindowedValue>.Context context) throws IOException, InterruptedException { LOG.info("key: {} value: {}.", key, value); - operation.process(value); + operation.process(WindowedValue.valueInGlobalWindow(KV.of(key, value))); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java index 674e30a4112f..6bd893a09149 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -45,8 +46,12 @@ */ public class FileReadOperation extends SourceOperation> { - public FileReadOperation(int producerStageId, String fileName, Coder coder) { - super(new FileBoundedSource<>(producerStageId, fileName, coder)); + public FileReadOperation( + int producerStageId, + String fileName, + Coder coder, + TupleTag tupleTag) { + super(new FileBoundedSource<>(producerStageId, fileName, coder), tupleTag); } private static class FileBoundedSource extends BoundedSource> { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java index 468856ac010e..af2e134cea79 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileWriteOperation.java @@ -24,7 +24,6 @@ import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.TaskInputOutputContext; @@ -39,13 +38,10 @@ public class FileWriteOperation extends Operation { private final Coder> coder; private transient MultipleOutputs mos; - public FileWriteOperation(String fileName, Coder coder) { + public FileWriteOperation(String fileName, Coder> coder) { super(0); this.fileName = checkNotNull(fileName, "fileName"); - checkNotNull(coder, "coder"); - // TODO: should not hard-code windows coder. - this.coder = WindowedValue.getFullCoder( - coder, WindowingStrategy.globalDefault().getWindowFn().windowCoder()); + this.coder = checkNotNull(coder, "coder"); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 13d215f159db..7c76823b1127 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -17,9 +17,20 @@ */ package org.apache.beam.runners.mapreduce.translation; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.util.ArrayList; import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** * Class that optimizes the initial graph to a fused graph. @@ -39,21 +50,26 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { continue; } Graphs.Step producer = fusedStep.getProducer(tag); - if (producer.getOperation() instanceof ViewOperation) { + if (producer.getOperation() instanceof FileWriteOperation) { continue; } String tagName = tag.getName(); String fileName = tagName.replaceAll("[^A-Za-z0-9]", "0"); + + // TODO: should not hard-code windows coder. + WindowedValue.WindowedValueCoder writeValueCoder = WindowedValue.getFullCoder( + tag.getCoder(), WindowingStrategy.globalDefault().getWindowFn().windowCoder()); + fusedStep.addStep( Graphs.Step.of( tagName + "/Write", - new FileWriteOperation(fileName, tag.getCoder())), + new FileWriteOperation(fileName, writeValueCoder)), ImmutableList.of(tag), ImmutableList.of()); String readStepName = tagName + "/Read"; Graphs.Tag readOutput = Graphs.Tag.of( - readStepName + ".out", new TupleTag<>(), tag.getCoder()); + readStepName + ".out", tag.getTupleTag(), tag.getCoder()); for (Graphs.FusedStep consumer : consumers) { // Re-direct tag to readOutput. List receivers = consumer.getConsumers(tag); @@ -64,13 +80,40 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { consumer.addStep( Graphs.Step.of( readStepName, - new FileReadOperation(fusedStep.getStageId(), fileName, tag.getCoder())), + new FileReadOperation( + fusedStep.getStageId(), fileName, tag.getCoder(), tag.getTupleTag())), ImmutableList.of(), ImmutableList.of(readOutput)); } } } + // Insert PartitionOperation + for (final Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + List readSteps = fusedStep.getStartSteps(); + + List sources = new ArrayList<>(); + List readOutTags = new ArrayList<>(); + List> readOutTupleTags = new ArrayList<>(); + StringBuilder partitionStepName = new StringBuilder(); + for (Graphs.Step step : readSteps) { + checkState(step.getOperation() instanceof SourceOperation); + sources.add(((SourceOperation) step.getOperation()).getTaggedSource()); + Graphs.Tag tag = Iterables.getOnlyElement(fusedStep.getOutputTags(step)); + readOutTags.add(tag); + readOutTupleTags.add(tag.getTupleTag()); + partitionStepName.append(step.getFullName()); + + fusedStep.removeStep(step); + } + if (partitionStepName.length() > 0) { + partitionStepName.deleteCharAt(partitionStepName.length() - 1); + } + + Graphs.Step partitionStep = + Graphs.Step.of(partitionStepName.toString(), new PartitionOperation(sources)); + fusedStep.addStep(partitionStep, ImmutableList.of(), readOutTags); + } return fusedGraph; } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java index 97b5441ba52b..9743d0980336 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java @@ -81,7 +81,7 @@ private boolean canFuse( Graphs.Step step, List inTags, List outTags) { - if (step.getOperation() instanceof ViewOperation) { + if (step.getOperation() instanceof FileWriteOperation) { return false; } if (outTags.size() != 1) { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 1016e22fb97e..677f3a727bf0 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -20,8 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import com.google.common.base.Function; -import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import java.io.IOException; @@ -30,7 +28,6 @@ import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; @@ -81,23 +78,14 @@ public Job build(Class jarClass, Configuration conf) throws IOException { String.format("/tmp/mapreduce/stage-%d", fusedStep.getStageId())); // Setup BoundedSources in BeamInputFormat. - // TODO: support more than one read steps by introducing a composed BeamInputFormat - // and a partition operation. - List readSteps = fusedStep.getStartSteps(); - ArrayList> sources = new ArrayList<>(); - sources.addAll( - FluentIterable.from(readSteps) - .transform(new Function>() { - @Override - public BoundedSource apply(Graphs.Step step) { - checkState(step.getOperation() instanceof SourceOperation); - return ((SourceOperation) step.getOperation()).getSource(); - }}) - .toList()); + Graphs.Step startStep = Iterables.getOnlyElement(fusedStep.getStartSteps()); + checkState(startStep.getOperation() instanceof PartitionOperation); + PartitionOperation partitionOperation = (PartitionOperation) startStep.getOperation(); conf.set( BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, - Base64.encodeBase64String(SerializableUtils.serializeToByteArray(sources))); + Base64.encodeBase64String(SerializableUtils.serializeToByteArray( + new ArrayList<>(partitionOperation.getTaggedSources())))); conf.set( BeamInputFormat.BEAM_SERIALIZED_PIPELINE_OPTIONS, Base64.encodeBase64String(SerializableUtils.serializeToByteArray( @@ -151,16 +139,14 @@ public BoundedSource apply(Graphs.Step step) { } // Setup DoFns in BeamMapper. - Graphs.Tag readOutputTag = Iterables.getOnlyElement(fusedStep.getOutputTags(readSteps.get(0))); - Graphs.Step mapperStartStep = Iterables.getOnlyElement(fusedStep.getConsumers(readOutputTag)); - chainOperations(mapperStartStep, fusedStep); + chainOperations(startStep, fusedStep); job.setMapOutputKeyClass(BytesWritable.class); job.setMapOutputValueClass(byte[].class); conf.set( BeamMapper.BEAM_PAR_DO_OPERATION_MAPPER, Base64.encodeBase64String( - SerializableUtils.serializeToByteArray(mapperStartStep.getOperation()))); + SerializableUtils.serializeToByteArray(startStep.getOperation()))); job.setMapperClass(BeamMapper.class); job.setOutputFormatClass(TextOutputFormat.class); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java index 574f152f780e..7504e1cf3030 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java @@ -76,6 +76,7 @@ public void finish() { } public List getOutputReceivers() { + // TODO: avoid allocating objects for each output emit. return ImmutableList.copyOf(receivers); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java new file mode 100644 index 000000000000..b8aefd63f99d --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; + +/** + * Operation that partitions input elements based on their {@link TupleTag} keys. + */ +public class PartitionOperation extends Operation, Object>> { + + private final List sources; + private final List> tupleTags; + + public PartitionOperation(List sources) { + super(sources.size()); + this.sources = checkNotNull(sources, "sources"); + this.tupleTags = FluentIterable.from(sources) + .transform(new Function>() { + @Override + public TupleTag apply(SourceOperation.TaggedSource input) { + return input.getTag(); + }}) + .toList(); + } + + public List getTaggedSources() { + return sources; + } + + @Override + public void process(WindowedValue, Object>> elem) throws IOException, + InterruptedException { + TupleTag tupleTag = elem.getValue().getKey(); + int outputIndex = getOutputIndex(tupleTag); + OutputReceiver receiver = getOutputReceivers().get(outputIndex); + receiver.process((WindowedValue) elem.getValue().getValue()); + } + + @Override + protected int getOutputIndex(TupleTag tupleTag) { + int index = tupleTags.indexOf(tupleTag); + checkState( + index >= 0, + String.format("Cannot find index for tuple tag: %s.", tupleTag)); + return index; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java index 86ee78aacc9d..e93986bdf644 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java @@ -27,7 +27,8 @@ class ReadBoundedTranslator extends TransformTranslator.Default extends Operation { - private final BoundedSource source; + private final TaggedSource source; - SourceOperation(BoundedSource source) { + SourceOperation(BoundedSource boundedSource, TupleTag tupleTag) { super(1); - this.source = checkNotNull(source, "source"); + checkNotNull(boundedSource, "boundedSource"); + checkNotNull(tupleTag, "tupleTag"); + this.source = TaggedSource.of(boundedSource, tupleTag); } @Override @@ -39,7 +44,18 @@ public void process(WindowedValue elem) { String.format("%s should not in execution graph.", this.getClass().getSimpleName())); } - BoundedSource getSource() { + TaggedSource getTaggedSource() { return source; } + + @AutoValue + abstract static class TaggedSource implements Serializable { + abstract BoundedSource getSource(); + abstract TupleTag getTag(); + + static TaggedSource of(BoundedSource boundedSource, TupleTag tupleTag) { + return new org.apache.beam.runners.mapreduce.translation + .AutoValue_SourceOperation_TaggedSource(boundedSource, tupleTag); + } + } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java deleted file mode 100644 index 093f00e2193f..000000000000 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewOperation.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.mapreduce.translation; - -import static com.google.common.base.Preconditions.checkNotNull; - -import com.google.common.base.Throwables; -import java.io.ByteArrayOutputStream; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapreduce.TaskInputOutputContext; - -/** - * {@link Operation} that materializes views. - */ -public class ViewOperation extends Operation { - - private final Coder> valueCoder; - - private transient TaskInputOutputContext taskContext; - - public ViewOperation(Coder> valueCoder) { - super(0); - this.valueCoder = checkNotNull(valueCoder, "valueCoder"); - } - - @Override - public void start(TaskInputOutputContext taskContext) { - this.taskContext = checkNotNull(taskContext, "taskContext"); - } - - @Override - public void process(WindowedValue elem) { - try { - ByteArrayOutputStream valueStream = new ByteArrayOutputStream(); - valueCoder.encode(elem, valueStream); - taskContext.write(new BytesWritable("view".getBytes()), valueStream.toByteArray()); - } catch (Exception e) { - Throwables.throwIfUnchecked(e); - throw new RuntimeException(e); - } - } -} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java index d5eac73a05eb..dfa18c8bf1cc 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java @@ -17,11 +17,14 @@ */ package org.apache.beam.runners.mapreduce.translation; -import org.apache.beam.sdk.coders.Coder; +import com.google.common.collect.Iterables; import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; /** - * Translates a {@link View.CreatePCollectionView} to a {@link ViewOperation}. + * Translates a {@link View.CreatePCollectionView} to a {@link FileWriteOperation}. */ public class ViewTranslator extends TransformTranslator.Default> { @@ -30,8 +33,16 @@ public void translateNode( View.CreatePCollectionView transform, TranslationContext context) { TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); - ViewOperation operation = - new ViewOperation<>((Coder) transform.getView().getPCollection().getCoder()); + PCollection inPCollection = transform.getView().getPCollection(); + WindowingStrategy windowingStrategy = inPCollection.getWindowingStrategy(); + + Graphs.Tag outTag = Iterables.getOnlyElement(userGraphContext.getOutputTags()); + String fileName = outTag.getName().replaceAll("[^A-Za-z0-9]", "0"); + + FileWriteOperation operation = new FileWriteOperation<>( + fileName, + WindowedValue.getFullCoder( + inPCollection.getCoder(), windowingStrategy.getWindowFn().windowCoder())); context.addInitStep( Graphs.Step.of(userGraphContext.getStepName(), operation), userGraphContext.getInputTags(), From 0ebd14c446421bdb29a95ae231975875b4532031 Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 8 Aug 2017 17:38:58 +0800 Subject: [PATCH 16/34] mr-runner: support side inputs by reading in all views contents. --- .../translation/FileReadOperation.java | 2 +- .../translation/FileSideInputReader.java | 128 ++++++++++++++++++ .../runners/mapreduce/translation/Graphs.java | 3 +- .../GroupAlsoByWindowsParDoOperation.java | 3 +- .../translation/NormalParDoOperation.java | 3 +- .../mapreduce/translation/ParDoOperation.java | 6 +- .../translation/ParDoTranslator.java | 1 + ...eifyTimestampAndWindowsParDoOperation.java | 4 +- .../translation/TranslationContext.java | 25 ++++ 9 files changed, 168 insertions(+), 7 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java index 6bd893a09149..70263c3e74d1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java @@ -95,7 +95,7 @@ public BoundedReader> createReader(PipelineOptions options) FileStatus[] files = fs.globStatus(pattern); Queue readers = new LinkedList<>(); for (FileStatus f : files) { - readers.add(new SequenceFile.Reader(fs, files[0].getPath(), conf)); + readers.add(new SequenceFile.Reader(fs, f.getPath(), conf)); } return new Reader<>(this, readers, coder); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java new file mode 100644 index 000000000000..18bff2a81157 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.SideInputReader; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; + +/** + * Files based {@link SideInputReader}. + */ +public class FileSideInputReader implements SideInputReader { + + private final Map, String> tupleTagToFileName; + private final Map, Coder> tupleTagToCoder; + + public FileSideInputReader(List sideInputTags) { + this.tupleTagToFileName = Maps.newHashMap(); + this.tupleTagToCoder = Maps.newHashMap(); + for (Graphs.Tag tag : sideInputTags) { + tupleTagToFileName.put(tag.getTupleTag(), toFileName(tag.getName())); + tupleTagToCoder.put(tag.getTupleTag(), tag.getCoder()); + } + } + + @Nullable + @Override + public T get(PCollectionView view, BoundedWindow window) { + String fileName = tupleTagToFileName.get(view.getTagInternal()); + IterableCoder> coder = + (IterableCoder>) tupleTagToCoder.get(view.getTagInternal()); + Coder> elemCoder = coder.getElemCoder(); + + final BoundedWindow sideInputWindow = + view.getWindowMappingFn().getSideInputWindow(window); + + Path pattern = new Path(String.format("/tmp/mapreduce/stage-1/%s*", fileName)); + Configuration conf = new Configuration(); + conf.set( + "io.serializations", + "org.apache.hadoop.io.serializer.WritableSerialization," + + "org.apache.hadoop.io.serializer.JavaSerialization"); + try { + FileSystem fs; + fs = pattern.getFileSystem(conf); + FileStatus[] files = fs.globStatus(pattern); + SequenceFile.Reader reader = new SequenceFile.Reader(fs, files[0].getPath(), conf); + + List> availableSideInputs = new ArrayList<>(); + BytesWritable value = new BytesWritable(); + while (reader.next(NullWritable.get(), value)) { + ByteArrayInputStream inStream = new ByteArrayInputStream(value.getBytes()); + availableSideInputs.add(elemCoder.decode(inStream)); + } + Iterable> sideInputForWindow = + Iterables.filter(availableSideInputs, new Predicate>() { + @Override + public boolean apply(@Nullable WindowedValue sideInputCandidate) { + if (sideInputCandidate == null) { + return false; + } + // first match of a sideInputWindow to the elementWindow is good enough. + for (BoundedWindow sideInputCandidateWindow: sideInputCandidate.getWindows()) { + if (sideInputCandidateWindow.equals(sideInputWindow)) { + return true; + } + } + // no match found. + return false; + } + }); + return view.getViewFn().apply(sideInputForWindow); + } catch (IOException e) { + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); + } + } + + @Override + public boolean contains(PCollectionView view) { + return tupleTagToFileName.containsKey(view.getTagInternal()); + } + + @Override + public boolean isEmpty() { + return tupleTagToFileName.isEmpty(); + } + + private String toFileName(String tagName) { + return tagName.replaceAll("[^A-Za-z0-9]", "0"); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java index 9743d0980336..b2f793a3cfb1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java @@ -20,6 +20,7 @@ import com.google.auto.value.AutoValue; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import java.io.Serializable; import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; @@ -229,7 +230,7 @@ public String toString() { } @AutoValue - public abstract static class Tag extends Graph.AbstractTag { + public abstract static class Tag extends Graph.AbstractTag implements Serializable { abstract String getName(); abstract TupleTag getTupleTag(); abstract Coder getCoder(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java index 1ae38da726c1..471c7f50d043 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java @@ -40,7 +40,8 @@ public GroupAlsoByWindowsParDoOperation( WindowingStrategy windowingStrategy, Coder inputCoder, Graphs.Tag outTag) { - super(options, outTag.getTupleTag(), ImmutableList.>of(), windowingStrategy); + super(options, outTag.getTupleTag(), ImmutableList.>of(), + ImmutableList.of(), windowingStrategy); this.inputCoder = checkNotNull(inputCoder, "inputCoder"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java index fd1b5280ddc7..58a7d6dfd152 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java @@ -37,8 +37,9 @@ public NormalParDoOperation( PipelineOptions options, TupleTag mainOutputTag, List> sideOutputTags, + List sideInputTags, WindowingStrategy windowingStrategy) { - super(options, mainOutputTag, sideOutputTags, windowingStrategy); + super(options, mainOutputTag, sideOutputTags, sideInputTags, windowingStrategy); this.doFn = checkNotNull(doFn, "doFn"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java index c6bf49cbee7c..020bfbe36787 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -24,7 +24,6 @@ import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; -import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -41,6 +40,7 @@ public abstract class ParDoOperation extends Operation protected final SerializedPipelineOptions options; protected final TupleTag mainOutputTag; private final List> sideOutputTags; + private final List sideInputTags; protected final WindowingStrategy windowingStrategy; protected DoFnInvoker doFnInvoker; @@ -50,11 +50,13 @@ public ParDoOperation( PipelineOptions options, TupleTag mainOutputTag, List> sideOutputTags, + List sideInputTags, WindowingStrategy windowingStrategy) { super(1 + sideOutputTags.size()); this.options = new SerializedPipelineOptions(checkNotNull(options, "options")); this.mainOutputTag = checkNotNull(mainOutputTag, "mainOutputTag"); this.sideOutputTags = checkNotNull(sideOutputTags, "sideOutputTags"); + this.sideInputTags = checkNotNull(sideInputTags, "sideInputTags"); this.windowingStrategy = checkNotNull(windowingStrategy, "windowingStrategy"); } @@ -74,7 +76,7 @@ public void start(TaskInputOutputContext taskCon fnRunner = DoFnRunners.simpleRunner( options.getPipelineOptions(), getDoFn(), - NullSideInputReader.empty(), + new FileSideInputReader(sideInputTags), createOutputManager(), mainOutputTag, sideOutputTags, diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java index 9bd89fd0af52..ae23f7188b55 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java @@ -35,6 +35,7 @@ public void translateNode( userGraphContext.getOptions(), transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), + userGraphContext.getSideInputTags(), ((PCollection) userGraphContext.getInput()).getWindowingStrategy()); context.addInitStep( Graphs.Step.of(userGraphContext.getStepName(), operation), diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java index 251828e7f133..459e93b461d3 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.mapreduce.translation; import com.google.common.collect.ImmutableList; +import java.util.Collections; import java.util.List; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -36,7 +37,8 @@ public ReifyTimestampAndWindowsParDoOperation( PipelineOptions options, WindowingStrategy windowingStrategy, Graphs.Tag outTag) { - super(options, outTag.getTupleTag(), ImmutableList.>of(), windowingStrategy); + super(options, outTag.getTupleTag(), ImmutableList.>of(), + ImmutableList.of(), windowingStrategy); } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java index da8ebff2e472..fd6c0ba376a1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java @@ -136,6 +136,31 @@ public Graphs.Tag apply(PValue pValue) { .toList(); } + public List getSideInputTags() { + if (!(currentNode.getTransform() instanceof ParDo.MultiOutput)) { + return ImmutableList.of(); + } + return FluentIterable.from(((ParDo.MultiOutput) currentNode.getTransform()).getSideInputs()) + .transform(new Function() { + @Override + public Graphs.Tag apply(PValue pValue) { + checkState( + pValueToTupleTag.containsKey(pValue), + String.format("Failed to find TupleTag for pValue: %s.", pValue)); + if (pValue instanceof PCollection) { + PCollection pc = (PCollection) pValue; + return Graphs.Tag.of( + pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder()); + } else { + return Graphs.Tag.of( + pValue.getName(), + pValueToTupleTag.get(pValue), + ((PCollectionView) pValue).getCoderInternal()); + } + }}) + .toList(); + } + public List getOutputTags() { if (currentNode.getTransform() instanceof View.CreatePCollectionView) { PCollectionView view = ((View.CreatePCollectionView) currentNode.getTransform()).getView(); From b87ae78b5e9c204e03c01b986abf8dc185b6a9ef Mon Sep 17 00:00:00 2001 From: Pei He Date: Tue, 8 Aug 2017 22:07:12 +0800 Subject: [PATCH 17/34] mr-runner: setup file paths for read and write sides of materialization. --- runners/map-reduce/pom.xml | 5 ++ .../mapreduce/MapReducePipelineOptions.java | 7 ++- .../runners/mapreduce/MapReduceRunner.java | 8 +-- .../translation/BeamInputFormat.java | 20 +++---- .../translation/ConfigurationUtils.java | 52 ++++++++++++++++++ .../translation/FileReadOperation.java | 41 ++++++++------ .../translation/FileSideInputReader.java | 41 ++++++-------- .../mapreduce/translation/GraphPlanner.java | 55 ++++++++++++++----- .../GroupAlsoByWindowsParDoOperation.java | 1 - .../mapreduce/translation/JobPrototype.java | 29 +++++++--- .../mapreduce/translation/Operation.java | 6 ++ .../mapreduce/translation/ParDoOperation.java | 26 ++++++++- .../translation/PartitionOperation.java | 20 +++---- .../translation/ReadBoundedTranslator.java | 6 +- ...ourceOperation.java => ReadOperation.java} | 20 +++---- ...eifyTimestampAndWindowsParDoOperation.java | 2 - .../SerializableConfiguration.java | 52 ++++++++++++++++++ .../translation/SourceReadOperation.java | 42 ++++++++++++++ .../mapreduce/translation/ViewTranslator.java | 2 +- .../translation/GraphPlannerTest.java | 2 +- 20 files changed, 321 insertions(+), 116 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java rename runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/{SourceOperation.java => ReadOperation.java} (76%) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializableConfiguration.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml index 06e5227380d7..e85803104c92 100644 --- a/runners/map-reduce/pom.xml +++ b/runners/map-reduce/pom.xml @@ -113,6 +113,11 @@ beam-runners-core-construction-java + + org.apache.beam + beam-sdks-java-io-hadoop-file-system + + com.google.auto.service diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java index 9224eb606930..cfbc006d4157 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -43,9 +43,10 @@ public interface MapReducePipelineOptions extends PipelineOptions { Class getJarClass(); void setJarClass(Class jarClass); - @Description("The jar class of the user Beam program.") - String getTmpDir(); - void setTmpDir(String tmpDir); + @Description("The directory for files output.") + @Default.String("/tmp/mapreduce/") + String getFileOutputDir(); + void setFileOutputDir(String fileOutputDir); class JarClassInstanceFactory implements DefaultValueFactory> { @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index a7e75bbb5e17..3f7680823545 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -69,18 +69,18 @@ public PipelineResult run(Pipeline pipeline) { Graphs.FusedGraph fusedGraph = new Graphs.FusedGraph(context.getInitGraph()); LOG.info(DotfileWriter.toDotfile(fusedGraph)); - GraphPlanner planner = new GraphPlanner(); + GraphPlanner planner = new GraphPlanner(options); fusedGraph = planner.plan(fusedGraph); LOG.info(DotfileWriter.toDotfile(fusedGraph)); - Configuration config = new Configuration(); - config.set("keep.failed.task.files", "true"); - fusedGraph.getFusedSteps(); int stageId = 0; for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + Configuration config = new Configuration(); + config.set("keep.failed.task.files", "true"); + JobPrototype jobPrototype = JobPrototype.create(stageId++, fusedStep, options); LOG.info("Running job-{}.", stageId); LOG.info(DotfileWriter.toDotfile(fusedStep)); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java index 23534de17c14..10d9ada80ba5 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -53,7 +53,7 @@ public class BeamInputFormat extends InputFormat { private static final long DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES = 5 * 1000 * 1000; - private List sources; + private List sources; private SerializedPipelineOptions options; public BeamInputFormat() { @@ -68,7 +68,7 @@ public List getSplits(JobContext context) throws IOException, Interr || Strings.isNullOrEmpty(serializedPipelineOptions)) { return ImmutableList.of(); } - sources = (List) SerializableUtils.deserializeFromByteArray( + sources = (List) SerializableUtils.deserializeFromByteArray( Base64.decodeBase64(serializedBoundedSource), "TaggedSources"); options = ((SerializedPipelineOptions) SerializableUtils.deserializeFromByteArray( Base64.decodeBase64(serializedPipelineOptions), "SerializedPipelineOptions")); @@ -77,17 +77,17 @@ public List getSplits(JobContext context) throws IOException, Interr return FluentIterable.from(sources) .transformAndConcat( - new Function>() { + new Function>() { @Override - public Iterable apply( - final SourceOperation.TaggedSource taggedSource) { + public Iterable apply( + final ReadOperation.TaggedSource taggedSource) { try { return FluentIterable.from(taggedSource.getSource().split( DEFAULT_DESIRED_BUNDLE_SIZE_SIZE_BYTES, options.getPipelineOptions())) - .transform(new Function, SourceOperation.TaggedSource>() { + .transform(new Function, ReadOperation.TaggedSource>() { @Override - public SourceOperation.TaggedSource apply(BoundedSource input) { - return SourceOperation.TaggedSource.of(input, taggedSource.getTag()); + public ReadOperation.TaggedSource apply(BoundedSource input) { + return ReadOperation.TaggedSource.of(input, taggedSource.getTag()); }}); } catch (Exception e) { Throwables.throwIfUnchecked(e); @@ -95,9 +95,9 @@ public SourceOperation.TaggedSource apply(BoundedSource input) { } } }) - .transform(new Function() { + .transform(new Function() { @Override - public InputSplit apply(SourceOperation.TaggedSource taggedSource) { + public InputSplit apply(ReadOperation.TaggedSource taggedSource) { return new BeamInputSplit(taggedSource.getSource(), options, taggedSource.getTag()); }}) .toList(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java new file mode 100644 index 000000000000..6d7a81adf408 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.ResolveOptions; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; + +/** + * Utilities to handle {@link Configuration}. + */ +public class ConfigurationUtils { + + public static ResourceId getResourceIdForOutput(String fileName, Configuration conf) { + ResourceId outDir = FileSystems.matchNewResource(conf.get(FileOutputFormat.OUTDIR), true); + return outDir.resolve(fileName, ResolveOptions.StandardResolveOptions.RESOLVE_FILE); + } + + public static String getFileOutputDir(String baseFileOutputDir, int stageId) { + if (baseFileOutputDir.endsWith("/")) { + return String.format("%sstage-%d", baseFileOutputDir, stageId); + } else { + return String.format("%s/stage-%d", baseFileOutputDir, stageId); + } + } + + public static String getFileOutputPath(String baseFileOutputDir, int stageId, String fileName) { + return String.format("%s/%s", getFileOutputDir(baseFileOutputDir, stageId), fileName); + } + + public static String toFileName(String tagName) { + return tagName.replaceAll("[^A-Za-z0-9]", "0"); + } + +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java index 70263c3e74d1..a95e79e2d2da 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java @@ -44,29 +44,41 @@ /** * Operation that reads from files. */ -public class FileReadOperation extends SourceOperation> { +public class FileReadOperation extends ReadOperation> { + + private final String fileName; + private final Coder coder; + private final TupleTag tupleTag; public FileReadOperation( - int producerStageId, String fileName, Coder coder, TupleTag tupleTag) { - super(new FileBoundedSource<>(producerStageId, fileName, coder), tupleTag); + super(); + this.fileName = checkNotNull(fileName, "fileName"); + this.coder = checkNotNull(coder, "coder"); + this.tupleTag = checkNotNull(tupleTag, "tupleTag"); + } + + @Override + TaggedSource getTaggedSource(Configuration conf) { + return TaggedSource.of( + new FileBoundedSource<>(fileName, coder, new SerializableConfiguration(conf)), + tupleTag); } private static class FileBoundedSource extends BoundedSource> { - private final int producerStageId; private final String fileName; private final Coder> coder; + private final SerializableConfiguration conf; - FileBoundedSource(int producerStageId, String fileName, Coder coder) { - this.producerStageId = producerStageId; + FileBoundedSource(String fileName, Coder coder, SerializableConfiguration conf) { this.fileName = checkNotNull(fileName, "fileName"); checkNotNull(coder, "coder"); this.coder = WindowedValue.getFullCoder( coder, WindowingStrategy.globalDefault().getWindowFn().windowCoder()); - + this.conf = checkNotNull(conf, "conf"); } @Override @@ -84,18 +96,15 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { @Override public BoundedReader> createReader(PipelineOptions options) throws IOException { - Path pattern = new Path(String.format("/tmp/mapreduce/stage-2/%s*", fileName)); - // TODO: use config from the job. - Configuration conf = new Configuration(); - conf.set( - "io.serializations", - "org.apache.hadoop.io.serializer.WritableSerialization," - + "org.apache.hadoop.io.serializer.JavaSerialization"); - FileSystem fs = pattern.getFileSystem(conf); + Path pattern = new Path( + ConfigurationUtils.getResourceIdForOutput(fileName, conf.getConf()) + "*"); + + FileSystem fs = pattern.getFileSystem(conf.getConf()); FileStatus[] files = fs.globStatus(pattern); + Queue readers = new LinkedList<>(); for (FileStatus f : files) { - readers.add(new SequenceFile.Reader(fs, f.getPath(), conf)); + readers.add(new SequenceFile.Reader(fs, f.getPath(), conf.getConf())); } return new Reader<>(this, readers, coder); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java index 18bff2a81157..cb3a8c48eb05 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java @@ -17,10 +17,11 @@ */ package org.apache.beam.runners.mapreduce.translation; +import static com.google.common.base.Preconditions.checkNotNull; + import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.ArrayList; @@ -47,22 +48,23 @@ */ public class FileSideInputReader implements SideInputReader { - private final Map, String> tupleTagToFileName; + private final Map, String> tupleTagToFilePath; private final Map, Coder> tupleTagToCoder; + private final Configuration conf; - public FileSideInputReader(List sideInputTags) { - this.tupleTagToFileName = Maps.newHashMap(); - this.tupleTagToCoder = Maps.newHashMap(); - for (Graphs.Tag tag : sideInputTags) { - tupleTagToFileName.put(tag.getTupleTag(), toFileName(tag.getName())); - tupleTagToCoder.put(tag.getTupleTag(), tag.getCoder()); - } + public FileSideInputReader( + Map, String> tupleTagToFilePath, + Map, Coder> tupleTagToCoder, + Configuration conf) { + this.tupleTagToFilePath = checkNotNull(tupleTagToFilePath, "tupleTagToFilePath"); + this.tupleTagToCoder = checkNotNull(tupleTagToCoder, "tupleTagToCoder"); + this.conf = checkNotNull(conf, "conf"); } @Nullable @Override public T get(PCollectionView view, BoundedWindow window) { - String fileName = tupleTagToFileName.get(view.getTagInternal()); + String filePath = tupleTagToFilePath.get(view.getTagInternal()); IterableCoder> coder = (IterableCoder>) tupleTagToCoder.get(view.getTagInternal()); Coder> elemCoder = coder.getElemCoder(); @@ -70,16 +72,11 @@ public T get(PCollectionView view, BoundedWindow window) { final BoundedWindow sideInputWindow = view.getWindowMappingFn().getSideInputWindow(window); - Path pattern = new Path(String.format("/tmp/mapreduce/stage-1/%s*", fileName)); - Configuration conf = new Configuration(); - conf.set( - "io.serializations", - "org.apache.hadoop.io.serializer.WritableSerialization," - + "org.apache.hadoop.io.serializer.JavaSerialization"); + Path pattern = new Path(filePath + "*"); try { - FileSystem fs; - fs = pattern.getFileSystem(conf); + FileSystem fs = pattern.getFileSystem(conf); FileStatus[] files = fs.globStatus(pattern); + // TODO: handle empty views which may result in no files case. SequenceFile.Reader reader = new SequenceFile.Reader(fs, files[0].getPath(), conf); List> availableSideInputs = new ArrayList<>(); @@ -114,15 +111,11 @@ public boolean apply(@Nullable WindowedValue sideInputCandidate) { @Override public boolean contains(PCollectionView view) { - return tupleTagToFileName.containsKey(view.getTagInternal()); + return tupleTagToFilePath.containsKey(view.getTagInternal()); } @Override public boolean isEmpty() { - return tupleTagToFileName.isEmpty(); - } - - private String toFileName(String tagName) { - return tagName.replaceAll("[^A-Za-z0-9]", "0"); + return tupleTagToFilePath.isEmpty(); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 7c76823b1127..b6e134e10888 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -17,17 +17,16 @@ */ package org.apache.beam.runners.mapreduce.translation; +import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import java.util.ArrayList; import java.util.List; -import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.Coder; +import java.util.Map; +import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; @@ -37,8 +36,10 @@ */ public class GraphPlanner { + private final MapReducePipelineOptions options; - public GraphPlanner() { + public GraphPlanner(MapReducePipelineOptions options) { + this.options = checkNotNull(options, "options"); } public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { @@ -54,7 +55,7 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { continue; } String tagName = tag.getName(); - String fileName = tagName.replaceAll("[^A-Za-z0-9]", "0"); + String fileName = ConfigurationUtils.toFileName(tagName); // TODO: should not hard-code windows coder. WindowedValue.WindowedValueCoder writeValueCoder = WindowedValue.getFullCoder( @@ -77,11 +78,13 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { consumer.addEdge(readOutput, step); } consumer.removeTag(tag); + + String filePath = ConfigurationUtils.getFileOutputPath( + options.getFileOutputDir(), fusedStep.getStageId(), fileName); consumer.addStep( Graphs.Step.of( readStepName, - new FileReadOperation( - fusedStep.getStageId(), fileName, tag.getCoder(), tag.getTupleTag())), + new FileReadOperation(filePath, tag.getCoder(), tag.getTupleTag())), ImmutableList.of(), ImmutableList.of(readOutput)); } @@ -92,13 +95,13 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { for (final Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { List readSteps = fusedStep.getStartSteps(); - List sources = new ArrayList<>(); + List readOperations = new ArrayList<>(); List readOutTags = new ArrayList<>(); List> readOutTupleTags = new ArrayList<>(); StringBuilder partitionStepName = new StringBuilder(); for (Graphs.Step step : readSteps) { - checkState(step.getOperation() instanceof SourceOperation); - sources.add(((SourceOperation) step.getOperation()).getTaggedSource()); + checkState(step.getOperation() instanceof ReadOperation); + readOperations.add(((ReadOperation) step.getOperation())); Graphs.Tag tag = Iterables.getOnlyElement(fusedStep.getOutputTags(step)); readOutTags.add(tag); readOutTupleTags.add(tag.getTupleTag()); @@ -110,10 +113,34 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { partitionStepName.deleteCharAt(partitionStepName.length() - 1); } - Graphs.Step partitionStep = - Graphs.Step.of(partitionStepName.toString(), new PartitionOperation(sources)); + Graphs.Step partitionStep = Graphs.Step.of( + partitionStepName.toString(), new PartitionOperation(readOperations, readOutTupleTags)); fusedStep.addStep(partitionStep, ImmutableList.of(), readOutTags); } + + // Setup side inputs + for (final Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { + for (Graphs.Step step : fusedStep.getSteps()) { + if (!(step.getOperation() instanceof ParDoOperation)) { + continue; + } + ParDoOperation parDo = (ParDoOperation) step.getOperation(); + List sideInputTags = parDo.getSideInputTags(); + if (sideInputTags.size() == 0) { + continue; + } + Map, String> tupleTagToFilePath = Maps.newHashMap(); + for (Graphs.Tag sideInTag : sideInputTags) { + tupleTagToFilePath.put( + sideInTag.getTupleTag(), + ConfigurationUtils.getFileOutputPath( + options.getFileOutputDir(), + fusedGraph.getProducer(sideInTag).getStageId(), + ConfigurationUtils.toFileName(sideInTag.getName()))); + } + parDo.setupSideInput(tupleTagToFilePath); + } + } return fusedGraph; } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java index 471c7f50d043..768f17c16824 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.collect.ImmutableList; -import java.util.List; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 677f3a727bf0..9f291d5794b1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -20,15 +20,17 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; @@ -49,23 +51,23 @@ public class JobPrototype { public static JobPrototype create( - int stageId, Graphs.FusedStep fusedStep, PipelineOptions options) { + int stageId, Graphs.FusedStep fusedStep, MapReducePipelineOptions options) { return new JobPrototype(stageId, fusedStep, options); } private final int stageId; private final Graphs.FusedStep fusedStep; - private final PipelineOptions options; + private final MapReducePipelineOptions options; - private JobPrototype(int stageId, Graphs.FusedStep fusedStep, PipelineOptions options) { + private JobPrototype(int stageId, Graphs.FusedStep fusedStep, MapReducePipelineOptions options) { this.stageId = stageId; this.fusedStep = checkNotNull(fusedStep, "fusedStep"); this.options = checkNotNull(options, "options"); } - public Job build(Class jarClass, Configuration conf) throws IOException { - Job job = new Job(conf); - conf = job.getConfiguration(); + public Job build(Class jarClass, Configuration initConf) throws IOException { + Job job = new Job(initConf); + final Configuration conf = job.getConfiguration(); job.setJarByClass(jarClass); conf.set( "io.serializations", @@ -75,17 +77,26 @@ public Job build(Class jarClass, Configuration conf) throws IOException { //TODO: config out dir with PipelineOptions. conf.set( FileOutputFormat.OUTDIR, - String.format("/tmp/mapreduce/stage-%d", fusedStep.getStageId())); + ConfigurationUtils.getFileOutputDir(options.getFileOutputDir(), fusedStep.getStageId())); // Setup BoundedSources in BeamInputFormat. Graphs.Step startStep = Iterables.getOnlyElement(fusedStep.getStartSteps()); checkState(startStep.getOperation() instanceof PartitionOperation); PartitionOperation partitionOperation = (PartitionOperation) startStep.getOperation(); + ArrayList taggedSources = new ArrayList<>(); + taggedSources.addAll(FluentIterable.from(partitionOperation + .getReadOperations()) + .transform(new Function() { + @Override + public ReadOperation.TaggedSource apply(ReadOperation operation) { + return operation.getTaggedSource(conf); + }}) + .toList()); conf.set( BeamInputFormat.BEAM_SERIALIZED_BOUNDED_SOURCE, Base64.encodeBase64String(SerializableUtils.serializeToByteArray( - new ArrayList<>(partitionOperation.getTaggedSources())))); + taggedSources))); conf.set( BeamInputFormat.BEAM_SERIALIZED_PIPELINE_OPTIONS, Base64.encodeBase64String(SerializableUtils.serializeToByteArray( diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java index 7504e1cf3030..bd24f05d7a79 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java @@ -30,6 +30,7 @@ */ public abstract class Operation implements Serializable { private final OutputReceiver[] receivers; + private SerializableConfiguration conf; public Operation(int numOutputs) { this.receivers = new OutputReceiver[numOutputs]; @@ -44,6 +45,7 @@ public Operation(int numOutputs) { *

Called after all successors consuming operations have been started. */ public void start(TaskInputOutputContext taskContext) { + conf = new SerializableConfiguration(taskContext.getConfiguration()); for (OutputReceiver receiver : receivers) { if (receiver == null) { continue; @@ -75,6 +77,10 @@ public void finish() { } } + public SerializableConfiguration getConf() { + return conf; + } + public List getOutputReceivers() { // TODO: avoid allocating objects for each output emit. return ImmutableList.copyOf(receivers); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java index 020bfbe36787..947d7733ce95 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -20,10 +20,14 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.collect.Maps; import java.util.List; +import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.NullSideInputReader; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -40,8 +44,10 @@ public abstract class ParDoOperation extends Operation protected final SerializedPipelineOptions options; protected final TupleTag mainOutputTag; private final List> sideOutputTags; - private final List sideInputTags; protected final WindowingStrategy windowingStrategy; + private final List sideInputTags; + private Map, String> tupleTagToFilePath; + protected DoFnInvoker doFnInvoker; private DoFnRunner fnRunner; @@ -56,8 +62,8 @@ public ParDoOperation( this.options = new SerializedPipelineOptions(checkNotNull(options, "options")); this.mainOutputTag = checkNotNull(mainOutputTag, "mainOutputTag"); this.sideOutputTags = checkNotNull(sideOutputTags, "sideOutputTags"); - this.sideInputTags = checkNotNull(sideInputTags, "sideInputTags"); this.windowingStrategy = checkNotNull(windowingStrategy, "windowingStrategy"); + this.sideInputTags = checkNotNull(sideInputTags, "sideInputTags"); } /** @@ -73,10 +79,16 @@ public void start(TaskInputOutputContext taskCon doFnInvoker = DoFnInvokers.invokerFor(doFn); doFnInvoker.invokeSetup(); + Map, Coder> tupleTagToCoder = Maps.newHashMap(); + for (Graphs.Tag tag : sideInputTags) { + tupleTagToCoder.put(tag.getTupleTag(), tag.getCoder()); + } fnRunner = DoFnRunners.simpleRunner( options.getPipelineOptions(), getDoFn(), - new FileSideInputReader(sideInputTags), + sideInputTags.isEmpty() + ? NullSideInputReader.empty() : + new FileSideInputReader(tupleTagToFilePath, tupleTagToCoder, getConf().getConf()), createOutputManager(), mainOutputTag, sideOutputTags, @@ -100,6 +112,14 @@ public void finish() { super.finish(); } + public void setupSideInput(Map, String> tupleTagToFilePath) { + this.tupleTagToFilePath = checkNotNull(tupleTagToFilePath, "tupleTagToFilePath"); + } + + public List getSideInputTags() { + return sideInputTags; + } + @Override protected int getOutputIndex(TupleTag tupleTag) { if (tupleTag == mainOutputTag) { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java index b8aefd63f99d..687b5b9e0485 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java @@ -33,23 +33,17 @@ */ public class PartitionOperation extends Operation, Object>> { - private final List sources; + private final List readOperations; private final List> tupleTags; - public PartitionOperation(List sources) { - super(sources.size()); - this.sources = checkNotNull(sources, "sources"); - this.tupleTags = FluentIterable.from(sources) - .transform(new Function>() { - @Override - public TupleTag apply(SourceOperation.TaggedSource input) { - return input.getTag(); - }}) - .toList(); + public PartitionOperation(List readOperations, List> tupleTags) { + super(readOperations.size()); + this.readOperations = checkNotNull(readOperations, "readOperations"); + this.tupleTags = checkNotNull(tupleTags, "tupleTags"); } - public List getTaggedSources() { - return sources; + public List getReadOperations() { + return readOperations; } @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java index e93986bdf644..138c00ea7a5a 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java @@ -20,15 +20,15 @@ import org.apache.beam.sdk.io.Read; /** - * Translates a {@link Read.Bounded} to a {@link SourceOperation}. + * Translates a {@link Read.Bounded} to a {@link ReadOperation}. */ class ReadBoundedTranslator extends TransformTranslator.Default> { @Override public void translateNode(Read.Bounded transform, TranslationContext context) { TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); - SourceOperation operation = - new SourceOperation(transform.getSource(), userGraphContext.getOnlyOutputTag()); + ReadOperation operation = + new SourceReadOperation(transform.getSource(), userGraphContext.getOnlyOutputTag()); context.addInitStep( Graphs.Step.of(userGraphContext.getStepName(), operation), userGraphContext.getInputTags(), diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java similarity index 76% rename from runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceOperation.java rename to runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java index 4ac850f0c05b..cb8b00e966c0 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadOperation.java @@ -17,25 +17,20 @@ */ package org.apache.beam.runners.mapreduce.translation; -import static com.google.common.base.Preconditions.checkNotNull; - import com.google.auto.value.AutoValue; import java.io.Serializable; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.hadoop.conf.Configuration; /** * A Read.Bounded place holder {@link Operation} during pipeline translation. */ -class SourceOperation extends Operation { - private final TaggedSource source; +abstract class ReadOperation extends Operation { - SourceOperation(BoundedSource boundedSource, TupleTag tupleTag) { + public ReadOperation() { super(1); - checkNotNull(boundedSource, "boundedSource"); - checkNotNull(tupleTag, "tupleTag"); - this.source = TaggedSource.of(boundedSource, tupleTag); } @Override @@ -44,9 +39,10 @@ public void process(WindowedValue elem) { String.format("%s should not in execution graph.", this.getClass().getSimpleName())); } - TaggedSource getTaggedSource() { - return source; - } + /** + * Returns a TaggedSource during pipeline construction time. + */ + abstract TaggedSource getTaggedSource(Configuration conf); @AutoValue abstract static class TaggedSource implements Serializable { @@ -55,7 +51,7 @@ abstract static class TaggedSource implements Serializable { static TaggedSource of(BoundedSource boundedSource, TupleTag tupleTag) { return new org.apache.beam.runners.mapreduce.translation - .AutoValue_SourceOperation_TaggedSource(boundedSource, tupleTag); + .AutoValue_ReadOperation_TaggedSource(boundedSource, tupleTag); } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java index 459e93b461d3..9a63b059b497 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java @@ -18,8 +18,6 @@ package org.apache.beam.runners.mapreduce.translation; import com.google.common.collect.ImmutableList; -import java.util.Collections; -import java.util.List; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializableConfiguration.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializableConfiguration.java new file mode 100644 index 000000000000..7af595c6b506 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SerializableConfiguration.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import org.apache.hadoop.conf.Configuration; + +/** + * A {@link Serializable} {@link Configuration}. + */ +class SerializableConfiguration implements Serializable { + + private transient Configuration conf; + + SerializableConfiguration(Configuration conf) { + this.conf = checkNotNull(conf, "conf"); + } + + Configuration getConf() { + return conf; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + conf.write(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + conf = new Configuration(); + conf.readFields(in); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java new file mode 100644 index 000000000000..19b032033692 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.hadoop.conf.Configuration; + +/** + * Operation that reads from {@link BoundedSource}. + */ +public class SourceReadOperation extends ReadOperation { + private final TaggedSource source; + + SourceReadOperation(BoundedSource boundedSource, TupleTag tupleTag) { + checkNotNull(boundedSource, "boundedSource"); + checkNotNull(tupleTag, "tupleTag"); + this.source = TaggedSource.of(boundedSource, tupleTag); + } + + @Override + TaggedSource getTaggedSource(Configuration conf) { + return source; + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java index dfa18c8bf1cc..d018345d2c2d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ViewTranslator.java @@ -37,7 +37,7 @@ public void translateNode( WindowingStrategy windowingStrategy = inPCollection.getWindowingStrategy(); Graphs.Tag outTag = Iterables.getOnlyElement(userGraphContext.getOutputTags()); - String fileName = outTag.getName().replaceAll("[^A-Za-z0-9]", "0"); + String fileName = ConfigurationUtils.toFileName(outTag.getName()); FileWriteOperation operation = new FileWriteOperation<>( fileName, diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java index ac965cb0e938..fca6131f117c 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/translation/GraphPlannerTest.java @@ -54,7 +54,7 @@ public void testCombine() throws Exception { GraphConverter graphConverter = new GraphConverter(context); p.traverseTopologically(graphConverter); - GraphPlanner planner = new GraphPlanner(); + GraphPlanner planner = new GraphPlanner(options); Graphs.FusedGraph fusedGraph = new Graphs.FusedGraph(context.getInitGraph()); fusedGraph = planner.plan(fusedGraph); From 6c2390a1f7d7d912d186e84eed18f94e36d2a65f Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 30 Aug 2017 17:11:07 +0800 Subject: [PATCH 18/34] [BEAM-2783] support metrics in MapReduceRunner. --- .../mapreduce/MapReducePipelineResult.java | 61 ++++++++++ .../runners/mapreduce/MapReduceRunner.java | 6 +- .../GroupAlsoByWindowsParDoOperation.java | 3 +- .../mapreduce/translation/JobPrototype.java | 8 +- .../translation/MapReduceMetricResults.java | 106 ++++++++++++++++++ .../translation/MetricsReporter.java | 97 ++++++++++++++++ .../translation/NormalParDoOperation.java | 3 +- .../mapreduce/translation/ParDoOperation.java | 33 +++++- .../translation/ParDoTranslator.java | 1 + ...eifyTimestampAndWindowsParDoOperation.java | 3 +- .../beam/runners/mapreduce/WordCountTest.java | 16 ++- 11 files changed, 325 insertions(+), 12 deletions(-) create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReduceMetricResults.java create mode 100644 runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MetricsReporter.java diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java new file mode 100644 index 000000000000..90c521a7e4f9 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.io.IOException; +import java.util.List; +import org.apache.beam.runners.mapreduce.translation.MapReduceMetricResults; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.metrics.MetricResults; +import org.apache.hadoop.mapreduce.Job; +import org.joda.time.Duration; + +public class MapReducePipelineResult implements PipelineResult { + + private final List jobs; + public MapReducePipelineResult(List jobs) { + this.jobs = checkNotNull(jobs, "jobs"); + } + + @Override + public State getState() { + return State.DONE; + } + + @Override + public State cancel() throws IOException { + return State.DONE; + } + + @Override + public State waitUntilFinish(Duration duration) { + return State.DONE; + } + + @Override + public State waitUntilFinish() { + return State.DONE; + } + + @Override + public MetricResults metrics() { + return new MapReduceMetricResults(jobs); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index 3f7680823545..88ed01ec0a89 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.common.base.Throwables; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.mapreduce.translation.DotfileWriter; import org.apache.beam.runners.mapreduce.translation.GraphConverter; import org.apache.beam.runners.mapreduce.translation.GraphPlanner; @@ -76,6 +78,7 @@ public PipelineResult run(Pipeline pipeline) { fusedGraph.getFusedSteps(); + List jobs = new ArrayList<>(); int stageId = 0; for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { Configuration config = new Configuration(); @@ -87,11 +90,12 @@ public PipelineResult run(Pipeline pipeline) { try { Job job = jobPrototype.build(options.getJarClass(), config); job.waitForCompletion(true); + jobs.add(job); } catch (Exception e) { Throwables.throwIfUnchecked(e); throw new RuntimeException(e); } } - return null; + return new MapReducePipelineResult(jobs); } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java index 768f17c16824..14e3a29d1b66 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GroupAlsoByWindowsParDoOperation.java @@ -35,11 +35,12 @@ public class GroupAlsoByWindowsParDoOperation extends ParDoOperation { private final Coder inputCoder; public GroupAlsoByWindowsParDoOperation( + String stepName, PipelineOptions options, WindowingStrategy windowingStrategy, Coder inputCoder, Graphs.Tag outTag) { - super(options, outTag.getTupleTag(), ImmutableList.>of(), + super(stepName, options, outTag.getTupleTag(), ImmutableList.>of(), ImmutableList.of(), windowingStrategy); this.inputCoder = checkNotNull(inputCoder, "inputCoder"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 9f291d5794b1..39487fd4dc0c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -117,15 +117,17 @@ public ReadOperation.TaggedSource apply(ReadOperation operation) { Graphs.Step reifyStep = Graphs.Step.of( reifyStepName, new ReifyTimestampAndWindowsParDoOperation( - options, operation.getWindowingStrategy(), reifyOutputTag)); + reifyStepName, options, operation.getWindowingStrategy(), reifyOutputTag)); Graphs.Step writeStep = Graphs.Step.of( groupByKey.getFullName() + "-Write", new ShuffleWriteOperation(kvCoder.getKeyCoder(), reifyValueCoder)); + String gabwStepName = groupByKey.getFullName() + "-GroupAlsoByWindows"; Graphs.Step gabwStep = Graphs.Step.of( - groupByKey.getFullName() + "-GroupAlsoByWindows", - new GroupAlsoByWindowsParDoOperation(options, windowingStrategy, kvCoder, gbkOutTag)); + gabwStepName, + new GroupAlsoByWindowsParDoOperation( + gabwStepName, options, windowingStrategy, kvCoder, gbkOutTag)); fusedStep.addStep( reifyStep, fusedStep.getInputTags(groupByKey), ImmutableList.of(reifyOutputTag)); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReduceMetricResults.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReduceMetricResults.java new file mode 100644 index 000000000000..1d1c9fff7396 --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MapReduceMetricResults.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.metrics.MetricFiltering; +import org.apache.beam.runners.core.construction.metrics.MetricKey; +import org.apache.beam.sdk.metrics.DistributionResult; +import org.apache.beam.sdk.metrics.GaugeResult; +import org.apache.beam.sdk.metrics.MetricName; +import org.apache.beam.sdk.metrics.MetricQueryResults; +import org.apache.beam.sdk.metrics.MetricResult; +import org.apache.beam.sdk.metrics.MetricResults; +import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.hadoop.mapreduce.Counter; +import org.apache.hadoop.mapreduce.CounterGroup; +import org.apache.hadoop.mapreduce.Job; + +/** + * Implementation of {@link MetricResults} for the MapReduce Runner. + */ +public class MapReduceMetricResults extends MetricResults { + + private final List jobs; + + public MapReduceMetricResults(List jobs) { + this.jobs = checkNotNull(jobs, "jobs"); + } + + @Override + public MetricQueryResults queryMetrics(MetricsFilter filter) { + List> counters = new ArrayList<>(); + for (Job job : jobs) { + Iterable groups; + try { + groups = job.getCounters(); + } catch (IOException e) { + throw new RuntimeException(e); + } + for (CounterGroup group : groups) { + String groupName = group.getName(); + for (Counter counter : group) { + MetricKey metricKey = MetricsReporter.toMetricKey(groupName, counter.getName()); + if (!MetricFiltering.matches(filter, metricKey)) { + continue; + } + counters.add( + MapReduceMetricResult.create( + metricKey.metricName(), + metricKey.stepName(), + counter.getValue())); + } + } + } + return MapReduceMetricQueryResults.create(counters); + } + + + @AutoValue + abstract static class MapReduceMetricQueryResults implements MetricQueryResults { + + public abstract @Nullable Iterable> distributions(); + public abstract @Nullable Iterable> gauges(); + + public static MetricQueryResults create(Iterable> counters) { + return new AutoValue_MapReduceMetricResults_MapReduceMetricQueryResults( + counters, null, null); + } + } + + @AutoValue + abstract static class MapReduceMetricResult implements MetricResult { + // need to define these here so they appear in the correct order + // and the generated constructor is usable and consistent + public abstract MetricName name(); + public abstract String step(); + public abstract @Nullable T committed(); + public abstract T attempted(); + + public static MetricResult create(MetricName name, String step, T attempted) { + return new AutoValue_MapReduceMetricResults_MapReduceMetricResult( + name, step, null, attempted); + } + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MetricsReporter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MetricsReporter.java new file mode 100644 index 000000000000..9fe139d2d6eb --- /dev/null +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/MetricsReporter.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.mapreduce.translation; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults; + +import com.google.common.collect.Maps; +import java.util.Map; +import org.apache.beam.runners.core.construction.metrics.MetricKey; +import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; +import org.apache.beam.sdk.metrics.MetricName; +import org.apache.beam.sdk.metrics.MetricQueryResults; +import org.apache.beam.sdk.metrics.MetricResult; +import org.apache.beam.sdk.metrics.MetricResults; +import org.apache.beam.sdk.metrics.MetricsContainer; +import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; + +/** + * Class that holds a {@link MetricsContainerStepMap}, and reports metrics to MapReduce framework. + */ +public class MetricsReporter { + + private static final String METRIC_KEY_SEPARATOR = "__"; + private static final String METRIC_PREFIX = "__metrics"; + + private final TaskAttemptContext context; + private final MetricsContainerStepMap metricsContainers; + private final Map reportedCounters = Maps.newHashMap(); + + MetricsReporter(TaskAttemptContext context) { + this.context = checkNotNull(context, "context"); + this.metricsContainers = new MetricsContainerStepMap(); + } + + public MetricsContainer getMetricsContainer(String stepName) { + return metricsContainers.getContainer(stepName); + } + + public void updateMetrics() { + MetricResults metricResults = asAttemptedOnlyMetricResults(metricsContainers); + MetricQueryResults metricQueryResults = + metricResults.queryMetrics(MetricsFilter.builder().build()); + updateCounters(metricQueryResults.counters()); + } + + private void updateCounters(Iterable> counters) { + for (MetricResult metricResult : counters) { + String reportedCounterKey = reportedCounterKey(metricResult); + Long updateValue = metricResult.attempted(); + Long oldValue = reportedCounters.get(reportedCounterKey); + + if (oldValue == null || oldValue < updateValue) { + Long incValue = (oldValue == null ? updateValue : updateValue - oldValue); + context.getCounter(groupName(metricResult), metricResult.name().name()) + .increment(incValue); + reportedCounters.put(reportedCounterKey, updateValue); + } + } + } + + private String groupName(MetricResult metricResult) { + return METRIC_PREFIX + + METRIC_KEY_SEPARATOR + metricResult.step() + + METRIC_KEY_SEPARATOR + metricResult.name().namespace(); + } + + private String reportedCounterKey(MetricResult metricResult) { + return metricResult.step() + + METRIC_KEY_SEPARATOR + metricResult.name().namespace() + + METRIC_KEY_SEPARATOR + metricResult.name().name(); + } + + public static MetricKey toMetricKey(String groupName, String counterName) { + String[] nameSplits = groupName.split(METRIC_KEY_SEPARATOR); + int length = nameSplits.length; + String stepName = length > 1 ? nameSplits[length - 2] : ""; + String namespace = length > 0 ? nameSplits[length - 1] : ""; + return MetricKey.create(stepName, MetricName.named(namespace, counterName)); + } +} diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java index 58a7d6dfd152..8b730ff1c752 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/NormalParDoOperation.java @@ -33,13 +33,14 @@ public class NormalParDoOperation extends ParDoOperation doFn; public NormalParDoOperation( + String stepName, DoFn doFn, PipelineOptions options, TupleTag mainOutputTag, List> sideOutputTags, List sideInputTags, WindowingStrategy windowingStrategy) { - super(options, mainOutputTag, sideOutputTags, sideInputTags, windowingStrategy); + super(stepName, options, mainOutputTag, sideOutputTags, sideInputTags, windowingStrategy); this.doFn = checkNotNull(doFn, "doFn"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java index 947d7733ce95..2c2fbde84364 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -21,6 +21,8 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.collect.Maps; +import java.io.Closeable; +import java.io.IOException; import java.util.List; import java.util.Map; import javax.annotation.Nullable; @@ -28,6 +30,7 @@ import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -41,6 +44,7 @@ * Operation for ParDo. */ public abstract class ParDoOperation extends Operation { + private final String stepName; protected final SerializedPipelineOptions options; protected final TupleTag mainOutputTag; private final List> sideOutputTags; @@ -48,17 +52,19 @@ public abstract class ParDoOperation extends Operation private final List sideInputTags; private Map, String> tupleTagToFilePath; - + private MetricsReporter metricsReporter; protected DoFnInvoker doFnInvoker; private DoFnRunner fnRunner; public ParDoOperation( + String stepName, PipelineOptions options, TupleTag mainOutputTag, List> sideOutputTags, List sideInputTags, WindowingStrategy windowingStrategy) { super(1 + sideOutputTags.size()); + this.stepName = checkNotNull(stepName, "stepName"); this.options = new SerializedPipelineOptions(checkNotNull(options, "options")); this.mainOutputTag = checkNotNull(mainOutputTag, "mainOutputTag"); this.sideOutputTags = checkNotNull(sideOutputTags, "sideOutputTags"); @@ -74,6 +80,8 @@ public ParDoOperation( @Override public void start(TaskInputOutputContext taskContext) { super.start(taskContext); + this.metricsReporter = new MetricsReporter(taskContext); + DoFn doFn = getDoFn(); // Process user's setup doFnInvoker = DoFnInvokers.invokerFor(doFn); @@ -94,7 +102,13 @@ public void start(TaskInputOutputContext taskCon sideOutputTags, null, windowingStrategy); - fnRunner.startBundle(); + + try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer( + metricsReporter.getMetricsContainer(stepName))) { + fnRunner.startBundle(); + } catch (IOException e) { + throw new RuntimeException(e); + } } /** @@ -102,12 +116,23 @@ public void start(TaskInputOutputContext taskCon */ @Override public void process(WindowedValue elem) { - fnRunner.processElement(elem); + try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer( + metricsReporter.getMetricsContainer(stepName))) { + fnRunner.processElement(elem); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override public void finish() { - fnRunner.finishBundle(); + try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer( + metricsReporter.getMetricsContainer(stepName))) { + fnRunner.finishBundle(); + } catch (IOException e) { + throw new RuntimeException(e); + } + metricsReporter.updateMetrics(); doFnInvoker.invokeTeardown(); super.finish(); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java index ae23f7188b55..e866fe2f16cc 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java @@ -31,6 +31,7 @@ public void translateNode( ParDo.MultiOutput transform, TranslationContext context) { TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); NormalParDoOperation operation = new NormalParDoOperation( + transform.getName(), transform.getFn(), userGraphContext.getOptions(), transform.getMainOutputTag(), diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java index 9a63b059b497..9d6b895e00d6 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java @@ -32,10 +32,11 @@ public class ReifyTimestampAndWindowsParDoOperation extends ParDoOperation { public ReifyTimestampAndWindowsParDoOperation( + String stepName, PipelineOptions options, WindowingStrategy windowingStrategy, Graphs.Tag outTag) { - super(options, outTag.getTupleTag(), ImmutableList.>of(), + super(stepName, options, outTag.getTupleTag(), ImmutableList.>of(), ImmutableList.of(), windowingStrategy); } diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java index 363ba0124dfb..263905c66b10 100644 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java +++ b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java @@ -18,9 +18,13 @@ package org.apache.beam.runners.mapreduce; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.MetricNameFilter; +import org.apache.beam.sdk.metrics.MetricResult; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; @@ -51,11 +55,14 @@ public class WordCountTest { */ static class ExtractWordsFn extends DoFn { private final Counter emptyLines = Metrics.counter(ExtractWordsFn.class, "emptyLines"); + private final Counter nonEmptyLines = Metrics.counter(ExtractWordsFn.class, "nonEmptyLines"); @ProcessElement public void processElement(ProcessContext c) { if (c.element().trim().isEmpty()) { emptyLines.inc(); + } else { + nonEmptyLines.inc(); } // Split the line into words. @@ -98,6 +105,13 @@ public void testWordCount() { .apply(MapElements.via(new FormatAsTextFn())) .apply("WriteCounts", TextIO.write().to(output)); - p.run(); + PipelineResult result = p.run(); + Iterable> counters = result.metrics() + .queryMetrics( + MetricsFilter.builder() + .addNameFilter(MetricNameFilter.named(ExtractWordsFn.class, "emptyLines")) + .build()) + .counters(); + System.out.println(counters.iterator().next()); } } From 4e7062cd9de6ae3f0616033823fd995eb10a3744 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 30 Aug 2017 19:16:06 +0800 Subject: [PATCH 19/34] mr-runner: fix the bug that steps are attached multiple times in diamond shaped DAG. --- .../mapreduce/translation/JobPrototype.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 39487fd4dc0c..a0c662665392 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -24,10 +24,12 @@ import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Set; import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -139,7 +141,7 @@ public ReadOperation.TaggedSource apply(ReadOperation operation) { // Setup BeamReducer Graphs.Step reducerStartStep = gabwStep; - chainOperations(reducerStartStep, fusedStep); + chainOperations(reducerStartStep, fusedStep, Sets.newHashSet()); conf.set( BeamReducer.BEAM_REDUCER_KV_CODER, Base64.encodeBase64String(SerializableUtils.serializeToByteArray( @@ -152,7 +154,7 @@ public ReadOperation.TaggedSource apply(ReadOperation operation) { } // Setup DoFns in BeamMapper. - chainOperations(startStep, fusedStep); + chainOperations(startStep, fusedStep, Sets.newHashSet()); job.setMapOutputKeyClass(BytesWritable.class); job.setMapOutputValueClass(byte[].class); @@ -177,7 +179,8 @@ public ReadOperation.TaggedSource apply(ReadOperation operation) { return job; } - private void chainOperations(Graphs.Step current, Graphs.FusedStep fusedStep) { + private void chainOperations( + Graphs.Step current, Graphs.FusedStep fusedStep, Set visited) { Operation operation = current.getOperation(); List outputTags = fusedStep.getOutputTags(current); for (Graphs.Tag outTag : outputTags) { @@ -185,9 +188,12 @@ private void chainOperations(Graphs.Step current, Graphs.FusedStep fusedStep) { operation.attachConsumer(outTag.getTupleTag(), consumer.getOperation()); } } + visited.add(current); for (Graphs.Tag outTag : outputTags) { for (Graphs.Step consumer : fusedStep.getConsumers(outTag)) { - chainOperations(consumer, fusedStep); + if (!visited.contains(consumer)) { + chainOperations(consumer, fusedStep, visited); + } } } } From 8d3386d479b5704fa9448c7a9b1eab9c66e75549 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 30 Aug 2017 19:40:24 +0800 Subject: [PATCH 20/34] mr-runner: setMetricsSupported to run ValidatesRunner tests with TestPipeline. --- runners/map-reduce/pom.xml | 14 +++++++++++ .../runners/mapreduce/MapReduceRunner.java | 5 ++++ .../translation/ConfigurationUtils.java | 23 ++++++++++++++----- .../mapreduce/translation/GraphPlanner.java | 11 ++++----- .../mapreduce/translation/JobPrototype.java | 4 +++- 5 files changed, 44 insertions(+), 13 deletions(-) diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml index e85803104c92..d65bb3448744 100644 --- a/runners/map-reduce/pom.xml +++ b/runners/map-reduce/pom.xml @@ -56,6 +56,20 @@ org.apache.beam.sdk.testing.ValidatesRunner + + org.apache.beam.sdk.testing.PAssertTest.java + + + org.apache.beam.sdk.testing.UsesSetState, + org.apache.beam.sdk.testing.UsesSplittableParDo, + org.apache.beam.sdk.testing.UsesDistributionMetrics, + org.apache.beam.sdk.testing.UsesGaugeMetrics, + org.apache.beam.sdk.testing.UsesCommittedMetrics, + org.apache.beam.sdk.testing.LargeKeys$Above10MB, + org.apache.beam.sdk.testing.UsesTimersInParDo, + org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesTestStream + none true diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index 88ed01ec0a89..71edf1a0b20a 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -31,9 +31,11 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineRunner; +import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Job; +import org.apache.log4j.BasicConfigurator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +64,9 @@ public static MapReduceRunner fromOptions(PipelineOptions options) { @Override public PipelineResult run(Pipeline pipeline) { + BasicConfigurator.configure(); + MetricsEnvironment.setMetricsSupported(true); + TranslationContext context = new TranslationContext(options); GraphConverter graphConverter = new GraphConverter(context); pipeline.traverseTopologically(graphConverter); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java index 6d7a81adf408..4ec50bd284bb 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java @@ -17,9 +17,13 @@ */ package org.apache.beam.runners.mapreduce.translation; +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; @@ -28,21 +32,28 @@ */ public class ConfigurationUtils { + private final MapReducePipelineOptions options; + + public ConfigurationUtils(MapReducePipelineOptions options) { + this.options = checkNotNull(options, "options"); + } + public static ResourceId getResourceIdForOutput(String fileName, Configuration conf) { ResourceId outDir = FileSystems.matchNewResource(conf.get(FileOutputFormat.OUTDIR), true); return outDir.resolve(fileName, ResolveOptions.StandardResolveOptions.RESOLVE_FILE); } - public static String getFileOutputDir(String baseFileOutputDir, int stageId) { - if (baseFileOutputDir.endsWith("/")) { - return String.format("%sstage-%d", baseFileOutputDir, stageId); + public String getFileOutputDir(int stageId) { + String fileOutputDir = options.getFileOutputDir(); + if (fileOutputDir.endsWith("/")) { + return String.format("%s%s/stage-%d", fileOutputDir, options.getJobName(), stageId); } else { - return String.format("%s/stage-%d", baseFileOutputDir, stageId); + return String.format("%s/%s/stage-%d", fileOutputDir, options.getJobName(), stageId); } } - public static String getFileOutputPath(String baseFileOutputDir, int stageId, String fileName) { - return String.format("%s/%s", getFileOutputDir(baseFileOutputDir, stageId), fileName); + public String getFileOutputPath(int stageId, String fileName) { + return String.format("%s/%s", getFileOutputDir(stageId), fileName); } public static String toFileName(String tagName) { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index b6e134e10888..608b3045670e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -36,10 +36,11 @@ */ public class GraphPlanner { - private final MapReducePipelineOptions options; + private final ConfigurationUtils configUtils; public GraphPlanner(MapReducePipelineOptions options) { - this.options = checkNotNull(options, "options"); + checkNotNull(options, "options"); + this.configUtils = new ConfigurationUtils(options); } public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { @@ -79,8 +80,7 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { } consumer.removeTag(tag); - String filePath = ConfigurationUtils.getFileOutputPath( - options.getFileOutputDir(), fusedStep.getStageId(), fileName); + String filePath = configUtils.getFileOutputPath(fusedStep.getStageId(), fileName); consumer.addStep( Graphs.Step.of( readStepName, @@ -133,8 +133,7 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { for (Graphs.Tag sideInTag : sideInputTags) { tupleTagToFilePath.put( sideInTag.getTupleTag(), - ConfigurationUtils.getFileOutputPath( - options.getFileOutputDir(), + configUtils.getFileOutputPath( fusedGraph.getProducer(sideInTag).getStageId(), ConfigurationUtils.toFileName(sideInTag.getName()))); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index a0c662665392..93ae33a0803d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -60,11 +60,13 @@ public static JobPrototype create( private final int stageId; private final Graphs.FusedStep fusedStep; private final MapReducePipelineOptions options; + private final ConfigurationUtils configUtils; private JobPrototype(int stageId, Graphs.FusedStep fusedStep, MapReducePipelineOptions options) { this.stageId = stageId; this.fusedStep = checkNotNull(fusedStep, "fusedStep"); this.options = checkNotNull(options, "options"); + this.configUtils = new ConfigurationUtils(options); } public Job build(Class jarClass, Configuration initConf) throws IOException { @@ -79,7 +81,7 @@ public Job build(Class jarClass, Configuration initConf) throws IOException { //TODO: config out dir with PipelineOptions. conf.set( FileOutputFormat.OUTDIR, - ConfigurationUtils.getFileOutputDir(options.getFileOutputDir(), fusedStep.getStageId())); + configUtils.getFileOutputDir(fusedStep.getStageId())); // Setup BoundedSources in BeamInputFormat. Graphs.Step startStep = Iterables.getOnlyElement(fusedStep.getStartSteps()); From d71975ed2be21657ada3e66950d4e1b2a4d8b148 Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 31 Aug 2017 19:24:19 +0800 Subject: [PATCH 21/34] mr-runner: filter out unsupported features in ValidatesRunner tests. --- runners/map-reduce/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml index d65bb3448744..3b253a797eaf 100644 --- a/runners/map-reduce/pom.xml +++ b/runners/map-reduce/pom.xml @@ -68,6 +68,7 @@ org.apache.beam.sdk.testing.LargeKeys$Above10MB, org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesTestStream none From 807f903413bec1d8052406adeac2ddd793765511 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 30 Aug 2017 20:01:06 +0800 Subject: [PATCH 22/34] mr-runner: fix NPE in PipelineTest.testIdentityTransform(). --- .../beam/runners/mapreduce/translation/GraphConverter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java index de1c80b758a3..458961f388da 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -70,7 +70,8 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { @Override public void leaveCompositeTransform(TransformHierarchy.Node node) { if (node.getTransform() != null) { - if (enclosedTransformCounts.get(node) > 1) { + Integer enclosedTransformCount = enclosedTransformCounts.get(node); + if (enclosedTransformCount != null && enclosedTransformCount > 1) { dotfileNodesBuilders.peek().insert(0, new StringBuilder() .append(getIndent()).append( String.format("subgraph \"cluster_%s\" {", node.getFullName())) From ca0b15ada6cdbdfba9ac8adb0b8c874477587fae Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 31 Aug 2017 17:29:04 +0800 Subject: [PATCH 23/34] mr-runner: handle no files case in FileSideInputReader for empty views. --- .../mapreduce/translation/FileSideInputReader.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java index cb3a8c48eb05..403de4e63069 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileSideInputReader.java @@ -76,14 +76,15 @@ public T get(PCollectionView view, BoundedWindow window) { try { FileSystem fs = pattern.getFileSystem(conf); FileStatus[] files = fs.globStatus(pattern); - // TODO: handle empty views which may result in no files case. - SequenceFile.Reader reader = new SequenceFile.Reader(fs, files[0].getPath(), conf); List> availableSideInputs = new ArrayList<>(); - BytesWritable value = new BytesWritable(); - while (reader.next(NullWritable.get(), value)) { - ByteArrayInputStream inStream = new ByteArrayInputStream(value.getBytes()); - availableSideInputs.add(elemCoder.decode(inStream)); + if (files.length > 0) { + SequenceFile.Reader reader = new SequenceFile.Reader(fs, files[0].getPath(), conf); + BytesWritable value = new BytesWritable(); + while (reader.next(NullWritable.get(), value)) { + ByteArrayInputStream inStream = new ByteArrayInputStream(value.getBytes()); + availableSideInputs.add(elemCoder.decode(inStream)); + } } Iterable> sideInputForWindow = Iterables.filter(availableSideInputs, new Predicate>() { From 2d8b12a6ec6d58a01084c6c06e92b3f884c166ba Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 31 Aug 2017 18:58:16 +0800 Subject: [PATCH 24/34] mr-runner: remove the hard-coded GlobalWindow coder, and fixes WindowingTest. --- .../translation/FileReadOperation.java | 11 ++++---- .../mapreduce/translation/GraphPlanner.java | 7 +++-- .../runners/mapreduce/translation/Graphs.java | 10 +++++-- .../mapreduce/translation/JobPrototype.java | 2 +- .../translation/TranslationContext.java | 27 +++++++++++++------ 5 files changed, 36 insertions(+), 21 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java index a95e79e2d2da..cbbfbd252536 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java @@ -47,12 +47,12 @@ public class FileReadOperation extends ReadOperation> { private final String fileName; - private final Coder coder; + private final Coder> coder; private final TupleTag tupleTag; public FileReadOperation( String fileName, - Coder coder, + Coder> coder, TupleTag tupleTag) { super(); this.fileName = checkNotNull(fileName, "fileName"); @@ -73,11 +73,10 @@ private static class FileBoundedSource extends BoundedSource private final Coder> coder; private final SerializableConfiguration conf; - FileBoundedSource(String fileName, Coder coder, SerializableConfiguration conf) { + FileBoundedSource( + String fileName, Coder> coder, SerializableConfiguration conf) { this.fileName = checkNotNull(fileName, "fileName"); - checkNotNull(coder, "coder"); - this.coder = WindowedValue.getFullCoder( - coder, WindowingStrategy.globalDefault().getWindowFn().windowCoder()); + this.coder = checkNotNull(coder, "coder"); this.conf = checkNotNull(conf, "conf"); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 608b3045670e..6c79277684ed 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -58,9 +58,8 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { String tagName = tag.getName(); String fileName = ConfigurationUtils.toFileName(tagName); - // TODO: should not hard-code windows coder. WindowedValue.WindowedValueCoder writeValueCoder = WindowedValue.getFullCoder( - tag.getCoder(), WindowingStrategy.globalDefault().getWindowFn().windowCoder()); + tag.getCoder(), tag.getWindowingStrategy().getWindowFn().windowCoder()); fusedStep.addStep( Graphs.Step.of( @@ -71,7 +70,7 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { String readStepName = tagName + "/Read"; Graphs.Tag readOutput = Graphs.Tag.of( - readStepName + ".out", tag.getTupleTag(), tag.getCoder()); + readStepName + ".out", tag.getTupleTag(), tag.getCoder(), tag.getWindowingStrategy()); for (Graphs.FusedStep consumer : consumers) { // Re-direct tag to readOutput. List receivers = consumer.getConsumers(tag); @@ -84,7 +83,7 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { consumer.addStep( Graphs.Step.of( readStepName, - new FileReadOperation(filePath, tag.getCoder(), tag.getTupleTag())), + new FileReadOperation(filePath, writeValueCoder, tag.getTupleTag())), ImmutableList.of(), ImmutableList.of(readOutput)); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java index b2f793a3cfb1..0b93c3aa7f9e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java @@ -25,6 +25,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** * Class that defines graph vertices. @@ -234,15 +235,20 @@ public abstract static class Tag extends Graph.AbstractTag implements Serializab abstract String getName(); abstract TupleTag getTupleTag(); abstract Coder getCoder(); + abstract WindowingStrategy getWindowingStrategy(); @Override public String toString() { return getName(); } - public static Tag of(String name, TupleTag tupleTag, Coder coder) { + public static Tag of( + String name, + TupleTag tupleTag, + Coder coder, + WindowingStrategy windowingStrategy) { return new org.apache.beam.runners.mapreduce.translation.AutoValue_Graphs_Tag( - name, tupleTag, coder); + name, tupleTag, coder, windowingStrategy); } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 93ae33a0803d..44f279b39458 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -117,7 +117,7 @@ public ReadOperation.TaggedSource apply(ReadOperation operation) { String reifyStepName = groupByKey.getFullName() + "-Reify"; Coder reifyValueCoder = getReifyValueCoder(kvCoder.getValueCoder(), windowingStrategy); Graphs.Tag reifyOutputTag = Graphs.Tag.of( - reifyStepName + ".out", new TupleTag<>(), reifyValueCoder); + reifyStepName + ".out", new TupleTag<>(), reifyValueCoder, windowingStrategy); Graphs.Step reifyStep = Graphs.Step.of( reifyStepName, new ReifyTimestampAndWindowsParDoOperation( diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java index fd6c0ba376a1..93856de38b74 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java @@ -125,12 +125,16 @@ public Graphs.Tag apply(PValue pValue) { if (pValue instanceof PCollection) { PCollection pc = (PCollection) pValue; return Graphs.Tag.of( - pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder()); - } else { + pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder(), pc.getWindowingStrategy()); + } else if (pValue instanceof PCollectionView){ + PCollectionView pView = (PCollectionView) pValue; return Graphs.Tag.of( pValue.getName(), pValueToTupleTag.get(pValue), - ((PCollectionView) pValue).getCoderInternal()); + pView.getCoderInternal(), + pView.getWindowingStrategyInternal()); + } else { + throw new RuntimeException("Unexpected PValue: " + pValue.getClass()); } }}) .toList(); @@ -150,12 +154,17 @@ public Graphs.Tag apply(PValue pValue) { if (pValue instanceof PCollection) { PCollection pc = (PCollection) pValue; return Graphs.Tag.of( - pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder()); - } else { + pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder(), + pc.getWindowingStrategy()); + } else if (pValue instanceof PCollectionView){ + PCollectionView pView = (PCollectionView) pValue; return Graphs.Tag.of( pValue.getName(), pValueToTupleTag.get(pValue), - ((PCollectionView) pValue).getCoderInternal()); + pView.getCoderInternal(), + pView.getWindowingStrategyInternal()); + } else { + throw new RuntimeException("Unexpected PValue: " + pValue.getClass()); } }}) .toList(); @@ -165,14 +174,16 @@ public List getOutputTags() { if (currentNode.getTransform() instanceof View.CreatePCollectionView) { PCollectionView view = ((View.CreatePCollectionView) currentNode.getTransform()).getView(); return ImmutableList.of( - Graphs.Tag.of(view.getName(), view.getTagInternal(), view.getCoderInternal())); + Graphs.Tag.of(view.getName(), view.getTagInternal(), view.getCoderInternal(), + view.getWindowingStrategyInternal())); } else { return FluentIterable.from(currentNode.getOutputs().entrySet()) .transform(new Function, PValue>, Graphs.Tag>() { @Override public Graphs.Tag apply(Map.Entry, PValue> entry) { PCollection pc = (PCollection) entry.getValue(); - return Graphs.Tag.of(pc.getName(), entry.getKey(), pc.getCoder()); + return Graphs.Tag.of( + pc.getName(), entry.getKey(), pc.getCoder(), pc.getWindowingStrategy()); }}) .toList(); } From 0b3718756b6c22a9cb58dbdf4a3bc8b41cca3d1c Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 31 Aug 2017 19:25:10 +0800 Subject: [PATCH 25/34] mr-runner: use the correct step name in ParDoTranslator, this fixes MetricsTest.testAttemptedCounterMetrics(). --- .../beam/runners/mapreduce/translation/ParDoTranslator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java index e866fe2f16cc..f8f1a02faaf1 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoTranslator.java @@ -31,7 +31,7 @@ public void translateNode( ParDo.MultiOutput transform, TranslationContext context) { TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); NormalParDoOperation operation = new NormalParDoOperation( - transform.getName(), + userGraphContext.getStepName(), transform.getFn(), userGraphContext.getOptions(), transform.getMainOutputTag(), From 9f312c561a7a21c92072e91eebdca7fb6f72c9eb Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 31 Aug 2017 21:01:59 +0800 Subject: [PATCH 26/34] mr-runner: use InMemoryStateInternals in ParDoOperation, this fixed ParDoTest that uses state. --- .../mapreduce/translation/ParDoOperation.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java index 2c2fbde84364..ef83e72953cd 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ParDoOperation.java @@ -28,7 +28,12 @@ import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.InMemoryStateInternals; +import org.apache.beam.runners.core.InMemoryTimerInternals; import org.apache.beam.runners.core.NullSideInputReader; +import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StepContext; +import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; @@ -91,6 +96,18 @@ public void start(TaskInputOutputContext taskCon for (Graphs.Tag tag : sideInputTags) { tupleTagToCoder.put(tag.getTupleTag(), tag.getCoder()); } + + final StateInternals stateInternals; + try { + stateInternals = InMemoryStateInternals.forKey(taskContext.getCurrentKey()); + } catch (IOException | InterruptedException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException(e); + } + final TimerInternals timerInternals = new InMemoryTimerInternals(); + fnRunner = DoFnRunners.simpleRunner( options.getPipelineOptions(), getDoFn(), @@ -100,7 +117,17 @@ public void start(TaskInputOutputContext taskCon createOutputManager(), mainOutputTag, sideOutputTags, - null, + new StepContext() { + @Override + public StateInternals stateInternals() { + return stateInternals; + } + + @Override + public TimerInternals timerInternals() { + return timerInternals; + } + }, windowingStrategy); try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer( From 989d7d8e4eabe6aafd2008eed16f533ec75a43d1 Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 1 Sep 2017 13:04:54 +0800 Subject: [PATCH 27/34] mr-runner: fail early in the runner when MapReduce job fails. --- .../org/apache/beam/runners/mapreduce/MapReduceRunner.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index 71edf1a0b20a..81988489676d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobStatus; import org.apache.log4j.BasicConfigurator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,6 +96,9 @@ public PipelineResult run(Pipeline pipeline) { try { Job job = jobPrototype.build(options.getJarClass(), config); job.waitForCompletion(true); + if (!job.getStatus().getState().equals(JobStatus.State.SUCCEEDED)) { + throw new RuntimeException("MapReduce job failed: " + job.getJobID()); + } jobs.add(job); } catch (Exception e) { Throwables.throwIfUnchecked(e); From e330d360b4c90899b0ea94060955d519fd190a95 Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 1 Sep 2017 13:06:49 +0800 Subject: [PATCH 28/34] mr-runner: Graph.getSteps() to return with topological order, this fixes few CombineTests. --- .../runners/mapreduce/translation/Graph.java | 34 ++++++++++++------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index 66e573f60fc4..144f9a4f4bb3 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.mapreduce.translation; +import static com.google.common.base.Preconditions.checkState; + import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.FluentIterable; @@ -25,6 +27,7 @@ import com.google.common.graph.ElementOrder; import com.google.common.graph.GraphBuilder; import com.google.common.graph.MutableGraph; +import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -91,24 +94,29 @@ public void removeEdge(StepT step, TagT outTag) { } public List getSteps() { - return castToStepList(FluentIterable.from(graph.nodes()) + List ret = new ArrayList<>(); + + Set pendingNodes = Sets.newHashSet(graph.nodes()); + while (!pendingNodes.isEmpty()) { + List readyNodes = new ArrayList<>(); + for (Vertex v : pendingNodes) { + if (Sets.intersection(pendingNodes, graph.predecessors(v)).isEmpty()) { + readyNodes.add(v); + } + } + checkState( + !readyNodes.isEmpty(), + "No ready nodes found, there are cycles in graph: " + graph); + ret.addAll(readyNodes); + pendingNodes.removeAll(readyNodes); + } + return castToStepList(FluentIterable.from(ret) .filter(new Predicate() { @Override public boolean apply(Vertex input) { return input instanceof AbstractStep; }})) - .toSortedList(new Comparator() { - @Override - public int compare(StepT left, StepT right) { - if (left.equals(right)) { - return 0; - } else if (com.google.common.graph.Graphs.reachableNodes(graph, left).contains(right)) { - return -1; - } else { - return 1; - } - } - }); + .toList(); } public List getStartSteps() { From 8627913eeb0a51a251a953930fc52025dbf8a723 Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 1 Sep 2017 14:10:22 +0800 Subject: [PATCH 29/34] mr-runner: ensure Operation only start/finish once for diamond shaped DAG, this fixes ParDoLifecycleTest. --- .../runners/mapreduce/translation/Operation.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java index bd24f05d7a79..a96806d0e5d7 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Operation.java @@ -31,12 +31,16 @@ public abstract class Operation implements Serializable { private final OutputReceiver[] receivers; private SerializableConfiguration conf; + private boolean started; + private boolean finished; public Operation(int numOutputs) { this.receivers = new OutputReceiver[numOutputs]; for (int i = 0; i < numOutputs; ++i) { receivers[i] = new OutputReceiver(); } + this.started = false; + this.finished = false; } /** @@ -45,6 +49,10 @@ public Operation(int numOutputs) { *

Called after all successors consuming operations have been started. */ public void start(TaskInputOutputContext taskContext) { + if (started) { + return; + } + started = true; conf = new SerializableConfiguration(taskContext.getConfiguration()); for (OutputReceiver receiver : receivers) { if (receiver == null) { @@ -67,6 +75,10 @@ public void start(TaskInputOutputContext taskCon *

Called after all predecessors producing operations have been finished. */ public void finish() { + if (finished) { + return; + } + finished = true; for (OutputReceiver receiver : receivers) { if (receiver == null) { continue; From 99bffd2a75b7461d15723567a57db6d3b17367cd Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 1 Sep 2017 14:11:30 +0800 Subject: [PATCH 30/34] mr-runner: translate empty flatten into EmptySource, this fixes few empty FalttenTests. --- .../translation/FlattenTranslator.java | 71 ++++++++++++++++++- 1 file changed, 68 insertions(+), 3 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java index b966f2a5b356..b86993666927 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java @@ -17,6 +17,14 @@ */ package org.apache.beam.runners.mapreduce.translation; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Flatten; /** @@ -26,11 +34,68 @@ public class FlattenTranslator extends TransformTranslator.Default transform, TranslationContext context) { TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); - - Operation operation = new FlattenOperation(); + List inputTags = userGraphContext.getInputTags(); + Operation operation; + if (inputTags.isEmpty()) { + // Create a empty source + operation = new SourceReadOperation(new EmptySource(), userGraphContext.getOnlyOutputTag()); + } else { + operation = new FlattenOperation(); + } context.addInitStep( Graphs.Step.of(userGraphContext.getStepName(), operation), - userGraphContext.getInputTags(), + inputTags, userGraphContext.getOutputTags()); } + + private static class EmptySource extends BoundedSource { + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + return Collections.EMPTY_LIST; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 0; + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + return new BoundedReader() { + @Override + public BoundedSource getCurrentSource() { + return EmptySource.this; + } + + @Override + public boolean start() throws IOException { + return false; + } + + @Override + public boolean advance() throws IOException { + return false; + } + + @Override + public Void getCurrent() throws NoSuchElementException { + throw new NoSuchElementException(); + } + + @Override + public void close() throws IOException { + } + }; + } + + @Override + public void validate() { + } + + @Override + public Coder getDefaultOutputCoder() { + return VoidCoder.of(); + } + } } From 5248ce42f3ab31e8952f6604ef804b342c57d962 Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 1 Sep 2017 15:10:48 +0800 Subject: [PATCH 31/34] mr-runner: introduces duplicateFactor in FlattenOperation, this fixes testFlattenInputMultipleCopies(). --- .../translation/FlattenOperation.java | 9 ++- .../translation/FlattenTranslator.java | 69 ++++++++++++++++--- 2 files changed, 67 insertions(+), 11 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java index 191b34629e31..3c5ac95baa67 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenOperation.java @@ -24,14 +24,19 @@ */ public class FlattenOperation extends Operation { - public FlattenOperation() { + private final int duplicateFactor; + + public FlattenOperation(int duplicateFactor) { super(1); + this.duplicateFactor = duplicateFactor; } @Override public void process(WindowedValue elem) { for (OutputReceiver receiver : getOutputReceivers()) { - receiver.process(elem); + for (int i = 0; i < duplicateFactor; ++i) { + receiver.process(elem); + } } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java index b86993666927..817f2bf4d89b 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java @@ -17,15 +17,22 @@ */ package org.apache.beam.runners.mapreduce.translation; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.values.TupleTag; /** * Translates a {@link Flatten} to a {@link FlattenOperation}. @@ -34,18 +41,62 @@ public class FlattenTranslator extends TransformTranslator.Default transform, TranslationContext context) { TranslationContext.UserGraphContext userGraphContext = context.getUserGraphContext(); - List inputTags = userGraphContext.getInputTags(); - Operation operation; - if (inputTags.isEmpty()) { + + Map inputTagToCount = Maps.newHashMap(); + boolean containsDuplicates = false; + for (Graphs.Tag inputTag : userGraphContext.getInputTags()) { + Integer count = inputTagToCount.get(inputTag); + if (count == null) { + count = Integer.valueOf(0); + } + inputTagToCount.put(inputTag, ++count); + if (count > 1) { + containsDuplicates = true; + } + } + + if (inputTagToCount.isEmpty()) { // Create a empty source - operation = new SourceReadOperation(new EmptySource(), userGraphContext.getOnlyOutputTag()); + Operation operation = + new SourceReadOperation(new EmptySource(), userGraphContext.getOnlyOutputTag()); + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), operation), + userGraphContext.getInputTags(), + userGraphContext.getOutputTags()); + } else if (!containsDuplicates) { + Operation operation = new FlattenOperation(1); + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), operation), + userGraphContext.getInputTags(), + userGraphContext.getOutputTags()); } else { - operation = new FlattenOperation(); + List intermediateTags = new ArrayList<>(); + for (Map.Entry entry : inputTagToCount.entrySet()) { + Integer dupFactor = entry.getValue(); + Graphs.Tag inTag = entry.getKey(); + checkState( + dupFactor > 0, "dupFactor should be positive, but was: " + dupFactor); + if (dupFactor == 1) { + intermediateTags.add(inTag); + } else { + String dupStepName = userGraphContext.getStepName() + "/Dup-" + dupFactor; + Graphs.Tag outTag = Graphs.Tag.of( + dupStepName + ".out", + new TupleTag(), + inTag.getCoder(), + inTag.getWindowingStrategy()); + context.addInitStep( + Graphs.Step.of(dupStepName, new FlattenOperation(dupFactor)), + ImmutableList.of(inTag), + ImmutableList.of(outTag)); + intermediateTags.add(outTag); + } + } + context.addInitStep( + Graphs.Step.of(userGraphContext.getStepName(), new FlattenOperation(1)), + intermediateTags, + userGraphContext.getOutputTags()); } - context.addInitStep( - Graphs.Step.of(userGraphContext.getStepName(), operation), - inputTags, - userGraphContext.getOutputTags()); } private static class EmptySource extends BoundedSource { From c62b3ad462c2c07ce36cce025dc52204e7eb87d2 Mon Sep 17 00:00:00 2001 From: Pei He Date: Fri, 1 Sep 2017 16:55:19 +0800 Subject: [PATCH 32/34] mr-runner: support SourceMetrics, this fixes MetricsTest.testBoundedSourceMetrics(). --- .../translation/BeamInputFormat.java | 39 ++++++++++++++----- .../translation/FileReadOperation.java | 4 ++ .../translation/FlattenTranslator.java | 12 +++--- .../mapreduce/translation/GraphPlanner.java | 3 +- .../mapreduce/translation/JobPrototype.java | 4 +- .../translation/ReadBoundedTranslator.java | 7 ++-- .../mapreduce/translation/ReadOperation.java | 5 ++- .../translation/SourceReadOperation.java | 6 ++- 8 files changed, 56 insertions(+), 24 deletions(-) diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java index 10d9ada80ba5..9dc339695a09 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -26,12 +26,15 @@ import com.google.common.collect.ImmutableList; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.Closeable; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.List; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; @@ -87,7 +90,8 @@ public Iterable apply( .transform(new Function, ReadOperation.TaggedSource>() { @Override public ReadOperation.TaggedSource apply(BoundedSource input) { - return ReadOperation.TaggedSource.of(input, taggedSource.getTag()); + return ReadOperation.TaggedSource.of( + taggedSource.getStepName(), input, taggedSource.getTag()); }}); } catch (Exception e) { Throwables.throwIfUnchecked(e); @@ -98,7 +102,8 @@ public ReadOperation.TaggedSource apply(BoundedSource input) { .transform(new Function() { @Override public InputSplit apply(ReadOperation.TaggedSource taggedSource) { - return new BeamInputSplit(taggedSource.getSource(), options, taggedSource.getTag()); + return new BeamInputSplit(taggedSource.getStepName(), taggedSource.getSource(), + options, taggedSource.getTag()); }}) .toList(); } catch (Exception e) { @@ -113,6 +118,7 @@ public RecordReader createRecordReader( } public static class BeamInputSplit extends InputSplit implements Writable { + private String stepName; private BoundedSource boundedSource; private SerializedPipelineOptions options; private TupleTag tupleTag; @@ -121,9 +127,11 @@ public BeamInputSplit() { } public BeamInputSplit( + String stepName, BoundedSource boundedSource, SerializedPipelineOptions options, TupleTag tupleTag) { + this.stepName = checkNotNull(stepName, "stepName"); this.boundedSource = checkNotNull(boundedSource, "boundedSources"); this.options = checkNotNull(options, "options"); this.tupleTag = checkNotNull(tupleTag, "tupleTag"); @@ -131,7 +139,7 @@ public BeamInputSplit( public BeamRecordReader createReader() throws IOException { return new BeamRecordReader<>( - boundedSource.createReader(options.getPipelineOptions()), tupleTag); + stepName, boundedSource.createReader(options.getPipelineOptions()), tupleTag); } @Override @@ -154,6 +162,7 @@ public String[] getLocations() throws IOException, InterruptedException { @Override public void write(DataOutput out) throws IOException { ByteArrayOutputStream stream = new ByteArrayOutputStream(); + StringUtf8Coder.of().encode(stepName, stream); SerializableCoder.of(BoundedSource.class).encode(boundedSource, stream); SerializableCoder.of(SerializedPipelineOptions.class).encode(options, stream); SerializableCoder.of(TupleTag.class).encode(tupleTag, stream); @@ -170,6 +179,7 @@ public void readFields(DataInput in) throws IOException { in.readFully(bytes); ByteArrayInputStream inStream = new ByteArrayInputStream(bytes); + stepName = StringUtf8Coder.of().decode(inStream); boundedSource = SerializableCoder.of(BoundedSource.class).decode(inStream); options = SerializableCoder.of(SerializedPipelineOptions.class).decode(inStream); tupleTag = SerializableCoder.of(TupleTag.class).decode(inStream); @@ -178,11 +188,15 @@ public void readFields(DataInput in) throws IOException { private static class BeamRecordReader extends RecordReader { + private final String stepName; private final BoundedSource.BoundedReader reader; - private TupleTag tupleTag; + private final TupleTag tupleTag; + private MetricsReporter metricsReporter; private boolean started; - public BeamRecordReader(BoundedSource.BoundedReader reader, TupleTag tupleTag) { + public BeamRecordReader( + String stepName, BoundedSource.BoundedReader reader, TupleTag tupleTag) { + this.stepName = checkNotNull(stepName, "stepName"); this.reader = checkNotNull(reader, "reader"); this.tupleTag = checkNotNull(tupleTag, "tupleTag"); this.started = false; @@ -191,15 +205,19 @@ public BeamRecordReader(BoundedSource.BoundedReader reader, TupleTag tuple @Override public void initialize( InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { + this.metricsReporter = new MetricsReporter(context); } @Override public boolean nextKeyValue() throws IOException, InterruptedException { - if (!started) { - started = true; - return reader.start(); - } else { - return reader.advance(); + try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer( + metricsReporter.getMetricsContainer(stepName))) { + if (!started) { + started = true; + return reader.start(); + } else { + return reader.advance(); + } } } @@ -233,6 +251,7 @@ public float getProgress() throws IOException, InterruptedException { @Override public void close() throws IOException { reader.close(); + metricsReporter.updateMetrics(); } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java index cbbfbd252536..f2122529568c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java @@ -46,15 +46,18 @@ */ public class FileReadOperation extends ReadOperation> { + private final String stepName; private final String fileName; private final Coder> coder; private final TupleTag tupleTag; public FileReadOperation( + String stepName, String fileName, Coder> coder, TupleTag tupleTag) { super(); + this.stepName = checkNotNull(stepName, "stepName"); this.fileName = checkNotNull(fileName, "fileName"); this.coder = checkNotNull(coder, "coder"); this.tupleTag = checkNotNull(tupleTag, "tupleTag"); @@ -63,6 +66,7 @@ public FileReadOperation( @Override TaggedSource getTaggedSource(Configuration conf) { return TaggedSource.of( + stepName, new FileBoundedSource<>(fileName, coder, new SerializableConfiguration(conf)), tupleTag); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java index 817f2bf4d89b..06ad367bbc3d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FlattenTranslator.java @@ -55,18 +55,20 @@ public void translateNode(Flatten.PCollections transform, TranslationContext } } + String stepName = userGraphContext.getStepName(); if (inputTagToCount.isEmpty()) { // Create a empty source Operation operation = - new SourceReadOperation(new EmptySource(), userGraphContext.getOnlyOutputTag()); + new SourceReadOperation( + stepName, new EmptySource(), userGraphContext.getOnlyOutputTag()); context.addInitStep( - Graphs.Step.of(userGraphContext.getStepName(), operation), + Graphs.Step.of(stepName, operation), userGraphContext.getInputTags(), userGraphContext.getOutputTags()); } else if (!containsDuplicates) { Operation operation = new FlattenOperation(1); context.addInitStep( - Graphs.Step.of(userGraphContext.getStepName(), operation), + Graphs.Step.of(stepName, operation), userGraphContext.getInputTags(), userGraphContext.getOutputTags()); } else { @@ -79,7 +81,7 @@ public void translateNode(Flatten.PCollections transform, TranslationContext if (dupFactor == 1) { intermediateTags.add(inTag); } else { - String dupStepName = userGraphContext.getStepName() + "/Dup-" + dupFactor; + String dupStepName = stepName + "/Dup-" + dupFactor; Graphs.Tag outTag = Graphs.Tag.of( dupStepName + ".out", new TupleTag(), @@ -93,7 +95,7 @@ public void translateNode(Flatten.PCollections transform, TranslationContext } } context.addInitStep( - Graphs.Step.of(userGraphContext.getStepName(), new FlattenOperation(1)), + Graphs.Step.of(stepName, new FlattenOperation(1)), intermediateTags, userGraphContext.getOutputTags()); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 6c79277684ed..09998ead92a8 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -83,7 +83,8 @@ public Graphs.FusedGraph plan(Graphs.FusedGraph fusedGraph) { consumer.addStep( Graphs.Step.of( readStepName, - new FileReadOperation(filePath, writeValueCoder, tag.getTupleTag())), + new FileReadOperation( + readStepName, filePath, writeValueCoder, tag.getTupleTag())), ImmutableList.of(), ImmutableList.of(readOutput)); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java index 44f279b39458..e8e6eab07608 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/JobPrototype.java @@ -42,6 +42,7 @@ import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.counters.Limits; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; @@ -77,8 +78,9 @@ public Job build(Class jarClass, Configuration initConf) throws IOException { "io.serializations", "org.apache.hadoop.io.serializer.WritableSerialization," + "org.apache.hadoop.io.serializer.JavaSerialization"); + conf.set("mapreduce.job.counters.group.name.max", "512"); + Limits.init(conf); - //TODO: config out dir with PipelineOptions. conf.set( FileOutputFormat.OUTDIR, configUtils.getFileOutputDir(fusedStep.getStageId())); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java index 138c00ea7a5a..5e5c99bf027e 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReadBoundedTranslator.java @@ -27,10 +27,11 @@ class ReadBoundedTranslator extends TransformTranslator.Default getSource(); abstract TupleTag getTag(); - static TaggedSource of(BoundedSource boundedSource, TupleTag tupleTag) { + static TaggedSource of(String stepName, BoundedSource boundedSource, TupleTag tupleTag) { return new org.apache.beam.runners.mapreduce.translation - .AutoValue_ReadOperation_TaggedSource(boundedSource, tupleTag); + .AutoValue_ReadOperation_TaggedSource(stepName, boundedSource, tupleTag); } } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java index 19b032033692..55b46a40030d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/SourceReadOperation.java @@ -27,12 +27,14 @@ * Operation that reads from {@link BoundedSource}. */ public class SourceReadOperation extends ReadOperation { + private final String stepName; private final TaggedSource source; - SourceReadOperation(BoundedSource boundedSource, TupleTag tupleTag) { + SourceReadOperation(String stepName, BoundedSource boundedSource, TupleTag tupleTag) { + this.stepName = checkNotNull(stepName, "stepName"); checkNotNull(boundedSource, "boundedSource"); checkNotNull(tupleTag, "tupleTag"); - this.source = TaggedSource.of(boundedSource, tupleTag); + this.source = TaggedSource.of(stepName, boundedSource, tupleTag); } @Override From 9d1db98af41eb5ab375c51a7057bd6b52dbabfc6 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 4 Sep 2017 11:06:11 +0800 Subject: [PATCH 33/34] mr-runner-hack: disable unrelated modules to shorten build time during development. --- pom.xml | 2 +- sdks/java/pom.xml | 3 ++- sdks/pom.xml | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index e0ec1362a678..25cd51b66e58 100644 --- a/pom.xml +++ b/pom.xml @@ -183,7 +183,7 @@ sdks/java/build-tools sdks runners - examples + sdks/java/javadoc diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index 3144193b9839..75b204327e0a 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -37,7 +37,8 @@ and other project configuration to be used in all modules. build-tools --> core - io + + io/hadoop-file-system maven-archetypes extensions From 32aeb7ac3d49ade0dc3ad79e711e7b624091d485 Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 31 Aug 2017 16:21:17 +0800 Subject: [PATCH 34/34] mr-runner: Removes WordCountTest, fixes checkstyle, findbugs, and addressed comments. --- runners/map-reduce/pom.xml | 2 +- .../mapreduce/MapReducePipelineOptions.java | 3 + .../mapreduce/MapReducePipelineResult.java | 6 + .../runners/mapreduce/MapReduceRegistrar.java | 6 + .../runners/mapreduce/MapReduceRunner.java | 5 +- .../beam/runners/mapreduce/package-info.java | 2 +- .../translation/BeamInputFormat.java | 2 +- .../mapreduce/translation/BeamMapper.java | 4 +- .../mapreduce/translation/BeamReducer.java | 6 +- .../translation/ConfigurationUtils.java | 1 - .../mapreduce/translation/DotfileWriter.java | 22 ++-- .../translation/FileReadOperation.java | 1 - .../runners/mapreduce/translation/Graph.java | 8 +- .../mapreduce/translation/GraphConverter.java | 2 +- .../mapreduce/translation/GraphPlanner.java | 1 - .../runners/mapreduce/translation/Graphs.java | 13 ++ .../translation/PartitionOperation.java | 2 - ...eifyTimestampAndWindowsParDoOperation.java | 2 +- .../translation/ShuffleWriteOperation.java | 1 - .../translation/TranslationContext.java | 7 +- .../beam/runners/mapreduce/WordCountTest.java | 117 ------------------ 21 files changed, 65 insertions(+), 148 deletions(-) delete mode 100644 runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml index 3b253a797eaf..90d876b6dccd 100644 --- a/runners/map-reduce/pom.xml +++ b/runners/map-reduce/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - beam-runners-map-reduce + beam-runners-mapreduce Apache Beam :: Runners :: MapReduce diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java index cfbc006d4157..7cff40d3f32b 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineOptions.java @@ -48,6 +48,9 @@ public interface MapReducePipelineOptions extends PipelineOptions { String getFileOutputDir(); void setFileOutputDir(String fileOutputDir); + /** + * Returns the {@link Class} that constructs MapReduce job through Beam. + */ class JarClassInstanceFactory implements DefaultValueFactory> { @Override public Class create(PipelineOptions options) { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java index 90c521a7e4f9..933d8f6bb589 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReducePipelineResult.java @@ -27,6 +27,12 @@ import org.apache.hadoop.mapreduce.Job; import org.joda.time.Duration; +/** + * A {@link PipelineResult} of executing {@link org.apache.beam.sdk.Pipeline Pipelines} using + * {@link MapReduceRunner}. + * + *

It is synchronous (returned after the pipeline is finished), and is used for querying metrics. + */ public class MapReducePipelineResult implements PipelineResult { private final List jobs; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java index c8b0eeafe30e..1029218bcaf8 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRegistrar.java @@ -31,6 +31,9 @@ public class MapReduceRegistrar { private MapReduceRegistrar() { } + /** + * Registers the {@link MapReduceRunner}. + */ @AutoService(PipelineRunnerRegistrar.class) public static class Runner implements PipelineRunnerRegistrar { @Override @@ -39,6 +42,9 @@ public Iterable>> getPipelineRunners() { } } + /** + * Registers the {@link MapReducePipelineOptions}. + */ @AutoService(PipelineOptionsRegistrar.class) public static class Options implements PipelineOptionsRegistrar { @Override diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java index 81988489676d..85b7d1b2afcc 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/MapReduceRunner.java @@ -41,7 +41,10 @@ import org.slf4j.LoggerFactory; /** - * {@link PipelineRunner} for MapReduce. + * {@link PipelineRunner} for Hadoop MapReduce. + * + *

It translates a Beam {@link Pipeline} to a series of MapReduce {@link Job jobs}, and executes + * them locally or on a Hadoop cluster. */ public class MapReduceRunner extends PipelineRunner { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java index d5114058a230..e452d92aade2 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/package-info.java @@ -16,6 +16,6 @@ * limitations under the License. */ /** - * MapReduce runner implementation. + * Implementation of the Beam runner for Apache Hadoop MapReduce. */ package org.apache.beam.runners.mapreduce; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java index 9dc339695a09..3d0b8eaa8729 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamInputFormat.java @@ -117,7 +117,7 @@ public RecordReader createRecordReader( return ((BeamInputSplit) split).createReader(); } - public static class BeamInputSplit extends InputSplit implements Writable { + private static class BeamInputSplit extends InputSplit implements Writable { private String stepName; private BoundedSource boundedSource; private SerializedPipelineOptions options; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java index b03236fd02bd..46c74c014dde 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamMapper.java @@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory; /** - * Adapter for executing Beam transforms in {@link Mapper}. + * Adapter for executing {@link Operation operations} in {@link Mapper}. */ public class BeamMapper extends Mapper, Object, WindowedValue> { @@ -58,6 +58,8 @@ protected void map( Mapper, Object, WindowedValue>.Context context) throws IOException, InterruptedException { LOG.info("key: {} value: {}.", key, value); + // Only needs to pass KV to the following PartitionOperation. However, we have to wrap it in a + // global window because of the method signature. operation.process(WindowedValue.valueInGlobalWindow(KV.of(key, value))); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java index a38290427f43..b69be32d9d6a 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/BeamReducer.java @@ -33,15 +33,13 @@ import org.apache.beam.sdk.values.KV; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.TaskInputOutputContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** - * Adapter for executing Beam transforms in {@link Reducer}. + * Adapter for executing {@link Operation operations} in {@link Reducer}. */ public class BeamReducer extends Reducer> { @@ -92,6 +90,8 @@ public Object apply(byte[] input) { }})); Object decodedKey = keyCoder.decode(new ByteArrayInputStream(key.getBytes())); LOG.info("key: {} value: {}.", decodedKey, decodedValues); + // Only needs to pass KV to the following GABW operation. However, we have to wrap it in a + // global window because of the method signature. operation.process( WindowedValue.valueInGlobalWindow(KV.of(decodedKey, decodedValues))); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java index 4ec50bd284bb..a905d29685c2 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ConfigurationUtils.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java index 863c4c96b3a3..12cc03c49550 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/DotfileWriter.java @@ -36,18 +36,18 @@ String toDotfile(Graphs.FusedGraph fusedGraph) { int i = 0; for (Graphs.FusedStep fusedStep : fusedGraph.getFusedSteps()) { String clusterId = String.format("cluster_%d", i++); - sb.append(String.format(" subgraph \"%s\" {\n", clusterId)); - sb.append(String.format(" \"%s\" [shape=point style=invis];\n", clusterId)); + sb.append(String.format(" subgraph \"%s\" {%n", clusterId)); + sb.append(String.format(" \"%s\" [shape=point style=invis];%n", clusterId)); fusedStepToId.put(fusedStep, clusterId); Set nodeDefines = Sets.newHashSet(); for (Graphs.Step step : fusedStep.getSteps()) { - nodeDefines.add(String.format(" \"%s\" [shape=box];\n", step.getFullName())); + nodeDefines.add(String.format(" \"%s\" [shape=box];%n", step.getFullName())); for (Graph.AbstractTag inTag : fusedStep.getInputTags(step)) { - nodeDefines.add(String.format(" \"%s\" [shape=ellipse];\n", inTag)); + nodeDefines.add(String.format(" \"%s\" [shape=ellipse];%n", inTag)); } for (Graph.AbstractTag outTag : fusedStep.getOutputTags(step)) { - nodeDefines.add(String.format(" \"%s\" [shape=ellipse];\n", outTag)); + nodeDefines.add(String.format(" \"%s\" [shape=ellipse];%n", outTag)); } } for (String str : nodeDefines) { @@ -59,16 +59,16 @@ String toDotfile(Graphs.FusedGraph fusedGraph) { // Edges within fused steps. for (Graphs.Step step : fusedStep.getSteps()) { for (Graph.AbstractTag inTag : fusedStep.getInputTags(step)) { - sb.append(String.format(" \"%s\" -> \"%s\";\n", inTag, step)); + sb.append(String.format(" \"%s\" -> \"%s\";%n", inTag, step)); } for (Graph.AbstractTag outTag : fusedStep.getOutputTags(step)) { - sb.append(String.format(" \"%s\" -> \"%s\";\n", step, outTag)); + sb.append(String.format(" \"%s\" -> \"%s\";%n", step, outTag)); } } // Edges between sub-graphs. for (Graphs.Tag inTag : fusedGraph.getInputTags(fusedStep)) { - sb.append(String.format(" \"%s\" -> \"%s\";\n", inTag, fusedStepToId.get(fusedStep))); + sb.append(String.format(" \"%s\" -> \"%s\";%n", inTag, fusedStepToId.get(fusedStep))); } } sb.append("}\n"); @@ -79,12 +79,12 @@ public static String toDotfile(Graphs.FusedStep fusedStep) { StringBuilder sb = new StringBuilder(); sb.append("\ndigraph G {\n"); for (Graphs.Step step : fusedStep.getSteps()) { - sb.append(String.format(" \"%s\" [shape=box];\n", step.getFullName())); + sb.append(String.format(" \"%s\" [shape=box];%n", step.getFullName())); for (Graph.AbstractTag inTag : fusedStep.getInputTags(step)) { - sb.append(String.format(" \"%s\" -> \"%s\";\n", inTag, step)); + sb.append(String.format(" \"%s\" -> \"%s\";%n", inTag, step)); } for (Graph.AbstractTag outTag : fusedStep.getOutputTags(step)) { - sb.append(String.format(" \"%s\" -> \"%s\";\n", step, outTag)); + sb.append(String.format(" \"%s\" -> \"%s\";%n", step, outTag)); } } sb.append("}\n"); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java index f2122529568c..eb5bef49d298 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/FileReadOperation.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java index 144f9a4f4bb3..b4549d3761a8 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graph.java @@ -29,9 +29,7 @@ import com.google.common.graph.MutableGraph; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.List; -import java.util.ListIterator; import java.util.Objects; import java.util.Set; @@ -206,9 +204,15 @@ public int hashCode() { interface Vertex { } + /** + * Step {@link Vertex}. + */ public abstract static class AbstractStep implements Vertex { } + /** + * Tag {@link Vertex}. + */ public abstract static class AbstractTag implements Vertex { } } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java index 458961f388da..1a4988b5c1cd 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphConverter.java @@ -123,7 +123,7 @@ private void markEnclosedTransformCounts(TransformHierarchy.Node node) { public String getDotfile() { return String.format( - "\ndigraph G {\n%s%s}\n", + "%ndigraph G {%n%s%s}%n", dotfileNodesBuilders.peek().toString(), dotfileEdgesBuilder.toString()); } diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java index 09998ead92a8..bc360fb4bf3d 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/GraphPlanner.java @@ -29,7 +29,6 @@ import org.apache.beam.runners.mapreduce.MapReducePipelineOptions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; /** * Class that optimizes the initial graph to a fused graph. diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java index 0b93c3aa7f9e..f23e572d1290 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/Graphs.java @@ -34,6 +34,9 @@ public class Graphs { private Graphs() {} + /** + * Class that represents an optimized graph. + */ public static class FusedGraph { private final Graph graph; private int stageId = 0; @@ -121,6 +124,10 @@ public List getOutputTags(FusedStep fusedStep) { } } + /** + * An {@link Graph.AbstractStep} that represents an optimized sub-graph that can be executed + * in one MapReduce job. + */ public static class FusedStep extends Graph.AbstractStep { private final int stageId; private final Graph steps; @@ -213,6 +220,9 @@ public String toString() { } } + /** + * An {@link Graph.AbstractStep} that represents one {@link Operation}. + */ @AutoValue public abstract static class Step extends Graph.AbstractStep { abstract String getFullName(); @@ -230,6 +240,9 @@ public String toString() { } } + /** + * An {@link Graph.AbstractTag} that contains information about input/output data. + */ @AutoValue public abstract static class Tag extends Graph.AbstractTag implements Serializable { abstract String getName(); diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java index 687b5b9e0485..dc0f81ac512f 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/PartitionOperation.java @@ -20,8 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import com.google.common.base.Function; -import com.google.common.collect.FluentIterable; import java.io.IOException; import java.util.List; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java index 9d6b895e00d6..0e02bbb822be 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ReifyTimestampAndWindowsParDoOperation.java @@ -45,7 +45,7 @@ DoFn getDoFn() { return (DoFn) new ReifyTimestampAndWindowsDoFn<>(); } - public class ReifyTimestampAndWindowsDoFn + private static class ReifyTimestampAndWindowsDoFn extends DoFn, KV>> { @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) throws Exception { diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ShuffleWriteOperation.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ShuffleWriteOperation.java index 782cfef9b444..a8fae1b46a6c 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ShuffleWriteOperation.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/ShuffleWriteOperation.java @@ -19,7 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import com.google.common.base.Throwables; import java.io.ByteArrayOutputStream; import java.io.IOException; import org.apache.beam.sdk.coders.Coder; diff --git a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java index 93856de38b74..e908e9392cf0 100644 --- a/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java +++ b/runners/map-reduce/src/main/java/org/apache/beam/runners/mapreduce/translation/TranslationContext.java @@ -86,7 +86,7 @@ public void setCurrentNode(TransformHierarchy.Node node) { this.currentNode = node; for (Map.Entry, PValue> entry : currentNode.getOutputs().entrySet()) { pValueToTupleTag.put(entry.getValue(), entry.getKey()); - // TODO: this is a hack to get around that ViewAsXXX.expand() return wrong output PValue. + // TODO: this is a hack to get around that ViewAsXYZ.expand() return wrong output PValue. if (node.getTransform() instanceof View.CreatePCollectionView) { View.CreatePCollectionView view = (View.CreatePCollectionView) node.getTransform(); pValueToTupleTag.put(view.getView(), view.getView().getTagInternal()); @@ -125,7 +125,10 @@ public Graphs.Tag apply(PValue pValue) { if (pValue instanceof PCollection) { PCollection pc = (PCollection) pValue; return Graphs.Tag.of( - pc.getName(), pValueToTupleTag.get(pValue), pc.getCoder(), pc.getWindowingStrategy()); + pc.getName(), + pValueToTupleTag.get(pValue), + pc.getCoder(), + pc.getWindowingStrategy()); } else if (pValue instanceof PCollectionView){ PCollectionView pView = (PCollectionView) pValue; return Graphs.Tag.of( diff --git a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java b/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java deleted file mode 100644 index 263905c66b10..000000000000 --- a/runners/map-reduce/src/test/java/org/apache/beam/runners/mapreduce/WordCountTest.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.mapreduce; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricResult; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.metrics.MetricsFilter; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.KV; -import org.apache.log4j.BasicConfigurator; -import org.joda.time.Duration; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Test that runs WordCount. - */ -@RunWith(JUnit4.class) -public class WordCountTest { - - public static final String TOKENIZER_PATTERN = "[^\\p{L}]+"; - - /** - * Concept #2: You can make your pipeline assembly code less verbose by defining your DoFns - * statically out-of-line. This DoFn tokenizes lines of text into individual words; we pass it - * to a ParDo in the pipeline. - */ - static class ExtractWordsFn extends DoFn { - private final Counter emptyLines = Metrics.counter(ExtractWordsFn.class, "emptyLines"); - private final Counter nonEmptyLines = Metrics.counter(ExtractWordsFn.class, "nonEmptyLines"); - - @ProcessElement - public void processElement(ProcessContext c) { - if (c.element().trim().isEmpty()) { - emptyLines.inc(); - } else { - nonEmptyLines.inc(); - } - - // Split the line into words. - String[] words = c.element().split(TOKENIZER_PATTERN); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - /** A SimpleFunction that converts a Word and Count into a printable string. */ - public static class FormatAsTextFn extends SimpleFunction, String> { - @Override - public String apply(KV input) { - return input.getKey() + ": " + input.getValue(); - } - } - - @Test - public void testWordCount() { - BasicConfigurator.configure(); - - String input = "/Users/peihe/github/beam/LICENSE"; - String output = "./output"; - MapReducePipelineOptions options = PipelineOptionsFactory.as(MapReducePipelineOptions.class); - //options.setJarClass(this.getClass()); - options.setRunner(MapReduceRunner.class); - Pipeline p = Pipeline.create(options); - - // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the - // static FormatAsTextFn() to the ParDo transform. - p.apply("ReadLines", TextIO.read().from(input)) - .apply(Window.into(FixedWindows.of(Duration.millis(1000)))) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()) - .apply(MapElements.via(new FormatAsTextFn())) - .apply("WriteCounts", TextIO.write().to(output)); - - PipelineResult result = p.run(); - Iterable> counters = result.metrics() - .queryMetrics( - MetricsFilter.builder() - .addNameFilter(MetricNameFilter.named(ExtractWordsFn.class, "emptyLines")) - .build()) - .counters(); - System.out.println(counters.iterator().next()); - } -}