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/04/08 19:58:10 UTC

[GitHub] [airflow] ashb opened a new pull request, #22867: Add concept doc for Dynamic Task Mapping

ashb opened a new pull request, #22867:
URL: https://github.com/apache/airflow/pull/22867

   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/main/UPDATING.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] uranusjr commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846622609


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)

Review Comment:
   ```suggestion
       # add(x=1, y=10)
       # add(x=2, y=10)
       # add(x=3, y=10)
   ```



-- 
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] ashb commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r847223664


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,260 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)) as dag:
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task.
+
+.. code-block:: python
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)) as dag:
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of ``[3, 4, 5]``.
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1, y=10)
+    # add(x=2, y=10)
+    # add(x=3, y=10)
+
+This would result in values of 11, 12, and 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's ``zip`` function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over.
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The ``make_list`` task runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the ``consumer`` task will be called four times, once with each value in the return of ``make_list``.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use ``partial`` and ``expand`` with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as ``task_id``, ``queue``, ``pool``, and most other arguments to ``BaseOperator``.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an ``XComArg`` object manually.
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: 

Review Comment:
   ```suggestion
   There are two limits that you can place on a task:
   
   ```



-- 
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] uranusjr commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846622440


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):

Review Comment:
   But this variable is never used



-- 
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] uranusjr commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846622629


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.

Review Comment:
   ```suggestion
   This would result in values of 11, 12, and 13.
   ```



-- 
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] jedcunningham commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846507622


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+

Review Comment:
   ```suggestion
   ```
   
   Let's drop these as we've already shown them in the first example.



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task

Review Comment:
   ```suggestion
   The result of one mapped task can also be used as input to the next mapped task.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:

Review Comment:
   ```suggestion
   If you want to map over the result of a classic operator you will need to create an ``XComArg`` object manually.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once
+
+- **Limiting number of mapped task**
+
+  The [core] ``max_map_length`` config option is the maximum number of task that expand can create – the default value is 1024.
+
+  If a source task ("make_list" in our earlier example) returns a list longer than this it will result in *that* task failing.

Review Comment:
   ```suggestion
     If a source task (``make_list`` in our earlier example) returns a list longer than this it will result in *that* task failing.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]

Review Comment:
   ```suggestion
   This would have a result of ``[3, 4, 5]``.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:

Review Comment:
   ```suggestion
   Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.

Review Comment:
   ```suggestion
   ``make_list`` runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the ``consumer`` task will be called four times, once with each value in the return of ``make_list``.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+

Review Comment:
   ```suggestion
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+

Review Comment:
   ```suggestion
   ```
   
   Let's be consistent.



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once
+
+- **Limiting number of mapped task**
+
+  The [core] ``max_map_length`` config option is the maximum number of task that expand can create – the default value is 1024.

Review Comment:
   ```suggestion
     The [core] ``max_map_length`` config option is the maximum number of tasks that ``expand`` can create – the default value is 1024.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once

Review Comment:
   ```suggestion
   There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.

Review Comment:
   ```suggestion
   It is possible to use ``partial`` and ``expand`` with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as ``task_id``, ``queue``, ``pool``, and most other arguments to ``BaseOperator``.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once
+
+- **Limiting number of mapped task**
+
+  The [core] ``max_map_length`` config option is the maximum number of task that expand can create – the default value is 1024.
+
+  If a source task ("make_list" in our earlier example) returns a list longer than this it will result in *that* task failing.
+
+- **Limiting parallel copies of a mapped task**
+
+  If you wish to not have a large mapped task consume all available runner slots you can use the ``max_active_tis_per_dag`` setting on the task to restrict how many can be running at the same time.
+
+  Note however that this applies to all copies of that task against all active DagRuns, not just to this one specific DagRun.
+
+  .. code-block:: python
+
+      @task(max_active_tis_per_dag=16)
+      def add_one(x: int):
+          return x + 1
+
+
+      BashOperator.partial(task_id="my_task", max_active_tis_per_dag=16).expand(
+          bash_command=commands
+      )
+
+Automatically skipping zero-length maps
+=======================================
+
+If the input is empty (zero length), no new tasks will be created and the mapped Task will be marked as SKIPPED.

