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/02/07 11:44:42 UTC

[GitHub] [airflow] george-zubrienko opened a new issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

george-zubrienko opened a new issue #21387:
URL: https://github.com/apache/airflow/issues/21387


   ### Apache Airflow version
   
   2.2.3 (latest released)
   
   ### What happened
   
   This problem popped up when we switched from using persistent volume provisioned from helm chart via PVC (`log.persistence.enabled`) to WASB logging.
   
   While a task is running, the only logs we see are these:
   
   ```
   *** Trying to get logs (last 100 lines) from worker pod anaplancopyanaplantestsamplestrino.cfbd54145b32466bb6bedef603ed89a1 ***
   
   [2022-02-07, 11:24:15 UTC] {dagbag.py:500} INFO - Filling up the DagBag from /opt/airflow/dags/ecco_airflow/dags/bronze/anaplan.py
   ```
   Then when the task completes and its log is moved by WASB provider, UI shows data from remote log file as expected.
   
   Problem here is that log is read from **pod** - as you can see in the sample above, instead of a **local log file**.
   
   Since WASB provider can find the local log file in order to move it, it exists. I also created a simple dag to prove it:
   
   ```python
   import os
   import time
   
   from airflow import DAG
   from airflow.operators.python import PythonOperator
   from airflow.utils.dates import days_ago
   from datetime import timedelta
   from airflow.configuration import conf
   from airflow.utils.log.file_task_handler import FileTaskHandler
   
   from ecco_airflow.utils.k8s import executor_config
   
   default_args = {
       'owner': 'test',
       'depends_on_past': False,
       'email': ['bla@bla.com'],
       'email_on_failure': False,
       'email_on_retry': False,
       'retries': 1,
       'retry_delay': timedelta(days=1),
   }
   
   with DAG(
           dag_id='test_dag',
           default_args=default_args,
           description='Test logs fetch',
           schedule_interval="0 0 * * *",
           start_date=days_ago(1),
           tags=[]
   ) as dag:
   
       def test_read(**context):
           time.sleep(30)
           BASE_LOG_FOLDER: str = conf.get('logging', 'BASE_LOG_FOLDER')
           FILENAME_TEMPLATE: str = conf.get('logging', 'LOG_FILENAME_TEMPLATE')
           h = FileTaskHandler(BASE_LOG_FOLDER, FILENAME_TEMPLATE)
   
   
           log_relative_path = h._render_filename(context['ti'], context['ti'].try_number)
   
           print(log_relative_path)
   
           location = os.path.join(h.local_base, log_relative_path)
   
           print(location)
           log = ""
           if os.path.exists(location):
               try:
                   with open(location, encoding="utf-8", errors="surrogateescape") as file:
                       log += f"*** Reading local file: {location}\n"
                       log += "".join(file.readlines())
               except Exception as e:
                   log = f"*** Failed to load local log file: {location}\n"
                   log += f"*** {str(e)}\n"
               print(log)
           else:
               print(-1)
   
           time.sleep(30)
   
       t = PythonOperator(
           task_id='test_logs',
           python_callable=test_read,
           op_kwargs={},
           executor_config=executor_config(
               nodepool_names=['general'],
               secret_env_vars={},
               cpu_memory_limit={
                   'cpu': '100m',
                   'memory': '500Mi'
               }),
       )
   
   ```
   
   What happens here, while we sleep, **local log file** is present on the machine, but Airflow still tries to read log from a pod.
   Assuming I'm looking in a [correct place](https://github.com/apache/airflow/blob/2.2.3/airflow/utils/log/file_task_handler.py), what happens is, local log cannot be found, according to this code https://github.com/apache/airflow/blob/2.2.3/airflow/utils/log/file_task_handler.py#L96-L143
   
   However, final log for the task in the dag above will print 
   ```
   [2022-02-07, 11:05:07 UTC] {taskinstance.py:1259} INFO - Executing <Task(PythonOperator): test_logs> on 2022-02-06 00:00:00+00:00
   [2022-02-07, 11:05:07 UTC] {standard_task_runner.py:52} INFO - Started process 11 to run task
   [2022-02-07, 11:05:07 UTC] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test_dag', 'test_logs', 'scheduled__2022-02-06T00:00:00+00:00', '--job-id', '15306', '--raw', '--subdir', 'DAGS_FOLDER/ecco_airflow/dags/test_dag.py', '--cfg-path', '/tmp/tmp_iv6qjpv', '--error-file', '/tmp/tmp_cc1dftp']
   [2022-02-07, 11:05:07 UTC] {standard_task_runner.py:77} INFO - Job 15306: Subtask test_logs
   [2022-02-07, 11:05:07 UTC] {logging_mixin.py:109} INFO - Running <TaskInstance: test_dag.test_logs scheduled__2022-02-06T00:00:00+00:00 [running]> on host testdagtestlogs.2466ee8d3bf244d9933d502894766d3c
   [2022-02-07, 11:05:08 UTC] {taskinstance.py:1424} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_EMAIL=esdsupport@ecco.com
   AIRFLOW_CTX_DAG_OWNER=test
   AIRFLOW_CTX_DAG_ID=test_dag
   AIRFLOW_CTX_TASK_ID=test_logs
   AIRFLOW_CTX_EXECUTION_DATE=2022-02-06T00:00:00+00:00
   AIRFLOW_CTX_DAG_RUN_ID=scheduled__2022-02-06T00:00:00+00:00
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'execution_date' from the template is deprecated and will be removed in a future version. Please use 'data_interval_start' or 'logical_date' instead.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'next_ds' from the template is deprecated and will be removed in a future version. Please use '{{ data_interval_end | ds }}' instead.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'next_ds_nodash' from the template is deprecated and will be removed in a future version. Please use '{{ data_interval_end | ds_nodash }}' instead.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'next_execution_date' from the template is deprecated and will be removed in a future version. Please use 'data_interval_end' instead.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'prev_ds' from the template is deprecated and will be removed in a future version.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'prev_ds_nodash' from the template is deprecated and will be removed in a future version.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'prev_execution_date' from the template is deprecated and will be removed in a future version.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'prev_execution_date_success' from the template is deprecated and will be removed in a future version. Please use 'prev_data_interval_start_success' instead.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'tomorrow_ds' from the template is deprecated and will be removed in a future version.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'tomorrow_ds_nodash' from the template is deprecated and will be removed in a future version.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'yesterday_ds' from the template is deprecated and will be removed in a future version.
   [2022-02-07, 11:05:08 UTC] {logging_mixin.py:109} WARNING - /home/airflow/.local/lib/python3.9/site-packages/airflow/utils/context.py:152 AirflowContextDeprecationWarning: Accessing 'yesterday_ds_nodash' from the template is deprecated and will be removed in a future version.
   [2022-02-07, 11:05:38 UTC] {logging_mixin.py:109} INFO - test_dag/test_logs/2022-02-06T00:00:00+00:00/5.log
   [2022-02-07, 11:05:38 UTC] {logging_mixin.py:109} INFO - /opt/airflow/logs/test_dag/test_logs/2022-02-06T00:00:00+00:00/5.log
   [2022-02-07, 11:05:38 UTC] {logging_mixin.py:109} INFO - *** Reading local file: /opt/airflow/logs/test_dag/test_logs/2022-02-06T00:00:00+00:00/5.log
   [2022-02-07, 11:05:06 UTC] {taskinstance.py:1032} INFO - Dependencies all met for <TaskInstance: test_dag.test_logs scheduled__2022-02-06T00:00:00+00:00 [queued]>
   [2022-02-07, 11:05:06 UTC] {taskinstance.py:1032} INFO - Dependencies all met for <TaskInstance: test_dag.test_logs scheduled__2022-02-06T00:00:00+00:00 [queued]>
   [2022-02-07, 11:05:06 UTC] {taskinstance.py:1238} INFO - 
   ```
   
   twice, as expected.
   
   This doesn't happen when a persistent volume is used to mount logs instead of `emptyDir`.
   
   
   
   ### What you expected to happen
   
   Logs from long-running tasks with KubernetesExecutor are read from local log file located under `$AIRFLOW_HOME/logs/...`, regardless how that path is mounted to a container - PV or emptyDir
   
   ### How to reproduce
   
   Change logs mount to `emptyDir`, enable remote logging to WASB and try to execute a long-running task. While a task is running, Airflow will always resort to reading logs from a pod, rather than from `$AIRFLOW_HOME/logs/...` location where the actual log file is.
   
   ### Operating System
   
   Debian GNU/Linux 10 (buster)
   
   ### Versions of Apache Airflow Providers
   
   apache-airflow-providers-cncf-kubernetes==2.2.0
   apache-airflow-providers-microsoft-azure==3.4.0
   
   
   ### Deployment
   
   Official Apache Airflow Helm Chart
   
   ### Deployment details
   
   Airflow is deployed via `helm_release` terraform plugin, chart version `1.3.0` with custom image, Dockerfile for it:
   ```
   FROM apache/airflow:2.2.3-python3.9
   
   COPY requirements.txt /tmp/requirements.txt
   
   RUN pip3 install --user -r /tmp/requirements.txt
   ```
   
   Requirements:
   
   ```
   apache-airflow-providers-databricks==2.2.0
   authlib==0.15.5
   ```
   
   ### Anything else
   
   Problem occurs at least on PythonOperator when using KubernetesExecutor
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
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] george-zubrienko commented on issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

