You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/02/27 20:17:50 UTC

[GitHub] [airflow] subkanthi opened a new pull request #21077: Kubernetes decorator

subkanthi opened a new pull request #21077:
URL: https://github.com/apache/airflow/pull/21077


   Decorator to execute functions in k8s using KubernetesPodOperator
   
   closes: #19135
   ```
   @task.kubernetes(image='python:3.8-slim-buster', name='k8s_test', namespace='default')
   def k8s_decorator_func():
          print("decorator func")
   ```
   
   If no parameters passed, defaults are set in kubernetes.py
   ```
           # Image, name and namespace are all required.
           if not 'image' in kwargs:
               kwargs['image'] = 'python:3.8-slim-buster'
   
           if not 'name' in kwargs:
               kwargs['name'] = f'k8s_airflow_pod_{uuid.uuid4().hex}'
   
           if not 'namespace' in kwargs:
               kwargs['namespace'] = 'default'
   ```
   
   This version sets the function as an environment variable - __PYTHON_SCRIPT__ and in the command , the logic is to retrieve the environment variable and write to '/tmp/script.py' to execute the function(very similar to Docker)
   
   1) Initially proceeded with trying to mount a local file using the hostPath, the problem is that requires Airflow also to run in the same node. For example running airflow locally and connecting to the kind cluster would need a mount.
   
   <!--
   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] josh-fell commented on a change in pull request #21077: Kubernetes decorator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #21077:
URL: https://github.com/apache/airflow/pull/21077#discussion_r830430023



##########
File path: airflow/decorators/__init__.pyi
##########
@@ -243,5 +243,123 @@ class TaskDecoratorCollection:
         :param cap_add: Include container capabilities
         """
         # [END decorator_signature]
+    def kubernetes(
+        self,
+        python_callable: Optional[Callable] = None,
+        multiple_outputs: Optional[bool] = None,
+        namespace: Optional[str] = None,
+        image: Optional[str] = None,
+        name: Optional[str] = None,
+        random_name_suffix: Optional[bool] = True,
+        cmds: Optional[List[str]] = None,
+        arguments: Optional[List[str]] = None,
+        ports: Optional[List[k8s.V1ContainerPort]] = None,
+        volume_mounts: Optional[List[k8s.V1VolumeMount]] = None,
+        volumes: Optional[List[k8s.V1Volume]] = None,
+        env_vars: Optional[List[k8s.V1EnvVar]] = None,
+        env_from: Optional[List[k8s.V1EnvFromSource]] = None,
+        secrets: Optional[List[Secret]] = None,
+        in_cluster: Optional[bool] = None,
+        cluster_context: Optional[str] = None,
+        labels: Optional[Dict] = None,
+        reattach_on_restart: bool = True,
+        startup_timeout_seconds: int = 120,
+        get_logs: bool = True,
+        image_pull_policy: Optional[str] = None,
+        annotations: Optional[Dict] = None,
+        resources: Optional[k8s.V1ResourceRequirements] = None,
+        affinity: Optional[k8s.V1Affinity] = None,
+        config_file: Optional[str] = None,
+        node_selectors: Optional[dict] = None,
+        node_selector: Optional[dict] = None,
+        image_pull_secrets: Optional[List[k8s.V1LocalObjectReference]] = None,
+        service_account_name: Optional[str] = None,
+        is_delete_operator_pod: bool = True,
+        hostnetwork: bool = False,
+        tolerations: Optional[List[k8s.V1Toleration]] = None,
+        security_context: Optional[Dict] = None,
+        dnspolicy: Optional[str] = None,
+        schedulername: Optional[str] = None,
+        full_pod_spec: Optional[k8s.V1Pod] = None,
+        init_containers: Optional[List[k8s.V1Container]] = None,
+        log_events_on_failure: bool = False,
+        do_xcom_push: bool = False,
+        pod_template_file: Optional[str] = None,
+        priority_class_name: Optional[str] = None,
+        pod_runtime_info_envs: Optional[List[PodRuntimeInfoEnv]] = None,
+        termination_grace_period: Optional[int] = None,
+        configmaps: Optional[List[str]] = None,
+        **kwargs,
+    ) -> TaskDecorator:
+        """Wraps a function to be executed on a k8s pod using KubernetesPodOperator
+
+        Also accepts any argument that KubernetesPodOperator will via ``kwargs``.
+
+        :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

