You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "hussein-awala (via GitHub)" <gi...@apache.org> on 2023/02/25 12:46:06 UTC

[GitHub] [airflow] hussein-awala opened a new pull request, #29762: Add an example for dynamic task mapping with non-TaskFlow operator

hussein-awala opened a new pull request, #29762:
URL: https://github.com/apache/airflow/pull/29762

   closes: #29754
   
   ---
   Add a full example for dynamic task mapping with non-TaskFlow operator and include it in dynamic task mapping documentation.
   


-- 
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] hussein-awala commented on pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala commented on PR #29762:
URL: https://github.com/apache/airflow/pull/29762#issuecomment-1454902445

   @josh-fell @potiuk  The test `test_deserialization_across_process` failes because of [this assertion](https://github.com/apache/airflow/blob/main/tests/serialization/test_dag_serialization.py#L619), where in  serialized_partial_kwargs, python_callable is a string:
   ```
   'def add_one(x: int):\n    return x + 1\n'
   ```
   but in the original_partial_kwargs it's a function.
   
   Is it supposed to keep the function as it is in the serialized mapped task or it's bug in test?


-- 
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] matthewblock commented on a diff in pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "matthewblock (via GitHub)" <gi...@apache.org>.
matthewblock commented on code in PR #29762:
URL: https://github.com/apache/airflow/pull/29762#discussion_r1118927475


##########
docs/apache-airflow/authoring-and-scheduling/dynamic-task-mapping.rst:
##########
@@ -177,9 +177,8 @@ 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:
   Maybe explicitly mention something about the example `sum_it` being a "reduce" 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] potiuk commented on pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #29762:
URL: https://github.com/apache/airflow/pull/29762#issuecomment-1455053390

   > Is it supposed to keep the function as it is in the serialized mapped task or it's bug in test?
   
   This is a very good question @uranusjr ?


-- 
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 pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on PR #29762:
URL: https://github.com/apache/airflow/pull/29762#issuecomment-1568597725

   Sounds good!


-- 
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] hussein-awala commented on pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala commented on PR #29762:
URL: https://github.com/apache/airflow/pull/29762#issuecomment-1445183541

   It seems like the serialized task from `Mapped(_PythonDecoratedOperator)` has a field `_operator_name` which we use in [this test](https://github.com/apache/airflow/blob/main/tests/serialization/test_dag_serialization.py#L608-L608), but the serialized task from `Mapped(PythonOperator)` does not. I will check if it's a bug in the serialization code or a problem in the test.


-- 
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 #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #29762:
URL: https://github.com/apache/airflow/pull/29762#discussion_r1210237270


##########
airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py:
##########
@@ -0,0 +1,56 @@
+#
+# 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.
+"""Example DAG demonstrating the usage of dynamic task mapping with non-TaskFlow operators."""
+from __future__ import annotations
+
+from datetime import datetime
+
+from airflow import DAG
+from airflow.operators.python import PythonOperator
+
+
+def add_one(x: int):
+    """A callable used in the mapped tasks to add one to the input."""
+    return x + 1
+
+
+def sum_it(values):
+    """A callable used in the final task to aggregate (reduce) the mapped tasks results"""
+    total = sum(values)
+    print(f"Total was {total}")
+
+
+with DAG(
+    dag_id="example_dynamic_task_mapping_with_no_taskflow_operators",
+    start_date=datetime(2022, 3, 4),
+    catchup=False,
+) as dag:

Review Comment:
   ```suggestion
   ):
   ```
   
   This variable isn’t used anywhere fron what I can tell.



-- 
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 pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on PR #29762:
URL: https://github.com/apache/airflow/pull/29762#issuecomment-1568395314

   I wonder if it’d be worthwhile to make the example _not_ use PythonOperator—it’s not that valuable to know about using this particular operator since you really should use taskflow instead.


-- 
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] hussein-awala commented on pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala commented on PR #29762:
URL: https://github.com/apache/airflow/pull/29762#issuecomment-1564126686

   I excluded callable from comparison because currently we serialize callable as str and there is no way to restore the original callable:
   https://github.com/apache/airflow/blob/778c3f6357642d00e9023304d15bba190ae1dd46/airflow/serialization/serialized_objects.py#L446-L447
   
   If there is a need to improve this serialization, then this should be done in a separate PR, IMO there is no problem to merge this one with this condition.