Posted by GitBox <gi...@apache.org>.
george-zubrienko commented on issue #21387:
URL: https://github.com/apache/airflow/issues/21387#issuecomment-1032792614


   Might be caused by a different issue. Investigating further.


-- 
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] george-zubrienko edited a comment on issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

Posted by GitBox <gi...@apache.org>.
george-zubrienko edited a comment on issue #21387:
URL: https://github.com/apache/airflow/issues/21387#issuecomment-1032810002


   Yeah ok. So the problem is that when persistence is disabled, webserver for obvious reasons can't get logs out since it cannot access logs that become task-local. 
   I'd still put a suggestion on a table, what about letting webserver read log file from a pod? It's just the matter of doing an exec into it and sending file contents out?


-- 
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] george-zubrienko edited a comment on issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

Posted by GitBox <gi...@apache.org>.
george-zubrienko edited a comment on issue #21387:
URL: https://github.com/apache/airflow/issues/21387#issuecomment-1032810002


   Yeah ok. So the problem is that when [persistence is disabled](https://github.com/apache/airflow/blob/e97b72994f18e40e302ba8a14dbe73d34846a557/chart/templates/webserver/webserver-deployment.yaml#L175-L178), webserver for obvious reasons can't get logs out since it cannot access logs that become task-local. 
   I'd still put a suggestion on a table, what about letting webserver read log file from a pod? It's just the matter of doing an exec into it and sending file contents out?


-- 
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 closed issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

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


   


-- 
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] george-zubrienko commented on issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

