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

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

                Author: ASF GitHub Bot
            Created on: 12/Apr/18 15:54
            Start Date: 12/Apr/18 15:54
    Worklog Time Spent: 10m 
      Work Description: aaltay closed pull request #4959: [BEAM-3956] Preserve 
stacktraces for Python exceptions
URL: https://github.com/apache/beam/pull/4959
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/python/apache_beam/runners/common.py 
b/sdks/python/apache_beam/runners/common.py
index 0bf5bac88b5..e3c768bfcfe 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -534,7 +534,7 @@ def _reraise_augmented(self, exn):
           traceback.format_exception_only(type(exn), exn)[-1].strip()
           + step_annotation)
       new_exn._tagged_with_step = True
-    six.raise_from(new_exn, original_traceback)
+    six.reraise(type(new_exn), new_exn, original_traceback)
 
 
 class OutputProcessor(object):
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py 
b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
index ca0734d1422..faad561f54b 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
@@ -21,6 +21,7 @@
 import os
 import tempfile
 import time
+import traceback
 import unittest
 
 import apache_beam as beam
@@ -265,7 +266,7 @@ def test_windowing(self):
              | beam.Map(lambda k_vs1: (k_vs1[0], sorted(k_vs1[1]))))
       assert_that(res, equal_to([('k', [1, 2]), ('k', [100, 101, 102])]))
 
-  def test_errors(self):
+  def test_error_message_includes_stage(self):
     with self.assertRaises(BaseException) as e_cm:
       with self.create_pipeline() as p:
         def raise_error(x):
@@ -280,6 +281,29 @@ def raise_error(x):
     self.assertIn('StageC', e_cm.exception.args[0])
     self.assertNotIn('StageB', e_cm.exception.args[0])
 
+  def test_error_traceback_includes_user_code(self):
+
+    def first(x):
+      return second(x)
+
+    def second(x):
+      return third(x)
+
+    def third(x):
+      raise ValueError('x')
+
+    try:
+      with self.create_pipeline() as p:
+        p | beam.Create([0]) | beam.Map(first)  # pylint: 
disable=expression-not-assigned
+    except Exception:  # pylint: disable=broad-except
+      message = traceback.format_exc()
+    else:
+      raise AssertionError('expected exception not raised')
+
+    self.assertIn('first', message)
+    self.assertIn('second', message)
+    self.assertIn('third', message)
+
   def test_no_subtransform_composite(self):
 
     class First(beam.PTransform):
diff --git 
a/sdks/python/apache_beam/runners/portability/universal_local_runner_test.py 
b/sdks/python/apache_beam/runners/portability/universal_local_runner_test.py
index 66181c014f7..56dc514b416 100644
--- a/sdks/python/apache_beam/runners/portability/universal_local_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/universal_local_runner_test.py
@@ -82,7 +82,7 @@ def test_assert_that(self):
       with self.create_pipeline() as p:
         assert_that(p | beam.Create(['a', 'b']), equal_to(['a']))
 
-  def test_errors(self):
+  def test_error_message_includes_stage(self):
     # TODO: figure out a way for runner to parse and raise the
     # underlying exception.
     with self.assertRaises(Exception):
@@ -97,6 +97,11 @@ def raise_error(x):
          | 'StageC' >> beam.Map(raise_error)
          | 'StageD' >> beam.Map(lambda x: x))
 
+  def test_error_traceback_includes_user_code(self):
+    # TODO: figure out a way for runner to parse and raise the
+    # underlying exception.
+    raise unittest.SkipTest('TODO')
+
   # Inherits all tests from fn_api_runner_test.FnApiRunnerTest
 
 
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py 
b/sdks/python/apache_beam/runners/worker/sdk_worker.py
index fe908be3d8a..972e94c0025 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py
@@ -127,15 +127,14 @@ def get_responses():
   def _execute(self, task, request):
     try:
       response = task()
-    except Exception as e:  # pylint: disable=broad-except
-      traceback.print_exc(file=sys.stderr)
+    except Exception:  # pylint: disable=broad-except
+      traceback_string = traceback.format_exc()
+      print(traceback_string, file=sys.stderr)
       logging.error(
           'Error processing instruction %s. Original traceback is\n%s\n',
-          request.instruction_id,
-          traceback.format_exc(e),
-          exc_info=True)
+          request.instruction_id, traceback_string)
       response = beam_fn_api_pb2.InstructionResponse(
-          instruction_id=request.instruction_id, error=str(e))
+          instruction_id=request.instruction_id, error=traceback_string)
     self._responses.put(response)
 
   def _request_register(self, request):


 

----------------------------------------------------------------
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: 90453)
    Time Spent: 7.5h  (was: 7h 20m)

> Stacktraces from exceptions in user code should be preserved in the Python SDK
> ------------------------------------------------------------------------------
>
>                 Key: BEAM-3956
>                 URL: https://issues.apache.org/jira/browse/BEAM-3956
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>            Reporter: Stephan Hoyer
>            Priority: Major
>          Time Spent: 7.5h
>  Remaining Estimate: 0h
>
> Currently, Beam's Python SDK loses stacktraces for exceptions. It does 
> helpfully add a tag like "[while running StageA]" to exception error 
> messages, but that doesn't include the stacktrace of Python functions being 
> called.
> Including the full stacktraces would make a big difference for the ease of 
> debugging Beam pipelines when things go wrong.



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

Reply via email to