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 2021/07/27 18:14:00 UTC

[GitHub] [airflow] dimberman opened a new pull request #17270: Make decorators pluggable

dimberman opened a new pull request #17270:
URL: https://github.com/apache/airflow/pull/17270


   This PR will allow users to add custom "@task.____" decorators by adding
   to their setup.cfg files. This will make decorators seem native to
   airflow while living in provider packages.
   
   <!--
   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] potiuk commented on a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678649250



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       I really think it's just unnecessary duplication of entrypoints - which might bring a lot of confusion and brings no benefits (unless there are some other reasons I am not aware of of course) - but if make it part of "provider" capabilities, and some other separate "feature", I see no reason why we should split that off.




-- 
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 change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r677794935



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +43,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['task_decorator_connections'] if e.name == name]

Review comment:
       Entrypoint names are globally unique across all python packagesx so this should be called something like `airflow.task_decorators`
   
   (`console_scripts` is a "core" one, but ones we create should be good citizens and scope ourselves appropriately)
   
   Additionally everytime we call `metadata.entry_points()` it has to read possibly hundreds of files, so we might want to think about more caching/preloading the _entrypoint names_ (but not the code themesleves), particularly when it comes to the fact that DAG parsing happens in a subprocess -- so the loading of entrypoint info would need to be warmed somehow in the parent before forking/creating the process pool.

##########
File path: airflow/decorators/__init__.py
##########
@@ -15,15 +15,17 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from typing import Callable, Dict, Iterable, List, Optional, Union
+from typing import Callable, Optional
+
+import importlib_metadata as metadata
 
-from airflow.decorators.python import python_task
-from airflow.decorators.python_virtualenv import _virtualenv_task
-from airflow.decorators.task_group import task_group  # noqa

Review comment:
       This one is not accessibly by `@task.task_group` -- I think you were over-zealous in your removal of stuff :D 




-- 
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] dimberman closed pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
dimberman closed pull request #17270:
URL: https://github.com/apache/airflow/pull/17270


   


-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678637429



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       But we do it anyway? The commit/docs mentions that this is all about providers, so the providers will be loaded anyway at that point I believe? Or if we foresee that as not-part-of-providers, this should be a separate non-provider-related feature.




-- 
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] dimberman commented on a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r677904629



##########
File path: airflow/decorators/__init__.py
##########
@@ -15,15 +15,17 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from typing import Callable, Dict, Iterable, List, Optional, Union
+from typing import Callable, Optional
+
+import importlib_metadata as metadata
 
-from airflow.decorators.python import python_task
-from airflow.decorators.python_virtualenv import _virtualenv_task
-from airflow.decorators.task_group import task_group  # noqa

Review comment:
       What do you mean by this? I'm not sure what you mean by not accessible?