Review comment:
       ```suggestion
           :param python_callable: Function to decorate
           :param multiple_outputs: if set, function return value will be
               unrolled to multiple XCom values. Dict will unroll to XCom values with keys as XCom keys.
               Defaults to False.
   ```
   `:type:` directives are no longer required (see #20951) and a pre-commit check was added which should eliminate these as well. Also `multiple_outputs` does not handle lists and tuples.

##########
File path: airflow/example_dags/example_kubernetes_decorator.py
##########
@@ -0,0 +1,56 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""
+This is an example dag for using the KubernetesPodOperator.
+"""
+
+from datetime import datetime
+
+from airflow import DAG
+from airflow.decorators import task
+
+with DAG(
+    dag_id='example_kubernetes_decorator',
+    schedule_interval=None,
+    start_date=datetime(2021, 1, 1),
+    tags=['example'],

Review comment:
       ```suggestion
       tags=['example'],
       catchup=False,
   ```
   Example DAGs should have `catchup=False` just to be proactive with any unforeseen headaches users _might_ have when modifying from this example.

##########
File path: airflow/decorators/__init__.pyi
##########
@@ -243,5 +243,123 @@ class TaskDecoratorCollection:
         :param cap_add: Include container capabilities
         """
         # [END decorator_signature]
+    def kubernetes(
+        self,
+        python_callable: Optional[Callable] = None,
+        multiple_outputs: Optional[bool] = None,
+        namespace: Optional[str] = None,
+        image: Optional[str] = None,
+        name: Optional[str] = None,
+        random_name_suffix: Optional[bool] = True,

Review comment:
       ```suggestion
           random_name_suffix: bool = True,
   ```

##########
File path: airflow/providers/cncf/kubernetes/decorators/kubernetes.py
##########
@@ -0,0 +1,149 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import inspect
+import os
+import pickle
+import uuid
+from tempfile import TemporaryDirectory
+from textwrap import dedent
+from typing import TYPE_CHECKING, Callable, Optional, Sequence, TypeVar
+
+from kubernetes.client import models as k8s
+
+from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+def _generate_decode_command(env_var, file):
+    # We don't need `f.close()` as the interpreter is about to exit anyway
+    return (
+        f'python -c "import base64, os;'
+        rf'x = os.environ[\"{env_var}\"];'
+        rf'f = open(\"{file}\", \"w\"); f.write(x);"'
+    )
+
+
+def _read_file_contents(filename):
+    with open(filename) as script_file:
+        return script_file.read()
+
+
+class _KubernetesDecoratedOperator(DecoratedOperator, KubernetesPodOperator):
+    """
+    Wraps a Python callable and executes in a kubernetes pod
+
+    :param python_callable: A reference to an object that is callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    """
+
+    template_fields: Sequence[str] = ('op_args', 'op_kwargs')
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects (e.g protobuf).
+    shallow_copy_attrs: Sequence[str] = ('python_callable',)
+
+    def __init__(
+        self,
+        **kwargs,
+    ) -> None:
+        self.pickling_library = pickle
+
+        # Image, name and namespace are all required.
+        if 'image' not in kwargs:
+            kwargs['image'] = 'python:3.8-slim-buster'
+
+        if 'name' not in kwargs:
+            kwargs['name'] = f'k8s_airflow_pod_{uuid.uuid4().hex}'
+
+        if 'namespace' not in kwargs:
+            kwargs['namespace'] = 'default'
+
+        super().__init__(**kwargs)
+
+    def execute(self, context: 'Context'):
+
+        with TemporaryDirectory(prefix='venv') as tmp_dir:
+            script_filename = os.path.join(tmp_dir, 'script.py')
+            py_source = self._get_python_source()
+
+            jinja_context = dict(
+                op_args=self.op_args,
+                op_kwargs=self.op_kwargs,
+                pickling_library=self.pickling_library.__name__,
+                python_callable=self.python_callable.__name__,
+                python_callable_source=py_source,
+                string_args_global=False,
+            )
+            write_python_script(
+                jinja_context=jinja_context,
+                filename=script_filename,
+                template_file='python_kubernetes_script.jinja2',
+            )
+
+            self.env_vars.append(
+                k8s.V1EnvVar(name="__PYTHON_SCRIPT", value=_read_file_contents(script_filename))
+            )
+
+            self.cmds.append("bash")
+
+            self.arguments.append("-cx")
+            self.arguments.append(
+                f'{_generate_decode_command("__PYTHON_SCRIPT", "/tmp/script.py")} && python /tmp/script.py'
+            )
+
+            return super().execute(context)
+
+    def _get_python_source(self):
+        raw_source = inspect.getsource(self.python_callable)
+        res = dedent(raw_source)
+        res = remove_task_decorator(res, "@task.kubernetes")
+        return res
+
+
+T = TypeVar("T", bound=Callable)
+
+
+def kubernetes_task(
+    python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
+) -> TaskDecorator:
+    """
+    Kubernetes operator decorator. Wraps a function to be executed in K8s using KubernetesPodOperator.
+    Also accepts any argument that DockerOperator will via ``kwargs``. Can be reused in a single DAG.
+
+    :param python_callable: Function to decorate
+    :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.

Review comment:
       ```suggestion
       :param multiple_outputs: if set, function return value will be
           unrolled to multiple XCom values. Dict will unroll to xcom values with keys as XCom keys.
           Defaults to False.
   ```
   Same comment here on types handled with `multiple_outputs`.




-- 
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] subkanthi commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/decorators/__init__.pyi
##########
@@ -126,7 +126,22 @@ class TaskDecoratorCollection:
             such as transmission a large amount of XCom to TaskAPI.
         """
     @overload
-    def virtualenv(self, python_callable: Function) -> Function: ...
+    def virtualenv(self, python_callable: F) -> F: ...

Review comment:
       Reverted change.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on pull request #21077: Kubernetes decorator

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


   ```
   airflow/example_dags/example_kubernetes_decorator.py:32:6: F811 redefinition of unused 'dag' from line 25
   airflow/providers/cncf/kubernetes/decorators/kubernetes.py:81:12: E713 test for membership should be 'not in'
   airflow/providers/cncf/kubernetes/decorators/kubernetes.py:84:12: E713 test for membership should be 'not in'
   airflow/providers/cncf/kubernetes/decorators/kubernetes.py:87:12: E713 test for membership should be 'not in'
   airflow/providers/cncf/kubernetes/decorators/kubernetes.py:90:9: F841 local variable 'kwargs_to_upstream' is assigned to but never used
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/providers/cncf/kubernetes/decorators/kubernetes.py
##########
@@ -0,0 +1,148 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import inspect
+import os
+import pickle
+import uuid
+from tempfile import TemporaryDirectory
+from textwrap import dedent
+from typing import TYPE_CHECKING, Callable, Optional, Sequence, TypeVar
+
+from kubernetes.client import models as k8s
+
+from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+def _generate_decode_command(env_var, file):
+    # We don't need `f.close()` as the interpreter is about to exit anyway
+    return (
+        f'python -c "import base64, os;'
+        rf'x = os.environ[\"{env_var}\"];'
+        rf'f = open(\"{file}\", \"w\"); f.write(x);"'
+    )
+
+
+def _read_file_contents(filename):
+    with open(filename) as script_file:
+        return script_file.read()
+
+
+class _KubernetesDecoratedOperator(DecoratedOperator, KubernetesPodOperator):
+    """
+    Wraps a Python callable and executes in a kubernetes pod
+
+    :param python_callable: A reference to an object that is callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    """
+
+    template_fields: Sequence[str] = ('op_args', 'op_kwargs')
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects (e.g protobuf).
+    shallow_copy_attrs: Sequence[str] = ('python_callable',)
+
+    def __init__(
+        self,
+        **kwargs,
+    ) -> None:
+        self.pickling_library = pickle
+
+        # Image, name and namespace are all required.
+        if 'image' not in kwargs:
+            kwargs['image'] = 'python:3.8-slim-buster'

Review comment:
       Why 3.8 and why slim? Maybe we should just make this a required argument instead of trying to guess what’s best for the user.




-- 
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] subkanthi edited a comment on pull request #21077: Kubernetes decorator

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


   > > Did you dollow the advice @subkanthi ?
   > > ```
   > > ./breeze build-image --python 3.7
   > > ```
   > 
   > Not really, sorry will try again.
   
   unfortunately that didnt help, will try to troubleshoot the problem.
   ```
   root@c7681a0b5172:/opt/airflow# mypy airflow/example_dags/example_kubernetes_decorator.py 
   Success: no issues found in 1 source file
   root@c7681a0b5172:/opt/airflow# mypy --namespace-packages airflow/example_dags
   Success: no issues found in 39 source files
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #21077: Kubernetes decorator

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


   Did you dollow the advice @subkanthi ?
   
   ```
   ./breeze build-image --python 3.7
   ```
   


-- 
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] subkanthi commented on pull request #21077: Kubernetes decorator

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


   > ```
   > airflow/example_dags/example_kubernetes_decorator.py:32:6: F811 redefinition of unused 'dag' from line 25
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:81:12: E713 test for membership should be 'not in'
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:84:12: E713 test for membership should be 'not in'
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:87:12: E713 test for membership should be 'not in'
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:90:9: F841 local variable 'kwargs_to_upstream' is assigned to but never used
   > ```
   
   Fixed these errors.


-- 
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] subkanthi commented on pull request #21077: Kubernetes decorator

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


   > Did you dollow the advice @subkanthi ?
   > 
   > ```
   > ./breeze build-image --python 3.7
   > ```
   
   Not really, sorry will try again.


