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 2020/06/01 11:22:01 UTC

[GitHub] [airflow] ashb commented on a change in pull request #8962: [AIRFLOW-8057] [AIP-31] Add @task decorator

ashb commented on a change in pull request #8962:
URL: https://github.com/apache/airflow/pull/8962#discussion_r433178827



##########
File path: airflow/operators/python.py
##########
@@ -145,6 +148,140 @@ def execute_callable(self):
         return self.python_callable(*self.op_args, **self.op_kwargs)
 
 
+class _PythonFunctionalOperator(BaseOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields = ('_op_args', '_op_kwargs')
+    ui_color = '#ffefeb'
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects(e.g protobuf).
+    shallow_copy_attrs = ('python_callable',)
+
+    @apply_defaults
+    def __init__(
+        self,
+        python_callable: Callable,
+        multiple_outputs: bool = False,
+        **kwargs
+    ) -> None:
+        self._validate_python_callable(python_callable)
+        super().__init__(**kwargs)
+        self.python_callable = python_callable
+        self.multiple_outputs = multiple_outputs
+        self._kwargs = kwargs
+        self._op_args: List[Any] = []
+        self._called = False
+        self._op_kwargs: Dict[str, Any] = {}
+
+    @staticmethod
+    def _get_unique_task_id(task_id: str, dag: Optional[DAG]) -> str:
+        dag = dag or DagContext.get_current_dag()
+        if not dag or task_id not in dag.task_ids:
+            return task_id
+        core = re.split(r'__\d+$', task_id)[0]
+        suffixes = sorted(
+            [int(re.split(r'^.+__', task_id)[1])
+             for task_id in dag.task_ids
+             if re.match(rf'^{core}__\d+$', task_id)]
+        )
+        if not suffixes:
+            return f'{core}__1'
+        return f'{core}__{suffixes[-1] + 1}'

Review comment:
       Yeah, we can use f-strings here, and manually fix it up when backporting -- it's not too much work on the release-manager/who ever does the backport.
   
   When it comes time to backport this I can run someone through the process (or better yet, I should document this process)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org