Posted by GitBox <gi...@apache.org>.
george-zubrienko commented on issue #21387:
URL: https://github.com/apache/airflow/issues/21387#issuecomment-1040270851


   > Note that in the future we are likely to integrate open-telemetry for logging (there is a work in-progress on that) and that will allow to stream logs to any external or custom open-telemetry-compatible log sink in real time. This is the ultimate goal.
   
   I think this is definitely a way to go, and no PR I could offer will be better. For now, I've resolved our issue by adjusting the setup a bit:
   - setup logging to a persistent volume, so webserver can discover "local" logs while a task is running
   - setup remote logging, so once a task is done, log is shipped to remote storage
   - set a job to clean up PV regularly, since local logs are of no use. A note on this one, we actually disabled sidecar log groomer, because a) sidecar container, b) with >1 scheduler replica, we have >1 log groomer running `find ...` on the whole PV, which is really unnecessary, plus they are racing against each other.
   
   This way we have realtime logs served from the PV (fileshare) and logs from done tasks are read from remote storage, which is also a cheaper setup since read transaction cost is lower on blob file storage.
   
   Let me know if I should close this issue and link it to the one where `open-telemetry` implementation is tracked!
   
   


-- 
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 issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

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


   I think in many installations you will not be able to exec into running container from webserver. This is a common security practice that you are not allowed to do it in production.


