You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by GitBox <gi...@apache.org> on 2020/03/19 23:19:05 UTC

[GitHub] [beam] KevinGG opened a new pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally

KevinGG opened a new pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally
URL: https://github.com/apache/beam/pull/11174
 
 
   1. The problem: when the CombineGlobally raises the ValueError, it
   breaks out the process of transform `apply` where the current transform
   has been pushed into a transform stack, to be expanded and then popped out in the end.
   Without the popping out, the pipeline will hold a current transform that
   is referring the failed CombineGlobally transform and never changes,
   meaning the pipeline will not accept any new transforms after.
   2. It did not affect non-interactive usages because when the pipeline
   construction runs into an error, the user has to restart the whole
   program and nothing has been executed at the point the error is raised,
   so no side-effects.
   3. It did leave the pipeline in a dead state when the user constructs
   pipelines in an interactive environment.
   4. The pop before raising the error allows the transform stack to
   recover from the failed state.
   
   **Please** add a meaningful description for your change here
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
   Java | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
   Python | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/) | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/)
   XLang | --- | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/)
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/) 
   Portable | --- | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   

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


With regards,
Apache Git Services

[GitHub] [beam] robertwb commented on a change in pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally

Posted by GitBox <gi...@apache.org>.
robertwb commented on a change in pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally
URL: https://github.com/apache/beam/pull/11174#discussion_r395883196
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1811,6 +1811,8 @@ def add_input_types(transform):
       return view
     else:
       if pcoll.windowing.windowfn != GlobalWindows():
+        # Remove the broken transform when running into value error.
+        pcoll.pipeline.transforms_stack.pop()
 
 Review comment:
   E.g. put this https://github.com/apache/beam/blob/release-2.19.0/sdks/python/apache_beam/pipeline.py#L330 in a finally clause of a try block that starts where it's pushed. 

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


With regards,
Apache Git Services

[GitHub] [beam] robertwb commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
robertwb commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#issuecomment-601929637
 
 
   retest this please

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


With regards,
Apache Git Services

[GitHub] [beam] davidyan74 commented on a change in pull request #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
davidyan74 commented on a change in pull request #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#discussion_r395933604
 
 

 ##########
 File path: sdks/python/apache_beam/pipeline.py
 ##########
 @@ -307,58 +307,61 @@ def _replace_if_needed(self, original_transform_node):
           elif len(inputs) == 0:
             input_node = pvalue.PBegin(self.pipeline)
 
-          # We have to add the new AppliedTransform to the stack before expand()
-          # and pop it out later to make sure that parts get added correctly.
-          self.pipeline.transforms_stack.append(replacement_transform_node)
-
-          # Keeping the same label for the replaced node but recursively
-          # removing labels of child transforms of original transform since they
-          # will be replaced during the expand below. This is needed in case
-          # the replacement contains children that have labels that conflicts
-          # with labels of the children of the original.
-          self.pipeline._remove_labels_recursively(original_transform_node)
-
-          new_output = replacement_transform.expand(input_node)
-          assert isinstance(
-              new_output, (dict, pvalue.PValue, pvalue.DoOutputsTuple))
-
-          if isinstance(new_output, pvalue.PValue):
-            new_output.element_type = None
-            self.pipeline._infer_result_type(
-                replacement_transform, inputs, new_output)
-
-          if isinstance(new_output, dict):
-            for new_tag, new_pcoll in new_output.items():
-              replacement_transform_node.add_output(new_pcoll, new_tag)
-          elif isinstance(new_output, pvalue.DoOutputsTuple):
-            replacement_transform_node.add_output(
-                new_output, new_output._main_tag)
-          else:
-            replacement_transform_node.add_output(new_output, new_output.tag)
-
-          # Recording updated outputs. This cannot be done in the same visitor
-          # since if we dynamically update output type here, we'll run into
-          # errors when visiting child nodes.
-          #
-          # NOTE: When replacing multiple outputs, the replacement PCollection
-          # tags must have a matching tag in the original transform.
-          if isinstance(new_output, pvalue.PValue):
-            if not new_output.producer:
-              new_output.producer = replacement_transform_node
-            output_map[original_transform_node.outputs[new_output.tag]] = \
-                new_output
-          elif isinstance(new_output, (pvalue.DoOutputsTuple, tuple)):
-            for pcoll in new_output:
-              if not pcoll.producer:
-                pcoll.producer = replacement_transform_node
-              output_map[original_transform_node.outputs[pcoll.tag]] = pcoll
-          elif isinstance(new_output, dict):
-            for tag, pcoll in new_output.items():
-              if not pcoll.producer:
-                pcoll.producer = replacement_transform_node
-              output_map[original_transform_node.outputs[tag]] = pcoll
-
-          self.pipeline.transforms_stack.pop()
+          try:
+            # We have to add the new AppliedTransform to the stack before
+            # expand() and pop it out later to make sure that parts get added
+            # correctly.
+            self.pipeline.transforms_stack.append(replacement_transform_node)
 
 Review comment:
   This is a python newbie question. Does list.append() ever throw an exception? If so, should we move this out of the try block so that we don't pop() if list.append() fails?

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


