You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/01/06 21:49:59 UTC

[GitHub] [airflow] malthe opened a new pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

malthe opened a new pull request #20731:
URL: https://github.com/apache/airflow/pull/20731


   This pull request adds a new meaning to exceptions deriving from `AirflowException` – when raised during task execution and bubbling up to the standard task runner.
   
   In this case, no stack trace is printed – instead, the exception message is printed and we provide the filename and line number of the stack frame where the exception originated.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r802533821



##########
File path: airflow/models/taskinstance.py
##########
@@ -1438,7 +1443,13 @@ def _execute_task_with_callbacks(self, context):
         """Prepare Task for Execution"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
 
+        parent_pid = os.getpid()
+
         def signal_handler(signum, frame):
+            pid = os.getpid()
+            if pid != parent_pid:
+                os._exit(1)

Review comment:
       Added in 38d7acff3ccbbce2e2a5789f13221b336f705ac3.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r807020759



##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       Is it possible that `self.task` is not in the dictionary?
   ```suggestion
                   execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task, None)
                   if execution_frame: ...
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r802448391



##########
File path: airflow/models/taskinstance.py
##########
@@ -1438,7 +1443,13 @@ def _execute_task_with_callbacks(self, context):
         """Prepare Task for Execution"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
 
+        parent_pid = os.getpid()
+
         def signal_handler(signum, frame):
+            pid = os.getpid()
+            if pid != parent_pid:
+                os._exit(1)

Review comment:
       What's this for?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r782130420



##########
File path: airflow/exceptions.py
##########
@@ -30,8 +30,8 @@
 
 class AirflowException(Exception):
     """
-    Base class for all Airflow's errors.
-    Each custom exception should be derived from this class
+    Base class for all Airflow's errors
+    Each custom exception should be derived from this class.
     """

Review comment:
       Fixed in #20811.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r782112708



##########
File path: airflow/exceptions.py
##########
@@ -30,8 +30,8 @@
 
 class AirflowException(Exception):
     """
-    Base class for all Airflow's errors.
-    Each custom exception should be derived from this class
+    Base class for all Airflow's errors
+    Each custom exception should be derived from this class.
     """

Review comment:
       Yeah I was thinking about what to do here because all of those classes are sort of wrong in how the docstring is formatted.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1040473789


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk merged pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #20731:
URL: https://github.com/apache/airflow/pull/20731


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007286352


   Another option:
   
   ```python
       try:
           r()
       except:
           (t,v,tb) = sys.exc_info()
           tb.tb_next = None
           logging.exception("boom", exc_info=(t,v,tb))
   ```
   
   Produces
   
   ```
   ERROR:root:boom
   Traceback (most recent call last):
     File "<ipython-input-11-449d6688bb3b>", line 3, in run2
       r()
   RuntimeError: a
   ```


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r802468952



##########
File path: airflow/models/taskinstance.py
##########
@@ -1438,7 +1443,13 @@ def _execute_task_with_callbacks(self, context):
         """Prepare Task for Execution"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
 
+        parent_pid = os.getpid()
+
         def signal_handler(signum, frame):
+            pid = os.getpid()
+            if pid != parent_pid:
+                os._exit(1)

Review comment:
       So that's because ... well it's a bit of an odd-ball, but it's motivated by a DAG in the test suite which forks inside DAG code.
   
   What happens then is that it returns back out to the calling framework, but we don't want to handle such a forked process coming back in, we just want to handle our own forked process.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r806823084



##########
File path: airflow/models/taskinstance.py
##########
@@ -1438,7 +1443,18 @@ def _execute_task_with_callbacks(self, context):
         """Prepare Task for Execution"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
 
+        parent_pid = os.getpid()
+
         def signal_handler(signum, frame):
+            pid = os.getpid()

Review comment:
       Nice one.

##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       Doesn't get() return None if task is missing already :P ? 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r782111930