-- 
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] subkanthi commented on pull request #21077: Kubernetes decorator

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


   > ```
   > airflow/example_dags/example_kubernetes_decorator.py:32:6: F811 redefinition of unused 'dag' from line 25
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:81:12: E713 test for membership should be 'not in'
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:84:12: E713 test for membership should be 'not in'
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:87:12: E713 test for membership should be 'not in'
   > airflow/providers/cncf/kubernetes/decorators/kubernetes.py:90:9: F841 local variable 'kwargs_to_upstream' is assigned to but never used
   > ```
   
   Fixed these errors.


-- 
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] subkanthi closed pull request #21077: Kubernetes decorator

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


   


-- 
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] subkanthi commented on pull request #21077: Kubernetes decorator

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


   Cant reproduce the mypy errors locally,  thoughts @uranusjr 
   ```Run mypy.................................................................................
   
   The CI image for Python 3.7 might be outdated
   
   Please run this command at earliest convenience: 
   
   ./breeze build-image --python 3.7
   
   
   
   Passed```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/decorators/__init__.pyi
##########
@@ -251,5 +251,120 @@ class TaskDecoratorCollection:
         :param cap_add: Include container capabilities
         """
         # [END decorator_signature]
+    def kubernetes(
+        self,

Review comment:
       ```suggestion
       def kubernetes(
           self,
           *,
   ```
   
   Arguments on `KubernetesOperator` are keyword-only, so these need to be as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/decorators/__init__.pyi
##########
@@ -126,7 +126,22 @@ class TaskDecoratorCollection:
             such as transmission a large amount of XCom to TaskAPI.
         """
     @overload
