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 2020/10/21 12:38:54 UTC

[GitHub] [airflow] mjpieters opened a new issue #11717: All task logging goes to the log for try_number 1

mjpieters opened a new issue #11717:
URL: https://github.com/apache/airflow/issues/11717


   **Apache Airflow version**: 2.0.0a1
   **What happened**:
   
   When a task fails on the first try, the log output for additional tries go to the log for the first attempt.
   
   **What you expected to happen**:
   
   The logs should go to the correct log file. For the default configuration, the log filename template is `log_filename_template = {{ ti.dag_id }}/{{ ti.task_id }}/{{ ts }}/{{ try_number }}.log`, so additional numbered `.log` files should be created.
   
   **How to reproduce it**:
   
   Create a test dag:
   
   ```
   from datetime import timedelta
   from airflow import DAG
   from airflow.operators.python import PythonOperator
   from airflow.utils.dates import days_ago
   
   
   with DAG(
       dag_id="trynumber_demo",
       default_args={"start_date": days_ago(2), "retries": 1, "retry_delay": timedelta(0)},
       schedule_interval=None,
   ) as dag:
   
       def demo_task(ti=None):
           print("Running demo_task, try_number =", ti.try_number)
           if ti.try_number <= 1:
               raise ValueError("Shan't")
   
       task = PythonOperator(task_id="demo_task", python_callable=demo_task)
   ```
   
   and trigger this dag:
   
   ```
   $ airflow dags trigger trynumber_demo
   ```
   
   then observe that `triggernumber_demo/demo_task/<execution_date>/` only contains 1.log, which contains the full output for 2 runs:
   
   ```
   [...]
   --------------------------------------------------------------------------------
   [2020-10-21 13:29:07,958] {taskinstance.py:1020} INFO - Starting attempt 1 of 2
   [2020-10-21 13:29:07,959] {taskinstance.py:1021} INFO -
   --------------------------------------------------------------------------------
   [...]
   [2020-10-21 13:29:08,163] {logging_mixin.py:110} INFO - Running demo_task, try_number = 1
   [2020-10-21 13:29:08,164] {taskinstance.py:1348} ERROR - Shan't
   Traceback (most recent call last):
   [...]
   ValueError: Shan't
   [2020-10-21 13:29:08,168] {taskinstance.py:1392} INFO - Marking task as UP_FOR_RETRY. dag_id=trynumber_demo, task_id=demo_task, execution_date=20201021T122907, start_date=20201021T122907, end_date=20201021T122908
   [...]
   [2020-10-21 13:29:09,121] {taskinstance.py:1019} INFO -
   --------------------------------------------------------------------------------
   [2020-10-21 13:29:09,121] {taskinstance.py:1020} INFO - Starting attempt 2 of 2
   [2020-10-21 13:29:09,121] {taskinstance.py:1021} INFO -
   --------------------------------------------------------------------------------
   [...]
   [2020-10-21 13:29:09,333] {logging_mixin.py:110} INFO - Running demo_task, try_number = 2
   [2020-10-21 13:29:09,334] {python.py:141} INFO - Done. Returned value was: None
   [2020-10-21 13:29:09,355] {taskinstance.py:1143} INFO - Marking task as SUCCESS.dag_id=trynumber_demo, task_id=demo_task, execution_date=20201021T122907, start_date=20201021T122909, end_date=20201021T122909
   [2020-10-21 13:29:09,404] {local_task_job.py:117} INFO - Task exited with return code 0
   ```
   
   The `TaskInstance()` created for the run needs to first be refreshed from the database, before setting the logging context.


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



[GitHub] [airflow] ashb closed issue #11717: All task logging goes to the log for try_number 1

Posted by GitBox <gi...@apache.org>.
ashb closed issue #11717:
URL: https://github.com/apache/airflow/issues/11717


   


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



[GitHub] [airflow] turbaszek commented on issue #11717: All task logging goes to the log for try_number 1

Posted by GitBox <gi...@apache.org>.
turbaszek commented on issue #11717:
URL: https://github.com/apache/airflow/issues/11717#issuecomment-713596731


   @mjpieters would you like to open a PR?


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



[GitHub] [airflow] mjpieters commented on issue #11717: All task logging goes to the log for try_number 1

Posted by GitBox <gi...@apache.org>.
mjpieters commented on issue #11717:
URL: https://github.com/apache/airflow/issues/11717#issuecomment-713555948


   I can confirm that changing
   
   https://github.com/apache/airflow/blob/172820db4d2009dd26fa8aef4a864fb8a3d7e78d/airflow/cli/commands/task_command.py#L172-L174
   
   to
   
   ```
       task = dag.get_task(task_id=args.task_id)
       ti = TaskInstance(task, args.execution_date)
       ti.refresh_from_db()
       ti.init_run_context(raw=args.raw)
   ```
   
   fixes this issue, but I don't know if the `refresh_..` call should be integrated into `init_run_context()` perhaps.


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



[GitHub] [airflow] mjpieters commented on issue #11717: All task logging goes to the log for try_number 1

Posted by GitBox <gi...@apache.org>.
mjpieters commented on issue #11717:
URL: https://github.com/apache/airflow/issues/11717#issuecomment-713583976


   I found #7370, which removed that call, specifically see [these review comments](https://github.com/apache/airflow/pull/7370#discussion_r375370760):
   
   > Does init_run_context look at any attributes of the TI? If so they won't be populated until this call.
   
   > No, it only setups logging context :)
   
   Unfortunately, the logging context looks at the attributes of the TI!


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



[GitHub] [airflow] ashb commented on issue #11717: All task logging goes to the log for try_number 1

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #11717:
URL: https://github.com/apache/airflow/issues/11717#issuecomment-713651256


   Could be same as https://github.com/apache/airflow/issues/11561 (though more detail/cause here)


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