##########
File path: airflow/exceptions.py
##########
@@ -30,8 +30,8 @@
 
 class AirflowException(Exception):
     """
-    Base class for all Airflow's errors.
-    Each custom exception should be derived from this class
+    Base class for all Airflow's errors
+    Each custom exception should be derived from this class.
     """

Review comment:
       PEP 257 recommends something like this:
   
   
   ```python
   """Base class for all Airflow's errors.
   
   Each custom exception should be derived from this class.
   """
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb edited a comment on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007286352


   Another option:
   
   ```python
       try:
           r()
       except:
           (t,v,tb) = sys.exc_info()
           while tb.tb_next:
               tb = tb.tb_next
           logging.exception("boom", exc_info=(t,v,tb))
   ```
   
   Produces
   
   ```
   ERROR:root:boom
   Traceback (most recent call last):
     File "<ipython-input-1-b39a8ed742d0>", line 1, in r
       def r():raise RuntimeError("a")
   RuntimeError: a
   ```


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1040264003


   Would you please rebase @malthe ? this one looks good to me but I'd feel more comfortable if it gets rebased.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r807045467



##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       But yeah the `None` part is not needed – `get` should be fine if the task is not there with `None` being returned in that case.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1043282480


   Seems like there are still some non-trivial test failures.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1043203513


   @potiuk fixed in 9ec803684ba61292b54e9aa7a9772c0a45610680 – I replaced `BaseOperator` with `Operator` which was already pulled in. I suppose it will work ...


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe removed a comment on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe removed a comment on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1043282480


   Seems like there are still some non-trivial test failures.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r807044544



##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       @ashb yes sometimes it is not in the dictionary, there are various ways a task can fail and only some of them will produce a traceback during execution.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r807028542



##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       Doesn't get() return None if task is missing already :P ? 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1040473789


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb edited a comment on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007286352


   Another option:
   
   ```python
       try:
           r()
       except:
           (t,v,tb) = sys.exc_info()
           while tb.tb_next:
               tb = tb.tb_next
           logging.exception("boom", exc_info=(t,v,tb))
   ```
   
   Produces
   
   ```
   ERROR:root:boom
   Traceback (most recent call last):
     File "<ipython-input-1-b39a8ed742d0>", line 1, in r
       def r():raise RuntimeError("a")
   RuntimeError: a
   ```


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1043066335


   `airflow/models/taskinstance.py:132: error: Name "BaseOperator" is not defined `


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1040264003


   Would you please rebase @malthe ? this one looks good to me but I'd feel more comfortable if it gets rebased.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r802468952



##########
File path: airflow/models/taskinstance.py
##########
@@ -1438,7 +1443,13 @@ def _execute_task_with_callbacks(self, context):
         """Prepare Task for Execution"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
 
+        parent_pid = os.getpid()
+
         def signal_handler(signum, frame):
+            pid = os.getpid()
+            if pid != parent_pid:
+                os._exit(1)

Review comment:
       So that's because ... well it's a bit of an odd-ball, but it's motivated by a DAG in the test suite which forks inside DAG code.
   
   What happens then is that it returns back out to the calling framework, but we don't want to handle such a forked process coming back in, we just want to handle our own forked process.
   
   I think there's an assumption that you can do things in DAG code which doesn't affect the framework since it's running a separate process and this `os._exit(1)` just makes sure that assumption holds.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r782112708



##########
File path: airflow/exceptions.py
##########
@@ -30,8 +30,8 @@
 
 class AirflowException(Exception):
     """
-    Base class for all Airflow's errors.
-    Each custom exception should be derived from this class
+    Base class for all Airflow's errors
+    Each custom exception should be derived from this class.
     """

Review comment:
       Yeah I was thinking about what to do here because all of those classes are sort of wrong in how the docstring is formatted, but let me get that period character back at least.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r782117198



##########
File path: airflow/exceptions.py
##########
@@ -30,8 +30,8 @@
 
 class AirflowException(Exception):
     """
-    Base class for all Airflow's errors.
-    Each custom exception should be derived from this class
+    Base class for all Airflow's errors
+    Each custom exception should be derived from this class.
     """

Review comment:
       There are a ton of weirdly formatted docstrings in Airflow unfortunately. But we can make the count one less since we are editing this anyway.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007318923


   @ashb I think there are two separate discussions here, one being whether the implementation is complicated and the other which output is the better one.
   
   While the output using the `while tb.tb_next` technique contains the same information, to me it seems a little stuck between two chairs. I think we should either have a full traceback or a simple error message output – the latter is what I have implemented here, the idea being that the user needn't bother with a low-level detail such as a Python traceback in the case where the error is an `AirflowException`.
   
   The error message should be enough to understand what the problem is – and `filename` and `lineno` is really just a bonus then.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007647466


   > @ashb actually, your `tb_next` snippet gave me an idea, which is that non-`AirflowException` errors should probably have their traceback shortened to right after `_execute_task` such that the first frame that is shown is within "user code".
   
   That sounds like an excellent idea!


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007286352






-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1008874442


   > Given the (over)use of `raise AirflowExcpetion` in various places inside Airflow ~now~ not showing the stacktrace for those is the wrong decision.
   
   Agree - we should only implement it together with reviewing and updating all those.
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1008874442


   > Given the (over)use of `raise AirflowExcpetion` in various places inside Airflow ~now~ not showing the stacktrace for those is the wrong decision.
   
   Agree - we shoudl only implement it together with reviewing and updating all those.
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1009939882


   @uranusjr I ran into some trouble with the `test_on_kill` method where I found `job_id` to be unset on the task instance, presumably due to the way the API is set up and used differently in various situations. This makes the combined changeset a little bigger than it really ought to be.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r802525012



##########
File path: airflow/models/taskinstance.py
##########
@@ -1438,7 +1443,13 @@ def _execute_task_with_callbacks(self, context):
         """Prepare Task for Execution"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
 
+        parent_pid = os.getpid()
+
         def signal_handler(signum, frame):
+            pid = os.getpid()
+            if pid != parent_pid:
+                os._exit(1)

Review comment:
       Comment explaining why needed then 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r807044544



##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       @ashb yes sometimes it is not in the dictionary, there are various ways a task can fail and only some of them will produce a traceback during execution.

##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       But yeah the `None` part is not needed – `get` should be fine if the task is not there with `None` being returned in that case.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r806823084



##########
File path: airflow/models/taskinstance.py
##########
@@ -1438,7 +1443,18 @@ def _execute_task_with_callbacks(self, context):
         """Prepare Task for Execution"""
         from airflow.models.renderedtifields import RenderedTaskInstanceFields
 
+        parent_pid = os.getpid()
+
         def signal_handler(signum, frame):
+            pid = os.getpid()

Review comment:
       Nice one.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r807020759



##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       Is it possible that `self.task` is not in the dictionary?
   ```suggestion
                   execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task, None)
                   if execution_frame: ...
   ```

##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       🤦🏻 
   
   For some reason I thought it would still error




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #20731: Truncate stack trace to DAG user code for exceptions raised during execution

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#discussion_r807039759



##########
File path: airflow/models/taskinstance.py
##########
@@ -1737,7 +1758,15 @@ def handle_failure(
 
         if error:
             if isinstance(error, BaseException):
-                self.log.error("Task failed with exception", exc_info=error)
+                execution_frame = _EXECUTION_FRAME_MAPPING.get(self.task)

Review comment:
       🤦🏻 
   
   For some reason I thought it would still error




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe closed pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe closed pull request #20731:
URL: https://github.com/apache/airflow/pull/20731


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007577920


   @ashb actually, your `tb_next` snippet gave me an idea, which is that non-`AirflowException` errors should probably have their traceback shortened to right after `_execute_task` such that the first frame that is shown is within "user code".


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] malthe commented on pull request #20731: Avoid printing a stacktrace for 'AirflowException'-derived errors

Posted by GitBox <gi...@apache.org>.
malthe commented on pull request #20731:
URL: https://github.com/apache/airflow/pull/20731#issuecomment-1007318923






-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org