With regards,
Apache Git Services

[GitHub] [beam] aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#issuecomment-602851060
 
 
   retest this please

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


With regards,
Apache Git Services

[GitHub] [beam] aaltay commented on issue #11174: [BEAM-7923] Pop failed transform in CombineGlobally

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #11174: [BEAM-7923] Pop failed transform in CombineGlobally
URL: https://github.com/apache/beam/pull/11174#issuecomment-601473505
 
 
   `expand` for any other transform could also fail. Do we expect implementers of each of those to do the same thing? Can we apply this somewhere at a stack, where if expand fails, we clean up this stack?
   
   @robertwb 

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


With regards,
Apache Git Services

[GitHub] [beam] KevinGG commented on issue #11174: [BEAM-7923] Pop failed transform in CombineGlobally

Posted by GitBox <gi...@apache.org>.
KevinGG commented on issue #11174: [BEAM-7923] Pop failed transform in CombineGlobally
URL: https://github.com/apache/beam/pull/11174#issuecomment-601467279
 
 
   yapf formatted.
   Lint passed locally.
   
   R: @aaltay 
   R: @davidyan74 
   R: @rohdesamuel 
   
   PTAL, thx!
   

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


With regards,
Apache Git Services

[GitHub] [beam] aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#issuecomment-603353061
 
 
   retest this please

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


With regards,
Apache Git Services

[GitHub] [beam] KevinGG commented on a change in pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally

Posted by GitBox <gi...@apache.org>.
KevinGG commented on a change in pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally
URL: https://github.com/apache/beam/pull/11174#discussion_r395890026
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1811,6 +1811,8 @@ def add_input_types(transform):
       return view
     else:
       if pcoll.windowing.windowfn != GlobalWindows():
+        # Remove the broken transform when running into value error.
+        pcoll.pipeline.transforms_stack.pop()
 
 Review comment:
   Yes, agree with it! I'll make the change.

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


With regards,
Apache Git Services

[GitHub] [beam] KevinGG commented on a change in pull request #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
KevinGG commented on a change in pull request #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#discussion_r396742548
 
 

 ##########
 File path: sdks/python/apache_beam/pipeline.py
 ##########
 @@ -307,58 +307,61 @@ def _replace_if_needed(self, original_transform_node):
           elif len(inputs) == 0:
             input_node = pvalue.PBegin(self.pipeline)
 
-          # We have to add the new AppliedTransform to the stack before expand()
-          # and pop it out later to make sure that parts get added correctly.
-          self.pipeline.transforms_stack.append(replacement_transform_node)
-
-          # Keeping the same label for the replaced node but recursively
-          # removing labels of child transforms of original transform since they
-          # will be replaced during the expand below. This is needed in case
-          # the replacement contains children that have labels that conflicts
-          # with labels of the children of the original.
-          self.pipeline._remove_labels_recursively(original_transform_node)
-
-          new_output = replacement_transform.expand(input_node)
-          assert isinstance(
-              new_output, (dict, pvalue.PValue, pvalue.DoOutputsTuple))
-
-          if isinstance(new_output, pvalue.PValue):
-            new_output.element_type = None
-            self.pipeline._infer_result_type(
-                replacement_transform, inputs, new_output)
-
-          if isinstance(new_output, dict):
-            for new_tag, new_pcoll in new_output.items():
-              replacement_transform_node.add_output(new_pcoll, new_tag)
-          elif isinstance(new_output, pvalue.DoOutputsTuple):
-            replacement_transform_node.add_output(
-                new_output, new_output._main_tag)
-          else:
-            replacement_transform_node.add_output(new_output, new_output.tag)
-
-          # Recording updated outputs. This cannot be done in the same visitor
-          # since if we dynamically update output type here, we'll run into
-          # errors when visiting child nodes.
-          #
-          # NOTE: When replacing multiple outputs, the replacement PCollection
-          # tags must have a matching tag in the original transform.
-          if isinstance(new_output, pvalue.PValue):
-            if not new_output.producer:
-              new_output.producer = replacement_transform_node
-            output_map[original_transform_node.outputs[new_output.tag]] = \
-                new_output
-          elif isinstance(new_output, (pvalue.DoOutputsTuple, tuple)):
-            for pcoll in new_output:
-              if not pcoll.producer:
-                pcoll.producer = replacement_transform_node
-              output_map[original_transform_node.outputs[pcoll.tag]] = pcoll
-          elif isinstance(new_output, dict):
-            for tag, pcoll in new_output.items():
-              if not pcoll.producer:
-                pcoll.producer = replacement_transform_node
-              output_map[original_transform_node.outputs[tag]] = pcoll
-
-          self.pipeline.transforms_stack.pop()
+          try:
+            # We have to add the new AppliedTransform to the stack before
+            # expand() and pop it out later to make sure that parts get added
+            # correctly.
+            self.pipeline.transforms_stack.append(replacement_transform_node)
 
 Review comment:
   It wouldn't raise an error if it's still a list on the line, and if it's not a list (becomes a None or some other object) at the moment it causes an error, the finally block would error out too. So it's not necessary to exclude it from the try block.
   
   Putting it inside the try makes it a little bit more self-explained.

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


