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/07/22 10:37:49 UTC

[GitHub] [airflow] feluelle commented on a diff in pull request #25161: Robust context of dag parsing

feluelle commented on code in PR #25161:
URL: https://github.com/apache/airflow/pull/25161#discussion_r927523656


##########
docs/apache-airflow/howto/dynamic-dag-generation.rst:
##########
@@ -140,3 +140,120 @@ Each of them can run separately with related configuration
 
 .. warning::
   Using this practice, pay attention to "late binding" behaviour in Python loops. See `that GitHub discussion <https://github.com/apache/airflow/discussions/21278#discussioncomment-2103559>`_ for more details
+
+
+Optimizing DAG parsing delays during execution
+----------------------------------------------
+
+Sometimes when you generate a lot of Dynamic DAGs from a single DAG file, it might cause unnecessary delays
+when the DAG file is parsed during task execution. The impact is a delay before a task starts.
+
+Why is this happening? You might not be aware but just before your task is executed,
+Airflow parses the Python file the DAG comes from.
+
+The Airflow Scheduler (or DAG Processor) requires loading of a complete DAG file to process all metadata.
+However, task execution requires only a single DAG object to execute a task. Knowing this, we can
+skip the generation of unnecessary DAG objects when a task is executed, shortening the parsing time.
+This optimization is most effective when the number of generated DAGs is high.
+
+There is an experimental approach that you can take to optimize this behaviour. Note that it is not always
+possible to use (for example when generation of subsequent DAGs depends on the previous DAGs) or when
+there are some side-effects of your DAGs generation. Also the code snippet below is pretty complex and while
+we tested it and it works in most circumstances, there might be cases where detection of the currently
+parsed DAG will fail and it will revert to creating all the DAGs or fail. Use this solution with care and
+test it thoroughly.
+
+A nice example of performance improvements you can gain is shown in the
+`Airflow's Magic Loop <https://medium.com/apache-airflow/airflows-magic-loop-ec424b05b629>`_ blog post
+that describes how parsing during task execution was reduced from 120 seconds to 200 ms.
+
+Airflow 2.4+
+............
+
+In Airflow 2.4 the following variables you can use
+``airflow.utils.dag_parsing_context import get_parsing_context`` method to retrieve the current context.
+
+Upon iterating over the collection of things to generate DAGs for, you can use the context to determine
+whether you need to generate all DAG objects (when parsing in the DAG File processor), or to generate only
+a single DAG object (when executing the task):
+
+.. code-block:: python
+  :emphasize-lines: 6,7,8,12,13
+
+  import os
+  from airflow.models.dag import DAG
+  from airflow.utils.dag_parsing_context import get_parsing_context, AirflowParsingContextType
+
+  current_dag = None
+  parsing_context = get_parsing_context()
+  if parsing_context and parsing_context.context_type == AirflowParsingContextType.TASK_EXECUTION:
+      current_dag = parsing_context.dag_id
+
+  for thing in list_of_things:
+      dag_id = f"generated_dag_{thing}"
+      if current_dag is not None and current_dag != dag_id:
+          continue  # skip generation of non-selected DAG
+
+      dag = DAG(dag_id=dag_id, ...)
+      globals()[dag_id] = dag
+
+
+
+Airflow 2.3 and below
+......................
+
+In Airflow 2.3 and below, upon evaluation of a DAG file, command line arguments are supplied
+which we can use to determine which Airflow component performs parsing:
+
+* Scheduler/DAG Processor args: ``["airflow", "scheduler"]`` or ``["airflow", "dag-processor"]``
+* Task execution args: ``["airflow", "tasks", "run", "dag_id", "task_id", ...]``
+
+However, depending on the executor used and forking model, those args might be available via ``sys.args``
+or via name of the process running. Airflow either executes tasks via running a new Python interpreter or
+sets the name of the process as "airflow task supervisor: {ARGS}" in case of celery forked process or
+"airflow task runner: dag_id task_id" in case of local executor forked process.
+
+Upon iterating over the collection of things to generate DAGs for, you can use these arguments to determine
+whether you need to generate all DAG objects (when parsing in the DAG File processor), or to generate only
+a single DAG object (when executing the task):
+
+.. code-block:: python
+  :emphasize-lines: 7,8,9,19,20,24,25,31,32
+
+  import sys
+  import ast
+  import setproctitle
+  from airflow.models.dag import DAG
+
+  current_dag = None
+  if len(sys.argv) > 3 and sys.argv[1] == "tasks":
+      # task executed by starting a new Python interpreter
+      current_dag = sys.argv[3]
+  else:
+      try:
+          PROCTITLE_SUPERVISOR_PREFIX = "airflow task supervisor: "
+          PROCTITLE_TASK_RUNNER_PREFIX = "airflow task runner: "
+          proctitle = str(setproctitle.getproctitle())
+          if proctitle.startswith(PROCTITLE_SUPERVISOR_PREFIX):
+              # task executed via forked process in celery
+              args_string = proctitle[len(PROCTITLE_SUPERVISOR_PREFIX) :]
+              args = ast.literal_eval(args_string)
+              if len(args) > 3 and args[1] == "tasks":
+                  current_dag = args[3]
+          elif proctitle.startswith(PROCTITLE_TASK_RUNNER_PREFIX):
+              # task executed via forked process in standard_task_runner
+              args = proctitle[len(PROCTITLE_TASK_RUNNER_PREFIX) :].split(" ")
+              if len(args) > 0:
+                  current_dag = args[0]
+      except Exception:
+          pass

