[NEMO-356] Visualize the name of beam transform in DAG (#199)
JIRA: [NEMO-356: Visualize the name of beam transform in DAG
](https://issues.apache.org/jira/projects/NEMO/issues/NEMO-356)
**Major changes:**
- populate the name of Beam transform.
The visualized DAGs have been uploaded to https://issues.apache.org/jira/projects/NEMO/issues/NEMO-356
diff --git a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java
index 7c63357..816c539 100644
--- a/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java
+++ b/compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/PipelineTranslator.java
@@ -26,6 +26,7 @@
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.runners.TransformHierarchy;
import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
import org.apache.nemo.common.ir.edge.IREdge;
import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
@@ -431,6 +432,10 @@
final PCollection<?> mainInput = (PCollection<?>)
Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(pTransform));
+ final HasDisplayData displayData = (builder) -> {
+ builder.add(DisplayData.item("name", beamNode.getFullName()));
+ };
+
if (sideInputMap.isEmpty()) {
return new DoFnTransform(
doFn,
@@ -440,7 +445,7 @@
additionalOutputTags.getAll(),
mainInput.getWindowingStrategy(),
ctx.getPipelineOptions(),
- DisplayData.from(beamNode.getTransform()));
+ DisplayData.from(displayData));
} else {
return new PushBackDoFnTransform(
doFn,
@@ -451,7 +456,7 @@
mainInput.getWindowingStrategy(),
sideInputMap,
ctx.getPipelineOptions(),
- DisplayData.from(beamNode.getTransform()));
+ DisplayData.from(displayData));
}
} catch (final IOException e) {