You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "jedcunningham (via GitHub)" <gi...@apache.org> on 2023/02/02 19:56:21 UTC

[GitHub] [airflow] jedcunningham commented on a diff in pull request #27758: Enable trigger logging in webserver

jedcunningham commented on code in PR #27758:
URL: https://github.com/apache/airflow/pull/27758#discussion_r1095013539


##########
chart/templates/triggerer/triggerer-service.yaml:
##########
@@ -0,0 +1,48 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+################################
+## Airflow triggerer Service
+#################################
+{{- if semverCompare ">=2.2.0" .Values.airflowVersion }}

Review Comment:
   Should this be 2.6.0 instead? There won't be any endpoints in earlier versions?



##########
docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst:
##########
@@ -47,11 +44,14 @@ These patterns can be adjusted by :ref:`config:logging__log_filename_template`.
 
 In addition, you can supply a remote location to store current logs and backups.
 
-In the Airflow UI, remote logs take precedence over local logs when remote logging is enabled. If remote logs
-can not be found or accessed, local logs will be displayed. Note that logs
-are only sent to remote storage once a task is complete (including failure). In other words, remote logs for
-running tasks are unavailable (but local logs are available).
+Interleaving of logs
+--------------------
+
+Airflow's remote task logging handlers can broadly be separated into two categories: streaming handlers (such as ElasticSearch, AWS Cloudwatch, and GCP operations logging, formerly stackdriver) and blob storage handlers (e.g. S3, GCS, WASB).
+
+For blob storage handlers, depending on the state of the task, logs could be in a lot of different places and in multiple different files.  For this reason, we need to check all locations and interleave what we find.  To do this we need to be able to parse the timestamp for each line.  If you are using a custom formatter you may need to override the default parser by providing a callable name at Airflow setting ``logging > interleave_timestamp_parser``.

Review Comment:
   ```suggestion
   For blob storage handlers, depending on the state of the task, logs could be in a lot of different places and in multiple different files.  For this reason, we need to check all locations and interleave what we find.  To do this we need to be able to parse the timestamp for each line.  If you are using a custom formatter you may need to override the default parser by providing a callable name at Airflow setting ``[logging] interleave_timestamp_parser``.
   ```
   
   super nit



##########
docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst:
##########
@@ -122,15 +122,33 @@ When using remote logging, you can configure Airflow to show a link to an extern
 
 Some external systems require specific configuration in Airflow for redirection to work but others do not.
 
-Serving logs from workers
--------------------------
+Serving logs from workers and triggerer
+---------------------------------------
 
-Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow automatically starts an HTTP server to serve the logs in the following cases:
+Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow starts an HTTP server to serve the logs in the following cases:
 
 - If ``SequentialExecutor`` or ``LocalExecutor`` is used, then when ``airflow scheduler`` is running.
 - If ``CeleryExecutor`` is used, then when ``airflow worker`` is running.
 
-The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section. By default, it is ``8793``.
+In triggerer, logs are served unless the service is started with option ``--skip-serve-logs``.
+
+The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section, and option ``triggerer_log_server_port`` for triggerer.  Defaults are 8793 and 8794, respectively.
 Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option  in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems.
 
 We are using `Gunicorn <https://gunicorn.org/>`__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings <https://docs.gunicorn.org/en/latest/settings.html#settings>`__.
+
+Implementing a custom file task handler
+---------------------------------------
+
+.. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`.
+
+In our providers we have a healthy variety of options with all the major cloud providers.  But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging.
+
+Triggers require a shift in the way that logging is set up.  In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler.  And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them.
+
+To accomplish this we have a few attributes that may be set either on the handler, either instance or the class.  Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics.  These params are described below:
+
+- ``trigger_should_wrap``: controls whether this handler should be wrapped by TriggerHandlerWrapper.  This is necessary when each instance of handler creates a file handler that it writes all messages to.
+- ``trigger_should_queue``: controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop.
+- ``trigger_send_end_marker``: whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed.

Review Comment:
   ```suggestion
   - ``trigger_send_end_marker``: controls whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed.
   ```



##########
docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst:
##########
@@ -122,15 +122,33 @@ When using remote logging, you can configure Airflow to show a link to an extern
 
 Some external systems require specific configuration in Airflow for redirection to work but others do not.
 
-Serving logs from workers
--------------------------
+Serving logs from workers and triggerer
+---------------------------------------
 
-Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow automatically starts an HTTP server to serve the logs in the following cases:
+Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow starts an HTTP server to serve the logs in the following cases:
 
 - If ``SequentialExecutor`` or ``LocalExecutor`` is used, then when ``airflow scheduler`` is running.
 - If ``CeleryExecutor`` is used, then when ``airflow worker`` is running.
 
-The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section. By default, it is ``8793``.
+In triggerer, logs are served unless the service is started with option ``--skip-serve-logs``.
+
+The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section, and option ``triggerer_log_server_port`` for triggerer.  Defaults are 8793 and 8794, respectively.
 Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option  in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems.
 
 We are using `Gunicorn <https://gunicorn.org/>`__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings <https://docs.gunicorn.org/en/latest/settings.html#settings>`__.
