Repository: incubator-beam
Updated Branches:
  refs/heads/master 13b45895e -> 73226168a


Fix inconsistent in formatting logs: leaveCompositeTransform always decrement 
depth, but enterCompositeTransform increment depth only on ENTER_TRANSFORM


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/cea201ea
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/cea201ea
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/cea201ea

Branch: refs/heads/master
Commit: cea201eaaea24d8cc1e117645d1c81f379beeb41
Parents: 98da6e8
Author: Alexey Diomin <diomi...@gmail.com>
Authored: Wed Aug 31 18:17:01 2016 +0400
Committer: Alexey Diomin <diomi...@gmail.com>
Committed: Wed Aug 31 18:17:54 2016 +0400

----------------------------------------------------------------------
 .../runners/flink/translation/FlinkBatchPipelineTranslator.java    | 2 +-
 .../flink/translation/FlinkStreamingPipelineTranslator.java        | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cea201ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git 
a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
 
b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
index 66c48b0..1cb604f 100644
--- 
a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
+++ 
b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
@@ -65,6 +65,7 @@ public class FlinkBatchPipelineTranslator extends 
FlinkPipelineTranslator {
   @Override
   public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
     LOG.info(genSpaces(this.depth) + "enterCompositeTransform- " + 
formatNodeName(node));
+    this.depth++;
 
     BatchTransformTranslator<?> translator = getTranslator(node);
 
@@ -73,7 +74,6 @@ public class FlinkBatchPipelineTranslator extends 
FlinkPipelineTranslator {
       LOG.info(genSpaces(this.depth) + "translated-" + formatNodeName(node));
       return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
     } else {
-      this.depth++;
       return CompositeBehavior.ENTER_TRANSFORM;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/cea201ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git 
a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
 
b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
index 284cd23..e5c0d76 100644
--- 
a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
+++ 
b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
@@ -52,6 +52,7 @@ public class FlinkStreamingPipelineTranslator extends 
FlinkPipelineTranslator {
   @Override
   public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
     LOG.info(genSpaces(this.depth) + "enterCompositeTransform- " + 
formatNodeName(node));
+    this.depth++;
 
     PTransform<?, ?> transform = node.getTransform();
     if (transform != null) {
@@ -64,7 +65,6 @@ public class FlinkStreamingPipelineTranslator extends 
FlinkPipelineTranslator {
         return CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
       }
     }
-    this.depth++;
     return CompositeBehavior.ENTER_TRANSFORM;
   }
 

Reply via email to