With regards,
Apache Git Services

[GitHub] [beam] KevinGG commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
KevinGG commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#issuecomment-602852406
 
 
   Fixed a failed test.
   
   TL;DR, the test was wrong and passed in the past because it got lucky.
   
   The test used to pass and started failing with the try-append-finally-pop change, because:
   1. The test tried to append two no name / label transforms into a same pipeline, both of them will generate the same label and raise errors while the tests asserted for errors, thus allowing pipeline construction to continue even if it ran into fatal errors;
   2. In the past, the pipeline was in a broken state where the current_transform was not popped when the 1st error was raised. The full label generated was `WriteToBigQuery`;
   3. In the past, the second transform appending wrongly added parts into the broken current_transform `WriteToBigQuery`, causing the transform node to be appended after a wrong failed parent node and luckily got a unique transform full label `WriteToBigQuery/WriteToBigQuery`, thus the test didn't fail due to duplicated labels. But the pipeline was constructed as `WriteToBigQuery`->`WriteToBigQuery` which was completely messed up;
   4. Still the test didn't fail because it was testing for errors even though it built a broken pipeline with broken usages.
   
   Once the try-append-finally-pop change is applied, the test functions as:
   1. First transform still fails to be appended and has side-effects including an applied transform label, but it would not leave the pipeline in a broken state where current transform is still the right node;
   2. Second transform still fails but now appended to the correct node;
   3. As long as the 2 transforms have different labels (or executed in different cells if in an interactive environment), the test still passes, pipeline is not broken and can be used for future development, side-effects are ruled out when data-centric APIs such as `show` and `collect` are invoked.
   
   The reason we keep the applied label is that we never know what side effects are when the error is raised.

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


With regards,
Apache Git Services

[GitHub] [beam] aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#issuecomment-602854440
 
 
   retest this please

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


With regards,
Apache Git Services

[GitHub] [beam] robertwb commented on a change in pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally

Posted by GitBox <gi...@apache.org>.
robertwb commented on a change in pull request #11174: [BEAM-7923] Pop failed transform in CombineGlobally
URL: https://github.com/apache/beam/pull/11174#discussion_r395882293
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/core.py
 ##########
 @@ -1811,6 +1811,8 @@ def add_input_types(transform):
       return view
     else:
       if pcoll.windowing.windowfn != GlobalWindows():
+        # Remove the broken transform when running into value error.
+        pcoll.pipeline.transforms_stack.pop()
 
 Review comment:
   This is not the right place to pop this (internal) stack. Instead, we should popping from the stack in a finally clause of a try block that pushes to the stack. (Alternatively, we could manage the stack with a Python context, but that might be overkill.)

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


With regards,
Apache Git Services

[GitHub] [beam] aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174#issuecomment-601949155
 
 
   retest this please

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


With regards,
Apache Git Services

[GitHub] [beam] aaltay merged pull request #11174: [BEAM-7923] Pop failed transform when error is raised

Posted by GitBox <gi...@apache.org>.
aaltay merged pull request #11174: [BEAM-7923] Pop failed transform when error is raised
URL: https://github.com/apache/beam/pull/11174
 
 
   

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


With regards,
Apache Git Services