Review Comment:
   This to me looks a bit like "_Don't try this at home._" 😅 I think I do I agree with @jedcunningham that this should not be in official docs.
   
   Your "robust" approach is much better :)



##########
docs/apache-airflow/howto/dynamic-dag-generation.rst:
##########
@@ -140,3 +140,120 @@ Each of them can run separately with related configuration
 
 .. warning::
   Using this practice, pay attention to "late binding" behaviour in Python loops. See `that GitHub discussion <https://github.com/apache/airflow/discussions/21278#discussioncomment-2103559>`_ for more details
+
+
+Optimizing DAG parsing delays during execution
+----------------------------------------------
+
+Sometimes when you generate a lot of Dynamic DAGs from a single DAG file, it might cause unnecessary delays
+when the DAG file is parsed during task execution. The impact is a delay before a task starts.
+
+Why is this happening? You might not be aware but just before your task is executed,
+Airflow parses the Python file the DAG comes from.
+
+The Airflow Scheduler (or DAG Processor) requires loading of a complete DAG file to process all metadata.
+However, task execution requires only a single DAG object to execute a task. Knowing this, we can
+skip the generation of unnecessary DAG objects when a task is executed, shortening the parsing time.
+This optimization is most effective when the number of generated DAGs is high.
+
+There is an experimental approach that you can take to optimize this behaviour. Note that it is not always
+possible to use (for example when generation of subsequent DAGs depends on the previous DAGs) or when
+there are some side-effects of your DAGs generation. Also the code snippet below is pretty complex and while
+we tested it and it works in most circumstances, there might be cases where detection of the currently
+parsed DAG will fail and it will revert to creating all the DAGs or fail. Use this solution with care and
+test it thoroughly.
+
+A nice example of performance improvements you can gain is shown in the
+`Airflow's Magic Loop <https://medium.com/apache-airflow/airflows-magic-loop-ec424b05b629>`_ blog post
+that describes how parsing during task execution was reduced from 120 seconds to 200 ms.
+
+Airflow 2.4+
+............
+
+In Airflow 2.4 the following variables you can use
+``airflow.utils.dag_parsing_context import get_parsing_context`` method to retrieve the current context.
+
+Upon iterating over the collection of things to generate DAGs for, you can use the context to determine
+whether you need to generate all DAG objects (when parsing in the DAG File processor), or to generate only
+a single DAG object (when executing the task):
+
+.. code-block:: python
+  :emphasize-lines: 6,7,8,12,13
+
+  import os

Review Comment:
   ```suggestion
   ```



