[ 
https://issues.apache.org/jira/browse/BEAM-5859?focusedWorklogId=172600&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-172600
 ]

ASF GitHub Bot logged work on BEAM-5859:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 06/Dec/18 09:28
            Start Date: 06/Dec/18 09:28
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on a change in pull request #7208: 
[BEAM-5859] Better handle fused composite stage names.
URL: https://github.com/apache/beam/pull/7208#discussion_r239379478
 
 

 ##########
 File path: 
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExecutableStageTranslation.java
 ##########
 @@ -45,19 +53,60 @@ public static String 
generateNameFromStagePayload(ExecutableStagePayload stagePa
     RunnerApi.Components components = stagePayload.getComponents();
     final int transformsCount = stagePayload.getTransformsCount();
     sb.append("[").append(transformsCount).append("]");
-    sb.append("{");
+    Collection<String> names = new ArrayList<>();
     for (int i = 0; i < transformsCount; i++) {
       String name = 
components.getTransformsOrThrow(stagePayload.getTransforms(i)).getUniqueName();
-      // Python: Remove the 'ref_AppliedPTransform_' prefix which just makes 
the name longer
-      name = name.replaceFirst("^ref_AppliedPTransform_", "");
       // Java: Remove the 'ParMultiDo(Anonymous)' suffix which just makes the 
name longer
       name = name.replaceFirst("/ParMultiDo\\(Anonymous\\)$", "");
-      sb.append(name);
-      if (i + 1 < transformsCount) {
-        sb.append(", ");
-      }
+      names.add(name);
     }
-    sb.append("}");
+    sb.append(generateNameFromTransformNames(names, true));
     return sb.toString();
   }
+
+  public static String generateNameFromTransformNames(Collection<String> 
names, boolean truncate) {
+    Multimap<String, String> groupByOuter = LinkedHashMultimap.create();
+    for (String name : names) {
+      int index = name.indexOf('/');
 
 Review comment:
   This is all we have to go on here. It's up to the SDK to choose how to 
follow this convention. If an extra slash is present, it'll just look more 
nested than it really is. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 172600)

> Improve Traceability of Pipeline translation
> --------------------------------------------
>
>                 Key: BEAM-5859
>                 URL: https://issues.apache.org/jira/browse/BEAM-5859
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-flink
>            Reporter: Maximilian Michels
>            Priority: Major
>              Labels: portability, portability-flink
>         Attachments: tfx.png, wordcount.png
>
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> Users often ask how they can reason about the pipeline translation. The Flink 
> UI display a confusingly large graph without any trace of the original Beam 
> pipeline:
> WordCount:
>  !wordcount.png! 
> TFX:
>  !tfx.png! 
> Some aspects which make understanding these graphs hard:
>  * Users don't know how the Runner maps Beam to Flink concepts
>  * The UI is awfully slow / hangs when the pipeline is reasonable complex
>  * The operator names seem to use {{transform.getUniqueName()}} which doesn't 
> generate readable name
>  * So called Chaining combines operators into a single operator which makes 
> understanding which Beam concept belongs to which Flink concept even harder
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to