-- 
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 change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678967159



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       RIght now the plugin manager has to load everything (of a type) when it loads anything for that type -- i.e. asking for one hook/connection loads _all_ hooks. - that is the primary difference to this approach here -- we can load a single thing by name.
   
   Looking at your change it does address most of my issues with the current approach, and while we do still load all provider infos at once, that's not _too_ bad (so long as we warm the cache in the DagFileProcessorManager so that each parser process doesn't need to load it each time it parses a dag file -- which this PR currently also doesn't get right)
   
   @dimberman Can you change this PR to use a new field in the existing provider_info.yaml approach. Using that we can still have the "only load the decorator code on first access" approach, but all that should live in plugin_manager and use.
   
   Something like this as the (new) fields in provider info:
   
   ```
   ---
   package-name: apache-airflow-providers-docker
   name: Docker
   description: |
       `Docker <https://docs.docker.com/install/>`__
   
   task_decorators:
       docker: airflow.providers.docker.operators.docker.docker_decorator
   ```
   
   (and then we can use the existing `import_string` util.)




-- 
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] dimberman commented on pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
dimberman commented on pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#issuecomment-895401710


   @potiuk just realized I need to fix the documentation too. Doing that now.


-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678920687



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       I think with the #17304, there is no scenario, where having connection classes, operator extra link classes (and task_decorators)  defined in provider_info, would negatively impact performance. They are truly lazy loaded when needed




-- 
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] kaxil commented on a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r679045940



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       That's better for registry too: https://registry.astronomer.io/providers/google :) 




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678644765



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       > The entrypoint metadata is loaded, but iterating over entrypoints on this line doesn't load any python code.
   
   Neither provider.yaml if it has no cusom hooks/connections/operators defined. 




-- 
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] dimberman edited a comment on pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
dimberman edited a comment on pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#issuecomment-887937376


   @ashb unfortunately the problem with leaving them where they are is that because we are loading this stuff in the `__init__` file, it leads to an infinite loop.


-- 
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 change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r685510506



##########
File path: docs/apache-airflow/tutorial_taskflow_api.rst
##########
@@ -181,6 +181,59 @@ and even a different python version to run your function.
 This option should allow for far greater flexibility for users who wish to keep their workflows more simple
 and pythonic.
 
+Creating Custom TaskFlow Decorators
+-----------------------------------
+
+As of Airflow 2.3, users can now integrate custom decorators into their provider packages and have those decorators

Review comment:
       ```suggestion
   As of Airflow 2.2, users can now integrate custom decorators into their provider packages and have those decorators
   ```
   
   This will be in 2.2, no?




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678649250



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       I really think it's just unnecessary multiplication of entrypoints - which might bring a lot of confusion and brings no benefits (unless there are some other reasons I am not aware of of course) - but if make it part of "provider" capabilities, and not some other separate "feature", I see no reason why we should split that off.




-- 
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 change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678337490



##########
File path: airflow/decorators/__init__.py
##########
@@ -15,15 +15,17 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from typing import Callable, Dict, Iterable, List, Optional, Union
+from typing import Callable, Optional
+
+import importlib_metadata as metadata
 
-from airflow.decorators.python import python_task
-from airflow.decorators.python_virtualenv import _virtualenv_task
-from airflow.decorators.task_group import task_group  # noqa

Review comment:
       This import is just so you can do `from airflow.decorators import task_group` -- it is not used anywhere else, and `@task.task_group` isn't a thing.




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678651712



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       Also we have the docs of "How to write a custom provider" - https://airflow.apache.org/docs/apache-airflow-providers/index.html#custom-provider-packages - if you try to fit that in this documentation (which we should if we make it part of provider's capabilities) you will see yourself how big of a confusion it might introduce. 




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678649250



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       I really think it's just unnecessary multiplication of entrypoints - which might bring a lot of confusion and brings no benefits (unless there are some other reasons I am not aware of of course) - but if make it part of "provider" capabilities, and some other separate "feature", I see no reason why we should split that off.




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678644765



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       > The entrypoint metadata is loaded, but iterating over entrypoints on this line doesn't load any python code.
   
   Neither provider_info.yaml if it has no custom hooks/connections/operators defined.  If it does, it will happen anyway




-- 
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 change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r679234504



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       Yeah, Links are probably always going to need code loading, but those are relatively few-and-far between, and also could also be loaded on demand on first access 




-- 
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] dimberman commented on pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
dimberman commented on pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#issuecomment-895527786


   @potiuk Ok now it's updated PTAL thank you :)


-- 
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 #17270: Make decorators pluggable

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


   Probably worth linking to https://setuptools.readthedocs.io/en/latest/userguide/entry_point.html


-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r679136365



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       Cool. In the (near) future we can optimize it further. I think we could easily push caching one level deeper (to dictionary values) and only import hook class when particular hook/operators is used. Should be rather easy. 
   
   And I hope eventually with the modern UI will be able to completely change the approach and move most of the Hook custom behaviours to declarative way which won't require ANY code imports in Providers which would be great (thought I don't think it will be easy for extra links - that one might be not as easy to replace with declarative way).




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678634716



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       Why do we want to make a separate entrypoint rather than make it an entry in provider_info one ?  Seems like unnecessary duplication of entrypoints. We already have provider_info schema where we validate the dictionary passed there and it is fully extendable (and makes it one place to keep all details about the provider). 




-- 
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 change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678635963



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       Primarily because you then don't have to load _all_ plugins and pay the import time cost just to use one.
   
   (There is no need to validate anything here -- it's a single name and a function)




-- 
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 #17270: Make decorators pluggable

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


   > @ashb unfortunately the problem with leaving them where they are is that because we are loading this stuff in the `__init__` file, it leads to an infinite loop.
   
   Given python operator is built in, for the sake of efficiencey of parsing it might be worth _not_ having an entrypoint for `task.python` etc. anyway.


-- 
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] dimberman commented on pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
dimberman commented on pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#issuecomment-895384816


   @potiuk PTAL I think this addresses what you wanted?


-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678637429



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       But we do it anyway? The commit mentions that this is all about providers, so the providers will be loaded anyway at that point I believe? Or if we foresee that as not-part-of-providers, this should be a separate non-provider-related feature.




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678913202



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       That actually reminded me of something that I wanted to do for some time. I thought that we can further optimise loading of provider hooks and extra links (thus imports)  by splitting the initialize_providers_manager() method and only running what is needed when we access properties. 
   
   The main effect is speeding up some of the command lines : https://github.com/apache/airflow/pull/17304
   
   But there are scenarios where we shave-off couple of seconds, when for example Tasks are never accessing .get_conn() method (that one requires to import all hooks)  but for some reasons would like to access list of Providers (those this scenario is quire unlikely). 
   
   However if opens up the same pattern for task_decorators - you can add lazy task_decorator initialization in the same way and then workers that never access get_conn() but they need decorator, will not needlessly import the connections.




-- 
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] dimberman commented on pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
dimberman commented on pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#issuecomment-887937376


   @ashb unfortunately the problem with leaving them where they are is that because we are loading this stuff in the __init__ file, it leads to an infinite loop.


-- 
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 change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678637972



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       The entrypoint metadata is loaded, but iterating over entrypoints on this line doesn't load any python code.




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678639451



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       > As of Airflow 2.3, users can now integrate custom decorators into their provider packages and have those decorators
   
   You can easily add providers without any hooks/operator custom connection extensions and you will not pay the price of importing, so I think it does not change anything when it comes to performance




-- 
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 a change in pull request #17270: Make decorators pluggable

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17270:
URL: https://github.com/apache/airflow/pull/17270#discussion_r678639451



##########
File path: airflow/decorators/__init__.py
##########
@@ -41,103 +46,13 @@ def __call__(
         """
         return self.python(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
 
-    @staticmethod
-    def python(python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs):
-        """
-        Python operator decorator. Wraps a function into an Airflow operator.
-        Accepts kwargs for operator kwarg. This decorator can be reused in a single DAG.
-
-        :param python_callable: Function to decorate
-        :type python_callable: Optional[Callable]
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        """
-        return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs)
-
-    @staticmethod
-    def virtualenv(
-        python_callable: Optional[Callable] = None,
-        multiple_outputs: Optional[bool] = None,
-        requirements: Optional[Iterable[str]] = None,
-        python_version: Optional[Union[str, int, float]] = None,
-        use_dill: bool = False,
-        system_site_packages: bool = True,
-        string_args: Optional[Iterable[str]] = None,
-        templates_dict: Optional[Dict] = None,
-        templates_exts: Optional[List[str]] = None,
-        **kwargs,
-    ):
-        """
-        Allows one to run a function in a virtualenv that is
-        created and destroyed automatically (with certain caveats).
-
-        The function must be defined using def, and not be
-        part of a class. All imports must happen inside the function
-        and no variables outside of the scope may be referenced. A global scope
-        variable named virtualenv_string_args will be available (populated by
-        string_args). In addition, one can pass stuff through op_args and op_kwargs, and one
-        can use a return value.
-        Note that if your virtualenv runs in a different Python major version than Airflow,
-        you cannot use return values, op_args, op_kwargs, or use any macros that are being provided to
-        Airflow through plugins. You can use string_args though.
-
-        .. seealso::
-            For more information on how to use this operator, take a look at the guide:
-            :ref:`howto/operator:PythonVirtualenvOperator`
-
-        :param python_callable: A python function with no references to outside variables,
-            defined with def, which will be run in a virtualenv
-        :type python_callable: function
-        :param multiple_outputs: if set, function return value will be
-            unrolled to multiple XCom values. List/Tuples will unroll to xcom values
-            with index as key. Dict will unroll to xcom values with keys as XCom keys.
-            Defaults to False.
-        :type multiple_outputs: bool
-        :param requirements: A list of requirements as specified in a pip install command
-        :type requirements: list[str]
-        :param python_version: The Python version to run the virtualenv with. Note that
-            both 2 and 2.7 are acceptable forms.
-        :type python_version: Optional[Union[str, int, float]]
-        :param use_dill: Whether to use dill to serialize
-            the args and result (pickle is default). This allow more complex types
-            but requires you to include dill in your requirements.
-        :type use_dill: bool
-        :param system_site_packages: Whether to include
-            system_site_packages in your virtualenv.
-            See virtualenv documentation for more information.
-        :type system_site_packages: bool
-        :param op_args: A list of positional arguments to pass to python_callable.
-        :type op_args: list
-        :param op_kwargs: A dict of keyword arguments to pass to python_callable.
-        :type op_kwargs: dict
-        :param string_args: Strings that are present in the global var virtualenv_string_args,
-            available to python_callable at runtime as a list[str]. Note that args are split
-            by newline.
-        :type string_args: list[str]
-        :param templates_dict: a dictionary where the values are templates that
-            will get templated by the Airflow engine sometime between
-            ``__init__`` and ``execute`` takes place and are made available
-            in your callable's context after the template has been applied
-        :type templates_dict: dict of str
-        :param templates_exts: a list of file extensions to resolve while
-            processing templated fields, for examples ``['.sql', '.hql']``
-        :type templates_exts: list[str]
-        """
-        return _virtualenv_task(
-            python_callable=python_callable,
-            multiple_outputs=multiple_outputs,
-            requirements=requirements,
-            python_version=python_version,
-            use_dill=use_dill,
-            system_site_packages=system_site_packages,
-            string_args=string_args,
-            templates_dict=templates_dict,
-            templates_exts=templates_exts,
-            **kwargs,
-        )
+    def __getattr__(self, name):
+        if self.store.get(name, None):
+            return self.store[name]
+        connections = [e for e in metadata.entry_points()['airflow.task_decorators'] if e.name == name]

Review comment:
       > As of Airflow 2.3, users can now integrate custom decorators into their provider packages and have those decorators
   
   You can easily add providers without any hooks/operator custom connection extensions and you will not pay the price of importing, so I think it does not change anything when it comes to performance, The importing in Providers manager happens only when you have some hooks/operators  defined there, If you already have them, you save nothing as they will be imported. If you do not have them, you gain/loose nothing again as nothing will be imported anyway.




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