-    def virtualenv(self, python_callable: Function) -> Function: ...
+    def virtualenv(self, python_callable: F) -> F: ...

Review comment:
       Why is this line changed? There is no `F`.

##########
File path: airflow/decorators/__init__.pyi
##########
@@ -126,7 +126,22 @@ class TaskDecoratorCollection:
             such as transmission a large amount of XCom to TaskAPI.
         """
     @overload
-    def virtualenv(self, python_callable: Function) -> Function: ...
+    def virtualenv(self, python_callable: F) -> F: ...
+    def kubernetes(
+        self, python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
+    ) -> TaskDecorator:
+        """Wraps a function to be executed on a k8s pod using KubernetesPodOperator
+
+        Also accepts any argument that KubernetesPodOperator will via ``kwargs``.
+
+        :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
+        """

Review comment:
       This should explicitly spell out most (or all) of the arguments on `KubernetesPodOperator`, like `docker` below.
   
   Also, can you move this to after `docker`? This better separates built-in and provider-installed decorators.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/decorators/__init__.pyi
##########
@@ -126,7 +126,25 @@ class TaskDecoratorCollection:
             such as transmission a large amount of XCom to TaskAPI.
         """
     @overload
-    def virtualenv(self, python_callable: Function) -> Function: ...
+    def virtualenv(self, python_callable: F) -> F: ...
+    @overload
+    def kubernetes(
+        python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
+    ) -> TaskDecorator:

Review comment:
       This is wrong; again, see the documentation and the Docker decorator.

##########
File path: airflow/decorators/__init__.py
##########
@@ -22,6 +22,7 @@
 from airflow.decorators.python_virtualenv import virtualenv_task
 from airflow.decorators.task_group import task_group
 from airflow.models.dag import dag
+from airflow.providers.cncf.kubernetes.decorators.kubernetes import kubernetes_task

Review comment:
       This can’t be done like this here because the user may not have the kubernetes provider installed. You need to use the provider mechanism to declare the decorator. See [development documentation](https://airflow.apache.org/docs/apache-airflow/stable/howto/create-custom-decorator.html) for a more detailed explaination, and how the `@task.docker` decorator is implemented as an example.

##########
File path: airflow/utils/python_kubernetes_script.jinja2
##########
@@ -0,0 +1,44 @@
+{#

Review comment:
       This entire file should be placed in the kubernetes 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] subkanthi closed pull request #21077: Kubernetes decorator

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


   


-- 
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] subkanthi commented on pull request #21077: Kubernetes decorator

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


   Thanks @josh-fell , updated.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/providers/cncf/kubernetes/decorators/kubernetes.py
##########
@@ -0,0 +1,161 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import inspect
+import os
+import pickle
+import uuid
+from tempfile import TemporaryDirectory
+from textwrap import dedent
+from typing import TYPE_CHECKING, Callable, Optional, Sequence, TypeVar
+
+from kubernetes.client import models as k8s
+
+from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+def _generate_decode_command(env_var, file):
+    # We don't need `f.close()` as the interpreter is about to exit anyway
+    return (
+        f'python -c "import base64, os;'
+        rf'x = os.environ[\"{env_var}\"];'
+        rf'f = open(\"{file}\", \"w\"); f.write(x);"'
+    )
+
+
+def _read_file_contents(filename):
+    with open(filename) as script_file:
+        return script_file.read()
+
+
+class _KubernetesDecoratedOperator(DecoratedOperator, KubernetesPodOperator):
+    """
+    Wraps a Python callable and executes in a kubernetes pod
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function (templated)
+    :type op_kwargs: dict
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable (templated)
+    :type op_args: list
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields: Sequence[str] = ('op_args', 'op_kwargs')
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects (e.g protobuf).
+    shallow_copy_attrs: Sequence[str] = ('python_callable',)
+
+    def __init__(
+        self,
+        **kwargs,
+    ) -> None:
+        self.pickling_library = pickle
+
+        # Image, name and namespace are all required.
+        if not 'image' in kwargs:
+            kwargs['image'] = 'python:3.8-slim-buster'
+
+        if not 'name' in kwargs:
+            kwargs['name'] = f'k8s_airflow_pod_{uuid.uuid4().hex}'
+
+        if not 'namespace' in kwargs:
+            kwargs['namespace'] = 'default'
+
+        kwargs_to_upstream = {
+            "python_callable": kwargs["python_callable"],
+            "op_args": kwargs["op_args"],
+            "op_kwargs": kwargs["op_kwargs"],
+        }
+
+        super().__init__(**kwargs)
+
+    def execute(self, context: 'Context'):
+
+        with TemporaryDirectory(prefix='venv') as tmp_dir:
+            script_filename = os.path.join(tmp_dir, 'script.py')
+            py_source = self._get_python_source()
+
+            jinja_context = dict(
+                op_args=self.op_args,
+                op_kwargs=self.op_kwargs,
+                pickling_library=self.pickling_library.__name__,
+                python_callable=self.python_callable.__name__,
+                python_callable_source=py_source,
+                string_args_global=False,
+            )
+            write_python_script(
+                jinja_context=jinja_context,
+                filename=script_filename,
+                template_file='python_kubernetes_script.jinja2',
+            )
+
+            self.env_vars.append(
+                k8s.V1EnvVar(name="__PYTHON_SCRIPT", value=_read_file_contents(script_filename))
+            )
+
+            self.cmds.append("bash")

Review comment:
       I’d say let’s not allow passing the command for now, and loosen the restriction if anyone asks for it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/decorators/__init__.pyi
##########
@@ -126,7 +126,25 @@ class TaskDecoratorCollection:
             such as transmission a large amount of XCom to TaskAPI.
         """
     @overload