-- 
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] josh-fell commented on a diff in pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "josh-fell (via GitHub)" <gi...@apache.org>.
josh-fell commented on code in PR #29762:
URL: https://github.com/apache/airflow/pull/29762#discussion_r1118931884


##########
airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+"""Example DAG demonstrating the usage of dynamic task mapping with non-TaskFlow operators."""
+from __future__ import annotations
+
+from datetime import datetime
+
+from airflow import DAG
+from airflow.operators.python import PythonOperator
+
+
+def add_one(x: int):
+    return x + 1
+
+
+def sum_it(values):
+    total = sum(values)
+    print(f"Total was {total}")
+
+
+with DAG(
+    dag_id="example_dynamic_task_mapping_with_no_taskflow_operators", start_date=datetime(2022, 3, 4)

Review Comment:
   ```suggestion
       dag_id="example_dynamic_task_mapping_with_no_taskflow_operators",
       start_date=datetime(2022, 3, 4),
       catchup=False,
   ```
   We should add `catchup=False` in the examples just in case users copy/paste this DAG into their environment as-is, and don't get surprised by all of the DAG runs that spawn.



##########
airflow/example_dags/example_dynamic_task_mapping_with_no_taskflow_operators.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+"""Example DAG demonstrating the usage of dynamic task mapping with non-TaskFlow operators."""
+from __future__ import annotations
+
+from datetime import datetime
+
+from airflow import DAG
+from airflow.operators.python import PythonOperator
+
+
+def add_one(x: int):
+    return x + 1
+
+
+def sum_it(values):
+    total = sum(values)
+    print(f"Total was {total}")
+
+
+with DAG(
+    dag_id="example_dynamic_task_mapping_with_no_taskflow_operators", start_date=datetime(2022, 3, 4)
+) as dag:
+
+    add_one_task = PythonOperator.partial(task_id="add_one", python_callable=add_one,).expand(

Review Comment:
   ```suggestion
       add_one_task = PythonOperator.partial(task_id="add_one", python_callable=add_one).expand(
   ```



-- 
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] hussein-awala merged pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala merged PR #29762:
URL: https://github.com/apache/airflow/pull/29762


-- 
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] hussein-awala commented on pull request #29762: Add an example for dynamic task mapping with non-TaskFlow operator

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala commented on PR #29762:
URL: https://github.com/apache/airflow/pull/29762#issuecomment-1568576556

   > I wonder if it’d be worthwhile to make the example not use PythonOperator—it’s not that valuable to know about using this particular operator since you really should use taskflow instead.
   
   @uranusjr I'd say that's a good idea, I can use this one:
   ```python
   """Example DAG demonstrating the usage of dynamic task mapping with non-TaskFlow operators."""
   from __future__ import annotations
   
   from datetime import datetime
   
   from airflow import DAG
   from airflow.models import BaseOperator
   
   
   class AddOneOperator(BaseOperator):
       """A custom operator that adds one to the input."""
   
       def __init__(self, input, **kwargs):
           super().__init__(**kwargs)
           self.input = input
   
       def execute(self, context):
           return self.input + 1
   
   
   class SumItOperator(BaseOperator):
       """A custom operator that sums the input."""
   
       def __init__(self, values, **kwargs):
           super().__init__(**kwargs)
           self.values = values
   
       def execute(self, context):
           total = sum(self.values)
           print(f"Total was {total}")
           return total
   
   
   with DAG(
       dag_id="example_dynamic_task_mapping_with_no_taskflow_operators",
       start_date=datetime(2022, 3, 4),
       catchup=False,
   ):
       # map the task to a list of values
       add_one_task = AddOneOperator.partial(task_id="add_one").expand(input=[1, 2, 3])
   
       # aggregate (reduce) the mapped tasks results
       sum_it_task = SumItOperator(task_id="sum_it", values=add_one_task.output)
   ```
   WDYT?


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