##########
airflow/utils/dag_parsing_context.py:
##########
@@ -0,0 +1,65 @@
+# 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.
+import os
+from contextlib import contextmanager
+from enum import Enum
+from typing import NamedTuple, Optional
+
+
+class AirflowParsingContextType(Enum):
+    """Type of context of parsing the DAG."""
+
+    DAG_PROCESSOR = "DAG_PROCESSOR"
+    TASK_EXECUTION = "TASK_EXECUTION"
+
+
+class AirflowParsingContext(NamedTuple):
+    """Context of parsing for the DAG."""
+
+    context_type: Optional[AirflowParsingContextType]
+    dag_id: Optional[str]
+    task_id: Optional[str]
+
+
+_AIRFLOW_PARSING_CONTEXT = "_AIRFLOW_PARSING_CONTEXT"
+_AIRFLOW_PARSING_CONTEXT_DAG_ID = "_AIRFLOW_PARSING_CONTEXT_DAG_ID"
+_AIRFLOW_PARSING_CONTEXT_TASK_ID = "_AIRFLOW_PARSING_CONTEXT_TASK_ID"
+
+
+@contextmanager
+def _airflow_parsing_context_manager(
+    context: AirflowParsingContextType, dag_id: Optional[str] = None, task_id: Optional[str] = None
+):
+    os.environ[_AIRFLOW_PARSING_CONTEXT] = context.value
+    if context == AirflowParsingContextType.TASK_EXECUTION:
+        os.environ[_AIRFLOW_PARSING_CONTEXT_DAG_ID] = dag_id if dag_id is not None else ""
+        os.environ[_AIRFLOW_PARSING_CONTEXT_TASK_ID] = task_id if task_id is not None else ""

Review Comment:
   ```suggestion
           os.environ[_AIRFLOW_PARSING_CONTEXT_DAG_ID] = dag_id
           os.environ[_AIRFLOW_PARSING_CONTEXT_TASK_ID] = task_id
   ```
   Why do you need to check this? Wouldn't `None` work as well? An empty `""` id does not make sense in my opinion, especially because you are using `Optional[str] = None` everywhere anyway. In the `get_parsing_context()` you are using `os.environ.get()` which means it will either be `None` or the value.



##########
docs/apache-airflow/howto/dynamic-dag-generation.rst:
##########
@@ -140,3 +140,120 @@ Each of them can run separately with related configuration
 
 .. warning::
   Using this practice, pay attention to "late binding" behaviour in Python loops. See `that GitHub discussion <https://github.com/apache/airflow/discussions/21278#discussioncomment-2103559>`_ for more details
+
+
+Optimizing DAG parsing delays during execution
+----------------------------------------------
+
+Sometimes when you generate a lot of Dynamic DAGs from a single DAG file, it might cause unnecessary delays
+when the DAG file is parsed during task execution. The impact is a delay before a task starts.
+
+Why is this happening? You might not be aware but just before your task is executed,
+Airflow parses the Python file the DAG comes from.
+
+The Airflow Scheduler (or DAG Processor) requires loading of a complete DAG file to process all metadata.
+However, task execution requires only a single DAG object to execute a task. Knowing this, we can
+skip the generation of unnecessary DAG objects when a task is executed, shortening the parsing time.
+This optimization is most effective when the number of generated DAGs is high.
+
+There is an experimental approach that you can take to optimize this behaviour. Note that it is not always
+possible to use (for example when generation of subsequent DAGs depends on the previous DAGs) or when
+there are some side-effects of your DAGs generation. Also the code snippet below is pretty complex and while
+we tested it and it works in most circumstances, there might be cases where detection of the currently
+parsed DAG will fail and it will revert to creating all the DAGs or fail. Use this solution with care and
+test it thoroughly.
+
+A nice example of performance improvements you can gain is shown in the
+`Airflow's Magic Loop <https://medium.com/apache-airflow/airflows-magic-loop-ec424b05b629>`_ blog post
+that describes how parsing during task execution was reduced from 120 seconds to 200 ms.
+
+Airflow 2.4+
+............
+
+In Airflow 2.4 the following variables you can use
+``airflow.utils.dag_parsing_context import get_parsing_context`` method to retrieve the current context.
+
+Upon iterating over the collection of things to generate DAGs for, you can use the context to determine
+whether you need to generate all DAG objects (when parsing in the DAG File processor), or to generate only
+a single DAG object (when executing the task):
+
+.. code-block:: python
+  :emphasize-lines: 6,7,8,12,13
+
+  import os
+  from airflow.models.dag import DAG
+  from airflow.utils.dag_parsing_context import get_parsing_context, AirflowParsingContextType
+
+  current_dag = None
+  parsing_context = get_parsing_context()
+  if parsing_context and parsing_context.context_type == AirflowParsingContextType.TASK_EXECUTION:
+      current_dag = parsing_context.dag_id

Review Comment:
   This looks so much cleaner. I like it a lot.



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