Review Comment:
   ```suggestion
   If the input is empty (zero length), no new tasks will be created and the mapped task will be marked as ``SKIPPED``.
   ```



-- 
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] ashb commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846455969


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the

Review Comment:
   No. List or Dict.



-- 
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] ashb commented on pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on PR #22867:
URL: https://github.com/apache/airflow/pull/22867#issuecomment-1093305431

   This would benefit from some screenshots, but @bbovenzi is still (sublty) tweaking things there so I won't add them just yet.


-- 
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] dstandish commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846483452


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.

Review Comment:
   yeah i know that thought occured to me too
   
   but sometimes being precise can get clunky ... 
   
   can think of it as a metaphor
   
   in a sense you could say that the dag is creating it because, it kindof is, that's the dynamic part of it... the pipeline is dynamic and it creates its own tasks, and whether it is scheduler or the "DAG"  is sort of implementation detail.  but yeah prob there's a way to make it both precise _and_ elegant 🤷 



-- 
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] ashb commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846494080


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator. 

Review Comment:
   ```suggestion
   It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
   ```



-- 
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] jedcunningham commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846642784


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.

Review Comment:
   No, "or" is correct here, as it means this:
   
   against a dict, a list, or one of those types (dict, list) stored in XCom



-- 
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] jedcunningham commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846642864


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.

Review Comment:
   Yeah, that works as well.



-- 
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] ashb commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846457188


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable, such as task_id, queue, pool, and most other arguments to BaseOperator and have to be passed to ``partial()``
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once

Review Comment:
   parent task should be the upstream task/source of the mapping list. Can you remember where I referred to it like that?



-- 
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] jedcunningham commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846644101


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.

Review Comment:
   Done.



-- 
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] jedcunningham commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846523919


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG

Review Comment:
   ```suggestion
       from datetime import datetime
   
       from airflow import DAG
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):

Review Comment:
   ```suggestion
       with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)) as dag:
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):

Review Comment:
   ```suggestion
       with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)) as dag:
   ```



-- 
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] ashb commented on pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on PR #22867:
URL: https://github.com/apache/airflow/pull/22867#issuecomment-1094103558

   > A few general questions:
   > 1. Should we say something about trigger rules? I guess if one of the items in mapped task fails than the whole task fails? This may open a big window of how trigger rules should work.
   
   Conceptually at a high level. The way it's implemented all upstream TaskInstances (mapped or not) are counted - so a task downstream of a mapped task is like having n upstreams to check in trigger rules.
   
   > 2. Referring to question 1... "one of the items in mapped task fails" is very long. Should we come up with an easy simple name so speak of a specific instance in a maped task?
   
   "Mapped task instance" is what we use in the code for this 
   
   > 3. This doc demonstrate a case where a list was pushed to Xcom and then we generate a mapped task from it to parse it. What happens if user wants to push to Xcom in the mapped task? Before the API - the key  would be the task_id. What it will be now?
   
   It now also includes map_index (default of -1 to mean not mapped. PK cols can't be null, so we had to go with -1)
   
   Pushing from a mapped task instance is shown (implicitly via return values) in the Repeated Mapping example.
   


-- 
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] dstandish commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846488139


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable, such as task_id, queue, pool, and most other arguments to BaseOperator and have to be passed to ``partial()``
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once

Review Comment:
   when i say "parent" i am referring to the mapped operator instance essentially.... which is itself never gonna be a task ... the tasks that get created are the instances that operato on each input.  and that's what you mean by `how many of the mapped task can run at once` no? like the instances -- not the "parent".  as to where, there are a lot of places where it's a little ambiguous (though not necessarily insufficiently clear).  e.g. L24 and L26.  L24 says "before a mapped task is executed the scheduler will create N copies of the task" so here you are referring to mapped task as "parent".   and same on L26 it is mapped task as parent: "collective output of a mapped task" meaning the collection of mapped _TIs_.  but when you refer "mapped task" in the line referenced above (L228) you are referring to the TIs and that's the ambiguity i'm talking about.  not big deal just a suggestion.