+
+Implementing a custom file task handler
+---------------------------------------
+
+.. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`.
+
+In our providers we have a healthy variety of options with all the major cloud providers.  But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging.
+
+Triggers require a shift in the way that logging is set up.  In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler.  And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them.
+
+To accomplish this we have a few attributes that may be set either on the handler, either instance or the class.  Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics.  These params are described below:

Review Comment:
   ```suggestion
   To accomplish this we have a few attributes that may be set on the handlers' instance or class.  Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics.  These params are described below:
   ```



##########
tests/charts/test_extra_env_env_from.py:
##########
@@ -114,6 +114,7 @@ def test_extra_env(self, k8s_obj_key, env_paths):
                   name: {RELEASE_NAME}-some-secret
             """
         ).lstrip()
+        print(k8s_obj_key)

Review Comment:
   ```suggestion
   ```



##########
docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst:
##########
@@ -122,15 +122,33 @@ When using remote logging, you can configure Airflow to show a link to an extern
 
 Some external systems require specific configuration in Airflow for redirection to work but others do not.
 
-Serving logs from workers
--------------------------
+Serving logs from workers and triggerer
+---------------------------------------
 
-Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow automatically starts an HTTP server to serve the logs in the following cases:
+Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow starts an HTTP server to serve the logs in the following cases:
 
 - If ``SequentialExecutor`` or ``LocalExecutor`` is used, then when ``airflow scheduler`` is running.
 - If ``CeleryExecutor`` is used, then when ``airflow worker`` is running.
 
-The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section. By default, it is ``8793``.
+In triggerer, logs are served unless the service is started with option ``--skip-serve-logs``.
+
+The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section, and option ``triggerer_log_server_port`` for triggerer.  Defaults are 8793 and 8794, respectively.
 Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option  in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems.
 
 We are using `Gunicorn <https://gunicorn.org/>`__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings <https://docs.gunicorn.org/en/latest/settings.html#settings>`__.
+
+Implementing a custom file task handler
+---------------------------------------
+
+.. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`.
+
+In our providers we have a healthy variety of options with all the major cloud providers.  But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging.

Review Comment:
   ```suggestion
   In our providers we have a healthy variety of options with all the major cloud providers.  However, should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging.
   ```



##########
docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst:
##########
@@ -122,15 +122,33 @@ When using remote logging, you can configure Airflow to show a link to an extern
 
 Some external systems require specific configuration in Airflow for redirection to work but others do not.
 
-Serving logs from workers
--------------------------
+Serving logs from workers and triggerer
+---------------------------------------
 
-Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow automatically starts an HTTP server to serve the logs in the following cases:
+Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow starts an HTTP server to serve the logs in the following cases:
 
 - If ``SequentialExecutor`` or ``LocalExecutor`` is used, then when ``airflow scheduler`` is running.
 - If ``CeleryExecutor`` is used, then when ``airflow worker`` is running.
 
-The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section. By default, it is ``8793``.
+In triggerer, logs are served unless the service is started with option ``--skip-serve-logs``.
+
+The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section, and option ``triggerer_log_server_port`` for triggerer.  Defaults are 8793 and 8794, respectively.
 Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option  in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems.
 
 We are using `Gunicorn <https://gunicorn.org/>`__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings <https://docs.gunicorn.org/en/latest/settings.html#settings>`__.
+
+Implementing a custom file task handler
+---------------------------------------
+
+.. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`.
+
+In our providers we have a healthy variety of options with all the major cloud providers.  But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging.
+
+Triggers require a shift in the way that logging is set up.  In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler.  And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them.

Review Comment:
   ```suggestion
   Triggers require a shift in the way that logging is set up.  In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler.  Due to the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them.
   ```



##########
docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst:
##########
@@ -122,15 +122,33 @@ When using remote logging, you can configure Airflow to show a link to an extern
 
 Some external systems require specific configuration in Airflow for redirection to work but others do not.
 
-Serving logs from workers
--------------------------
+Serving logs from workers and triggerer
+---------------------------------------
 
-Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow automatically starts an HTTP server to serve the logs in the following cases:
+Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow starts an HTTP server to serve the logs in the following cases:
 
 - If ``SequentialExecutor`` or ``LocalExecutor`` is used, then when ``airflow scheduler`` is running.
 - If ``CeleryExecutor`` is used, then when ``airflow worker`` is running.
 
-The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section. By default, it is ``8793``.
+In triggerer, logs are served unless the service is started with option ``--skip-serve-logs``.
+
+The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section, and option ``triggerer_log_server_port`` for triggerer.  Defaults are 8793 and 8794, respectively.
 Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option  in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems.
 
 We are using `Gunicorn <https://gunicorn.org/>`__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings <https://docs.gunicorn.org/en/latest/settings.html#settings>`__.
+
+Implementing a custom file task handler
+---------------------------------------
+
+.. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`.
+
+In our providers we have a healthy variety of options with all the major cloud providers.  But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging.
+
+Triggers require a shift in the way that logging is set up.  In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler.  And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them.
+
+To accomplish this we have a few attributes that may be set either on the handler, either instance or the class.  Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics.  These params are described below:
+
+- ``trigger_should_wrap``: controls whether this handler should be wrapped by TriggerHandlerWrapper.  This is necessary when each instance of handler creates a file handler that it writes all messages to.
+- ``trigger_should_queue``: controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop.
+- ``trigger_send_end_marker``: whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed.
+- ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers.  But if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages.  Essentially, this should be true for handlers that "natively" support triggers.  And one such example of this is the StackdriverTaskHandler.

Review Comment:
   ```suggestion
   - ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers.  However, if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages.  Essentially, this should be true for handlers that "natively" support triggers. One such example of this is the StackdriverTaskHandler.
   ```



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