-- 
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 issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

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






-- 
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] george-zubrienko commented on issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

Posted by GitBox <gi...@apache.org>.
george-zubrienko commented on issue #21387:
URL: https://github.com/apache/airflow/issues/21387#issuecomment-1040270851


   > Note that in the future we are likely to integrate open-telemetry for logging (there is a work in-progress on that) and that will allow to stream logs to any external or custom open-telemetry-compatible log sink in real time. This is the ultimate goal.
   
   I think this is definitely a way to go, and no PR I could offer will be better. For now, I've resolved our issue by adjusting the setup a bit:
   - setup logging to a persistent volume, so webserver can discover "local" logs while a task is running
   - setup remote logging, so once a task is done, log is shipped to remote storage
   - set a job to clean up PV regularly, since local logs are of no use. A note on this one, we actually disabled sidecar log groomer, because a) sidecar container, b) with >1 scheduler replica, we have >1 log groomer running `find ...` on the whole PV, which is really unnecessary, plus they are racing against each other.
   
   This way we have realtime logs served from the PV (fileshare) and logs from done tasks are read from remote storage, which is also a cheaper setup since read transaction cost is lower on blob file storage.
   
   Let me know if I should close this issue and link it to the one where `open-telemetry` implementation is tracked!
   
   


-- 
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] george-zubrienko commented on issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

Posted by GitBox <gi...@apache.org>.
george-zubrienko commented on issue #21387:
URL: https://github.com/apache/airflow/issues/21387#issuecomment-1032810002


   Yeah ok. So the problem is that when persistence is disabled, webserver for obvious reasons can't get logs out since it cannot access logs that become task-local. 
   I'd still put a suggestion on a table, what about letting webserver read log file from a pod? It's just the matter of doing and exec into it and sending file contents out?


-- 
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] george-zubrienko commented on issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

Posted by GitBox <gi...@apache.org>.
george-zubrienko commented on issue #21387:
URL: https://github.com/apache/airflow/issues/21387#issuecomment-1033002973


   > I think in many installations you will not be able to exec into running container from webserver. This is a common security practice that you are not allowed to do it in production.
   
   This assumes you have rbac in place that blocks exec. But airflow default pod security context has fsGroup 0, which is also not a common practice in production. Moreover, this is internal call, compared to `exec`s mainly used by outside callers.
   
   I'm not sure if opening a pod-pod websocket connection, which is what an SDK will do, is the same as `exec`? 
   
   Even if we put this aside, there there is still an issue with pod log != task log. And k8s pod logs are usually scraped to central logging processing system (datadog , cloudwatch etc.). Given the chaotic nature of airflow logs, it is not a great practice to feed them into pod stdout as some solutions on the internet suggest. So we are left with elastic/custom logger as the only viable options. But custom loggers are not guaranteed to survive next update and elastic is not the only log processing solution.
   Can we then change the way logs are written to remote, aka they are streamed instead of moved when done? What are the potential pitfalls there?
   


-- 
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 issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

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


   Closing as the logging will be added during the open-telemetry integration: #12771 


-- 
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 closed issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

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


   


-- 
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 issue #21387: Logs not read from emptyDir mount (local path) on KubernetesExecutor

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


   Opening a socket and serviing logs is FAR more secure than execing arbitrary command - and this is the pattern we already use in Celery workers (And it is even secured with secret key).
   
   If you think you can propose a good solution for that - feel free to propose a PR.
   
   Note that in the future we are likely to integrate open-telemetry for logging (there is a work in-progress on that) and that will allow to stream logs to any external or custom open-telemetry-compatible log sink in real time. This is the ultimate goal. 


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