-- 
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] ashb commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846455638


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.

Review Comment:
   We haven't written that yet :)



-- 
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] github-actions[bot] commented on pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #22867:
URL: https://github.com/apache/airflow/pull/22867#issuecomment-1093322491

   The PR is likely ready to be merged. No tests are needed as no important environment files, nor python files were modified by it. However, committers might decide that full test matrix is needed and add the 'full tests needed' label. Then you should rebase it to the latest main or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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] uranusjr commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846623055


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.

Review Comment:
   ```suggestion
   Currently it is only possible to map against a dict, a list, of one of those types stored in XCom as the result of a task.
   ```
   
   This seems to make more sense? Because we certainly can’t map against most types stored in XCom.



-- 
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] dstandish commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846441456


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.

Review Comment:
   ```suggestion
   Dynamic Task Mapping allows DAGs to create a varying number of tasks at runtime based upon data from upstream tasks, rather than the DAG author having to know in advance how many tasks would be needed.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.

Review Comment:
   ```suggestion
   This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable, such as task_id, queue, pool, and most other arguments to BaseOperator and have to be passed to ``partial()``
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))

Review Comment:
   this example is a bit confusing.   not clear why you have to use an xcom arg or what an xcom arg is.  could the task -> xcomarg conversion not be handled in `expand`?



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable, such as task_id, queue, pool, and most other arguments to BaseOperator and have to be passed to ``partial()``
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once

Review Comment:
   > how many of the mapped task can run at once
   
   some ambiguity here
   
   you refer to "mapped task" both as the "parent" task (elsewhere) and it seems as the actual generated task instances (here).  may be possible to clarify here you're talking about total mapped instances and not just num "parent" mapped tasks



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.

Review Comment:
   that seems odd... is there really no way?  perhaps that's where XComArg.map comes in? cc @uranusjr 



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the

Review Comment:
   since it's what you advertise, maybe it makes sense to actually demo an API call here e.g. with requests.  or with a sql query.



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.

Review Comment:
   ```suggestion
   As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
   ```



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the

Review Comment:
   also... is returning a generator permitted?



##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable, such as task_id, queue, pool, and most other arguments to BaseOperator and have to be passed to ``partial()``

Review Comment:
   ```suggestion
   It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator. 
   ```



-- 
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] uranusjr commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846622705


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.

Review Comment:
   ```suggestion
   It is not possible to achieve an effect similar to Python's ``zip`` function with mapped arguments.
   ```



-- 
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] eladkal commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846654233


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,258 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)) as dag:
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task.
+
+.. code-block:: python
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)) as dag:
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of ``[3, 4, 5]``.
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1, y=10)
+    # add(x=2, y=10)
+    # add(x=3, y=10)
+
+This would result in values of 11, 12, and 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's ``zip`` function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over.
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The ``make_list`` task runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the ``consumer`` task will be called four times, once with each value in the return of ``make_list``.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable and must be passed to ``partial()``, such as task_id, queue, pool, and most other arguments to BaseOperator.
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an ``XComArg`` object manually.
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once.

Review Comment:
   ```suggestion
   There are two limits that you can place on a task: 
     #. the number of mapped task instances can be created as the result of expansion.
     #. The number of the mapped task can run at once.
   ```
   
   let's use [numbering](https://sublime-and-sphinx-guide.readthedocs.io/en/latest/lists.html) for easier read.



-- 
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] jedcunningham commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846640207


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):

Review Comment:
   With the way parsing works, it's required even if it isn't explicitly "used". The DAG has to be in globals.



-- 
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] uranusjr commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846622860


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,264 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+This is the resulting DAG structure:
+
+.. image:: /img/mapping-simple-graph.png
+
+The grid view also provides visibility into your mapped tasks in the details panel:
+
+.. image:: /img/mapping-simple-grid.png
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.

Review Comment:
   I was going to suggest _The `make_list` task_.



-- 
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] jedcunningham commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846515726


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This is similar to defining your tasks in a for loop, but instead of having the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product", calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.

Review Comment:
   ```suggestion
   The ``make_list`` task runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the ``consumer`` task will be called four times, once with each value in the return of ``make_list``.
   ```



