From dc8a9a0e2845146f1d806c3074c88cdb873d867e Mon Sep 17 00:00:00 2001 From: Ryan Hatter <25823361+RNHTTR@users.noreply.github.com> Date: Sat, 21 Oct 2023 00:22:31 -0400 Subject: [PATCH 1/3] Replace numpy example with a practical exercise demonstrating top-level code --- docs/apache-airflow/best-practices.rst | 44 +++++++++++++------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index a0c277f4a94da..c6a0f83af9b7b 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -115,10 +115,10 @@ One of the important factors impacting DAG loading time, that might be overlooke that top-level imports might take surprisingly a lot of time and they can generate a lot of overhead and this can be easily avoided by converting them to local imports inside Python callables for example. -Consider the example below - the first DAG will parse significantly slower (in the orders of seconds) -than equivalent DAG where the ``numpy`` module is imported as local import in the callable. +Consider the two examples below. In the first example, DAG will take an additional 1000 seconds to parse +than the functionally equivalent DAG in the second example where the ``expensive_api_call`` is executed from the context of its task. -Bad example: +Not avoiding top-level DAG code: .. code-block:: python @@ -127,7 +127,12 @@ Bad example: from airflow import DAG from airflow.decorators import task - import numpy as np # <-- THIS IS A VERY BAD IDEA! DON'T DO THAT! + + def expensive_api_call(): + print("Hello from Airflow!") + sleep(1000) + + my_expensive_response = expensive_api_call() with DAG( dag_id="example_python_operator", @@ -138,15 +143,10 @@ Bad example: ) as dag: @task() - def print_array(): - """Print Numpy array.""" - a = np.arange(15).reshape(3, 5) - print(a) - return a - - print_array() + def print_expensive_api_call(): + print(my_expensive_response) -Good example: +Avoiding top-level DAG code: .. code-block:: python @@ -155,6 +155,11 @@ Good example: from airflow import DAG from airflow.decorators import task + + def expensive_api_call(): + sleep(1000) + return "Hello from Airflow!" + with DAG( dag_id="example_python_operator", schedule=None, @@ -164,19 +169,14 @@ Good example: ) as dag: @task() - def print_array(): - """Print Numpy array.""" - import numpy as np # <- THIS IS HOW NUMPY SHOULD BE IMPORTED IN THIS CASE! - - a = np.arange(15).reshape(3, 5) - print(a) - return a + def print_expensive_api_call(): + my_expensive_response = expensive_api_call() + print(my_expensive_response) - print_array() +In the first example, ``expensive_api_call`` is executed each time the DAG file is parsed, which will result in suboptimal performance in the DAG file processing. In the second example, ``expensive_api_call`` is only called when the task is running and thus is able to be parsed without suffering any performance hits. To test it out yourself, implement the first DAG and see "Hello from Airflow!" printed _in the scheduler logs_! -In the Bad example, NumPy is imported each time the DAG file is parsed, which will result in suboptimal performance in the DAG file processing. In the Good example, NumPy is only imported when the task is running. +Note that import statements also count as top-level code. So, if you have an import statement that takes a long time or the imported module itself executes code at the top-level, that too can impact the performance of the scheduler. -Since it is not always obvious, see the next chapter to check how my code is "top-level" code. How to check if my code is "top-level" code ------------------------------------------- From 8abb2c1d06ffd1c51c57181b974cb6faaef9809d Mon Sep 17 00:00:00 2001 From: Ryan Hatter Date: Tue, 2 Jan 2024 19:56:53 -0500 Subject: [PATCH 2/3] add warning --- docs/apache-airflow/best-practices.rst | 36 +++++++++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index c6a0f83af9b7b..2b68003791bc9 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -132,6 +132,7 @@ Not avoiding top-level DAG code: print("Hello from Airflow!") sleep(1000) + my_expensive_response = expensive_api_call() with DAG( @@ -160,6 +161,7 @@ Avoiding top-level DAG code: sleep(1000) return "Hello from Airflow!" + with DAG( dag_id="example_python_operator", schedule=None, @@ -175,7 +177,39 @@ Avoiding top-level DAG code: In the first example, ``expensive_api_call`` is executed each time the DAG file is parsed, which will result in suboptimal performance in the DAG file processing. In the second example, ``expensive_api_call`` is only called when the task is running and thus is able to be parsed without suffering any performance hits. To test it out yourself, implement the first DAG and see "Hello from Airflow!" printed _in the scheduler logs_! -Note that import statements also count as top-level code. So, if you have an import statement that takes a long time or the imported module itself executes code at the top-level, that too can impact the performance of the scheduler. +Note that import statements also count as top-level code. So, if you have an import statement that takes a long time or the imported module itself executes code at the top-level, that can also impact the performance of the scheduler. The following example illustrates how to handle expensive imports. + +.. code-block:: python + + # It's ok to import modules that are not expensive to load at top-level of DAG file + import random + import pendulum + + # Expensive imports should be avoided as top level imports because DAG is imported frequently + # even if it does not follow PEP8 advice (PEP8 have not foreseen that certain imports will be very expensive) + # DON'T DO THAT - import them locally instead (see below) + # + # import pandas + # import torch + # import tensorflow + # + + ... + + + @task() + def do_stuff_with_pandas_and_torch(): + import pandas + import torch + + # do some operations using pandas and torch + + + @task() + def do_stuff_with_tensorflow(): + import tensorflow + + # do some operations using tensorflow How to check if my code is "top-level" code From f0b1dc48922510e358a95d87db5a3274db6bfa06 Mon Sep 17 00:00:00 2001 From: Ryan Hatter Date: Fri, 5 Jan 2024 11:37:53 -0500 Subject: [PATCH 3/3] fix build errors --- docs/apache-airflow/best-practices.rst | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/docs/apache-airflow/best-practices.rst b/docs/apache-airflow/best-practices.rst index 2b68003791bc9..e166c1024797c 100644 --- a/docs/apache-airflow/best-practices.rst +++ b/docs/apache-airflow/best-practices.rst @@ -175,19 +175,17 @@ Avoiding top-level DAG code: my_expensive_response = expensive_api_call() print(my_expensive_response) -In the first example, ``expensive_api_call`` is executed each time the DAG file is parsed, which will result in suboptimal performance in the DAG file processing. In the second example, ``expensive_api_call`` is only called when the task is running and thus is able to be parsed without suffering any performance hits. To test it out yourself, implement the first DAG and see "Hello from Airflow!" printed _in the scheduler logs_! +In the first example, ``expensive_api_call`` is executed each time the DAG file is parsed, which will result in suboptimal performance in the DAG file processing. In the second example, ``expensive_api_call`` is only called when the task is running and thus is able to be parsed without suffering any performance hits. To test it out yourself, implement the first DAG and see "Hello from Airflow!" printed in the scheduler logs! Note that import statements also count as top-level code. So, if you have an import statement that takes a long time or the imported module itself executes code at the top-level, that can also impact the performance of the scheduler. The following example illustrates how to handle expensive imports. .. code-block:: python - # It's ok to import modules that are not expensive to load at top-level of DAG file + # It's ok to import modules that are not expensive to load at top-level of a DAG file import random import pendulum - # Expensive imports should be avoided as top level imports because DAG is imported frequently - # even if it does not follow PEP8 advice (PEP8 have not foreseen that certain imports will be very expensive) - # DON'T DO THAT - import them locally instead (see below) + # Expensive imports should be avoided as top level imports, because DAG files are parsed frequently, resulting in top-level code being executed. # # import pandas # import torch