-    def virtualenv(self, python_callable: Function) -> Function: ...
+    def virtualenv(self, python_callable: F) -> F: ...
+    @overload
+    def kubernetes(
+        python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
+    ) -> TaskDecorator:

Review comment:
       This is wrong; again, see the documentation and the Docker decorator.

##########
File path: airflow/decorators/__init__.py
##########
@@ -22,6 +22,7 @@
 from airflow.decorators.python_virtualenv import virtualenv_task
 from airflow.decorators.task_group import task_group
 from airflow.models.dag import dag
+from airflow.providers.cncf.kubernetes.decorators.kubernetes import kubernetes_task

Review comment:
       This can’t be done like this here because the user may not have the kubernetes provider installed. You need to use the provider mechanism to declare the decorator. See [development documentation](https://airflow.apache.org/docs/apache-airflow/stable/howto/create-custom-decorator.html) for a more detailed explaination, and how the `@task.docker` decorator is implemented as an example.

##########
File path: airflow/utils/python_kubernetes_script.jinja2
##########
@@ -0,0 +1,44 @@
+{#

Review comment:
       This entire file should be placed in the kubernetes 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] subkanthi commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/providers/cncf/kubernetes/decorators/kubernetes.py
##########
@@ -0,0 +1,161 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import inspect
+import os
+import pickle
+import uuid
+from tempfile import TemporaryDirectory
+from textwrap import dedent
+from typing import TYPE_CHECKING, Callable, Optional, Sequence, TypeVar
+
+from kubernetes.client import models as k8s
+
+from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+def _generate_decode_command(env_var, file):
+    # We don't need `f.close()` as the interpreter is about to exit anyway
+    return (
+        f'python -c "import base64, os;'
+        rf'x = os.environ[\"{env_var}\"];'
+        rf'f = open(\"{file}\", \"w\"); f.write(x);"'
+    )
+
+
+def _read_file_contents(filename):
+    with open(filename) as script_file:
+        return script_file.read()
+
+
+class _KubernetesDecoratedOperator(DecoratedOperator, KubernetesPodOperator):
+    """
+    Wraps a Python callable and executes in a kubernetes pod
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function (templated)
+    :type op_kwargs: dict
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable (templated)
+    :type op_args: list
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields: Sequence[str] = ('op_args', 'op_kwargs')
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects (e.g protobuf).
+    shallow_copy_attrs: Sequence[str] = ('python_callable',)
+
+    def __init__(
+        self,
+        **kwargs,
+    ) -> None:
+        self.pickling_library = pickle
+
+        # Image, name and namespace are all required.
+        if not 'image' in kwargs:
+            kwargs['image'] = 'python:3.8-slim-buster'
+
+        if not 'name' in kwargs:
+            kwargs['name'] = f'k8s_airflow_pod_{uuid.uuid4().hex}'
+
+        if not 'namespace' in kwargs:
+            kwargs['namespace'] = 'default'
+
+        kwargs_to_upstream = {
+            "python_callable": kwargs["python_callable"],
+            "op_args": kwargs["op_args"],
+            "op_kwargs": kwargs["op_kwargs"],
+        }
+
+        super().__init__(**kwargs)
+
+    def execute(self, context: 'Context'):
+
+        with TemporaryDirectory(prefix='venv') as tmp_dir:
+            script_filename = os.path.join(tmp_dir, 'script.py')
+            py_source = self._get_python_source()
+
+            jinja_context = dict(
+                op_args=self.op_args,
+                op_kwargs=self.op_kwargs,
+                pickling_library=self.pickling_library.__name__,
+                python_callable=self.python_callable.__name__,
+                python_callable_source=py_source,
+                string_args_global=False,
+            )
+            write_python_script(
+                jinja_context=jinja_context,
+                filename=script_filename,
+                template_file='python_kubernetes_script.jinja2',
+            )
+
+            self.env_vars.append(
+                k8s.V1EnvVar(name="__PYTHON_SCRIPT", value=_read_file_contents(script_filename))
+            )
+
+            self.cmds.append("bash")

Review comment:
       Thoughts here, The user can still pass cmds and arguments, 2 options, either we limit the user not to pass command and arguments or check here if its passed and not append.




-- 
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] subkanthi commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/providers/cncf/kubernetes/decorators/kubernetes.py
##########
@@ -0,0 +1,161 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import inspect
+import os
+import pickle
+import uuid
+from tempfile import TemporaryDirectory
+from textwrap import dedent
+from typing import TYPE_CHECKING, Callable, Optional, Sequence, TypeVar
+
+from kubernetes.client import models as k8s
+
+from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils.python_virtualenv import remove_task_decorator, write_python_script
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+def _generate_decode_command(env_var, file):
+    # We don't need `f.close()` as the interpreter is about to exit anyway
+    return (
+        f'python -c "import base64, os;'
+        rf'x = os.environ[\"{env_var}\"];'
+        rf'f = open(\"{file}\", \"w\"); f.write(x);"'
+    )
+
+
+def _read_file_contents(filename):
+    with open(filename) as script_file:
+        return script_file.read()
+
+
+class _KubernetesDecoratedOperator(DecoratedOperator, KubernetesPodOperator):
+    """
+    Wraps a Python callable and executes in a kubernetes pod
+
+    :param python_callable: A reference to an object that is callable
+    :type python_callable: python callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+        in your function (templated)
+    :type op_kwargs: dict
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable (templated)
+    :type op_args: list
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    :type multiple_outputs: bool
+    """
+
+    template_fields: Sequence[str] = ('op_args', 'op_kwargs')
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects (e.g protobuf).
+    shallow_copy_attrs: Sequence[str] = ('python_callable',)
+
+    def __init__(
+        self,
+        **kwargs,
+    ) -> None:
+        self.pickling_library = pickle
+
+        # Image, name and namespace are all required.
+        if not 'image' in kwargs:
+            kwargs['image'] = 'python:3.8-slim-buster'
+
+        if not 'name' in kwargs:
+            kwargs['name'] = f'k8s_airflow_pod_{uuid.uuid4().hex}'
+
+        if not 'namespace' in kwargs:
+            kwargs['namespace'] = 'default'
+
+        kwargs_to_upstream = {
+            "python_callable": kwargs["python_callable"],
+            "op_args": kwargs["op_args"],
+            "op_kwargs": kwargs["op_kwargs"],
+        }
+
+        super().__init__(**kwargs)
+
+    def execute(self, context: 'Context'):
+
+        with TemporaryDirectory(prefix='venv') as tmp_dir:
+            script_filename = os.path.join(tmp_dir, 'script.py')
+            py_source = self._get_python_source()
+
+            jinja_context = dict(
+                op_args=self.op_args,
+                op_kwargs=self.op_kwargs,
+                pickling_library=self.pickling_library.__name__,
+                python_callable=self.python_callable.__name__,
+                python_callable_source=py_source,
+                string_args_global=False,
+            )
+            write_python_script(
+                jinja_context=jinja_context,
+                filename=script_filename,
+                template_file='python_kubernetes_script.jinja2',
+            )
+
+            self.env_vars.append(
+                k8s.V1EnvVar(name="__PYTHON_SCRIPT", value=_read_file_contents(script_filename))
+            )
+
+            self.cmds.append("bash")

Review comment:
       Thoughts here, The user can still pass cmds and arguments, 2 options, either we limit the user not to pass command and arguments or check here if its passed and not append.




-- 
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] subkanthi commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/utils/python_kubernetes_script.jinja2
##########
@@ -0,0 +1,44 @@
+{#

Review comment:
       currently this template is loaded using the
   `write_python_script` function in python_virtualenv.py, it shared by virtualenv and docker decorators. Thats why the jinja template needs to be in the same folder. 
   Ideally we should move this function out of python_virtualenv.

##########
File path: airflow/decorators/__init__.pyi
##########
@@ -126,7 +126,25 @@ class TaskDecoratorCollection:
             such as transmission a large amount of XCom to TaskAPI.
         """
     @overload
-    def virtualenv(self, python_callable: Function) -> Function: ...
+    def virtualenv(self, python_callable: F) -> F: ...
+    @overload
+    def kubernetes(
+        python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
+    ) -> TaskDecorator:

Review comment:
       Moved, thanks
   

##########
File path: airflow/decorators/__init__.py
##########
@@ -22,6 +22,7 @@
 from airflow.decorators.python_virtualenv import virtualenv_task
 from airflow.decorators.task_group import task_group
 from airflow.models.dag import dag
+from airflow.providers.cncf.kubernetes.decorators.kubernetes import kubernetes_task

Review comment:
       Removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #21077: Kubernetes decorator

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


   @uranusjr  - looks ready for re-review :)


-- 
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] subkanthi commented on a change in pull request #21077: Kubernetes decorator

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



##########
File path: airflow/decorators/__init__.pyi
##########
@@ -126,7 +126,22 @@ class TaskDecoratorCollection:
             such as transmission a large amount of XCom to TaskAPI.
         """
     @overload
-    def virtualenv(self, python_callable: Function) -> Function: ...
+    def virtualenv(self, python_callable: F) -> F: ...
+    def kubernetes(
+        self, python_callable: Optional[Callable] = None, multiple_outputs: Optional[bool] = None, **kwargs
+    ) -> TaskDecorator:
+        """Wraps a function to be executed on a k8s pod using KubernetesPodOperator
+
+        Also accepts any argument that KubernetesPodOperator will via ``kwargs``.
+
+        :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
+        """

Review comment:
       Added, thanks.




-- 
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] subkanthi commented on pull request #21077: Kubernetes decorator

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


   > > Did you dollow the advice @subkanthi ?
   > > ```
   > > ./breeze build-image --python 3.7
   > > ```
   > 
   > Not really, sorry will try again.
   
   unfortunately that didnt help, will try to troubleshoot the problem.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #21077: Kubernetes decorator

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


   Static checks need fixing


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