-- 
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] ashb commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846458613


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable, such as task_id, queue, pool, and most other arguments to BaseOperator and have to be passed to ``partial()``
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))

Review Comment:
   It likely could, yes. Right now I'm just going with documenting what the current behaviour is.



-- 
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] ashb commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846454355


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.

Review Comment:
   Reading this now I see both of these are very passive voice ("allows DAGs to create" -- dags don't do anything, authors do) 



-- 
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] ashb merged pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
ashb merged PR #22867:
URL: https://github.com/apache/airflow/pull/22867


-- 
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] dstandish commented on a diff in pull request #22867: Add concept doc for Dynamic Task Mapping

Posted by GitBox <gi...@apache.org>.
dstandish commented on code in PR #22867:
URL: https://github.com/apache/airflow/pull/22867#discussion_r846488139


##########
docs/apache-airflow/concepts/dynamic-task-mapping.rst:
##########
@@ -0,0 +1,256 @@
+ .. 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.
+
+====================
+Dynamic Task Mapping
+====================
+
+Dynamic Task Mapping allows a way for a workflow to create a number of tasks at runtime based upon current data, rather than the DAG author having to know in advance how many tasks would be needed.
+
+This can be thought of as defining your tasks in a for loop, but instead of having to have the DAG file fetch the data and do that itself, the scheduler can do this based on the output of a previous task. Right before a mapped task is executed the scheduler will create *n* copies of the task, one for each input.
+
+It is also possible to have a task operate on the collected output of a mapped task, commonly known as map and reduce.
+
+Simple mapping
+==============
+
+In its simplest form you can map over a list defined directly in your DAG file using the ``expand()`` function instead of calling your task directly.
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="simple_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        @task
+        def sum_it(values):
+            total = sum(values)
+            print(f"Total was {total}")
+
+        added_values = add_one.expand(x=[1, 2, 3])
+        sum_it(added_values)
+
+This will show ``Total was 9`` in the task logs when executed.
+
+.. note:: A reduce task is not required.
+
+    Although we show a "reduce" task here (``sum_it``) you don't have to have one, the mapped tasks will still be executed even if they have no downstream tasks.
+
+Repeated Mapping
+================
+
+The result of one mapped task can also be used as input to the next mapped task
+
+.. code-block:: python
+
+    from airflow import DAG
+    from airflow.decorators import task
+
+
+    with DAG(dag_id="repeated_mapping", start_date=datetime(2022, 3, 4)):
+
+        @task
+        def add_one(x: int):
+            return x + 1
+
+        first = add_one.expand(x=[1, 2, 3])
+        second = add_one.expand(x=first)
+
+This would have a result of [3, 4, 5]
+
+Constant parameters
+===================
+
+As well as passing arguments that get expanded at run-time, it is possible to pass arguments that don't change – in order to clearly differentiate between the two kinds we use different functions, ``expand()`` for mapped arguments, and ``partial()`` for unmapped ones.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.partial(y=10).expand(x=[1, 2, 3])
+    # This results in add function being expanded to
+    # add(x=1,y=10)
+    # add(x=2,y=10)
+    # add(x=3,y=10)
+
+This would result in values of 11, 12, 13.
+
+This is also useful for passing things such as connection IDs, database table names, or bucket names to tasks.
+
+Mapping over multiple parameters
+================================
+
+As well as a single parameter it is possible to pass multiple parameters to expand. This will have the effect of creating a "cross product" effect, calling the mapped task with each combination of parameters.
+
+For example:
+
+.. code-block:: python
+
+    @task
+    def add(x: int, y: int):
+        return x + y
+
+
+    added_values = add.expand(x=[2, 4, 8], y=[5, 10])
+    # This results in the add function being called with
+    # add(x=2, y=5)
+    # add(x=2, y=10)
+    # add(x=4, y=5)
+    # add(x=4, y=10)
+    # add(x=8, y=5)
+    # add(x=8, y=10)
+
+This would result in the add task being called 6 times. Please note however that the order of expansion is not guaranteed.
+
+It is not possible to achieve an effect similar to Python's zip function with mapped arguments.
+
+Task-generated Mapping
+======================
+
+Up until now the examples we've shown could all be achieved with a ``for`` loop in the DAG file, but the real power of dynamic task mapping comes from being able to have a task generate the list to iterate over. For example:
+
+.. code-block:: python
+
+    @task
+    def make_list():
+        # This can also be from an API call, checking a database, -- almost anything you like, as long as the
+        # resulting list/dictionary can be stored in the current XCom backend.
+        return [1, 2, {"a": "b"}, "str"]
+
+
+    @task
+    def consumer(arg):
+        print(repr(arg))
+
+
+    with DAG(dag_id="dynamic-map", start_date=datetime(2022, 4, 2)) as dag:
+        consumer.expand(arg=make_list())
+
+The make_list runs as a normal task and must return a list or dict (see `What data types can be expanded?`_), and then the consumer task will be called four times, once with each value in the return of make_list.
+
+Mapping with non-TaskFlow operators
+===================================
+
+It is possible to use partial and expand with classic style operators as well. Some arguments are not mappable, such as task_id, queue, pool, and most other arguments to BaseOperator and have to be passed to ``partial()``
+
+
+.. code-block:: python
+
+    BashOperator.partial(task_id="bash", do_xcom_push=False).expand(
+        bash_command=["echo 1", "echo 2"]
+    )
+
+Mapping over result of classic operators
+----------------------------------------
+
+If you want to map over the result of a classic operator you will need to create an XComArg object manually:
+
+.. code-block:: python
+
+    from airflow import XComArg
+
+    task = MyOperator(task_id="source")
+
+    downstream = MyOperator2.partial(task_id="consumer").expand(input=XComArg(task))
+
+Putting it all together
+=======================
+
+In this example you have a regular data delivery to an S3 bucket and want to apply the same processing to every file that arrives, no matter how many arrive each time.
+
+.. code-block:: python
+
+    from datetime import datetime
+
+    from airflow import DAG, XComArg
+    from airflow.decorators import task
+    from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+    from airflow.providers.amazon.aws.operators.s3 import S3ListOperator
+
+
+    with DAG(dag_id="mapped_s3", start_date=datetime(2020, 4, 7)) as dag:
+        files = S3ListOperator(
+            task_id="get_input",
+            bucket="example-bucket",
+            prefix='incoming/provider_a/{{ data_interval_start.strftime("%Y-%m-%d") }}',
+        )
+
+        @task
+        def count_lines(aws_conn_id, bucket, file):
+            hook = S3Hook(aws_conn_id=aws_conn_id)
+
+            return len(hook.read_key(file, bucket).splitlines())
+
+        @task
+        def total(lines):
+            return sum(lines)
+
+        counts = count_lines.partial(aws_conn_id="aws_default", bucket=files.bucket).expand(
+            file=XComArg(files)
+        )
+        total(lines=counts)
+
+What data types can be expanded?
+================================
+
+Currently it is only possible to map against a dict, a list, or one of those types stored in XCom as the result of a task.
+
+If an upstream task returns an unmappable type, the mapped task will fail at run-time with an ``UnmappableXComTypePushed`` exception. For instance, you can't have the upstream task return a plain string – it must be a list or a dict.
+
+Placing limits on mapped tasks
+==============================
+
+There are two limits that you can place on a task: how many mapped task instances can be created as the result of expansion, and how many of the mapped task can run at once

Review Comment:
   when i say "parent" i am referring to the mapped operator instance essentially.... which is itself never gonna be a task ... the tasks that get created are the instances that operato on each input.  and that's what you mean by `how many of the mapped task can run at once` no? like the instances -- not the "parent".  as to where, there are a lot of places where "mapped task" is taken to mean the parent e.g. L24 and L26.  L24 says "before a mapped task is executed the scheduler will create N copies of the task" so here you are referring to mapped task as "parent".   and same on L26 it is mapped task as parent: "collective output of a mapped task" meaning the collection of mapped _TIs_.  but when you refer "mapped task" in the line referenced above (L228) you are referring to the TIs and that's the ambiguity i'm talking about.  not big deal just a suggestion.



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