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

2022-08-04 Thread GitBox


uranusjr commented on code in PR #21077:
URL: https://github.com/apache/airflow/pull/21077#discussion_r938405709


##
airflow/providers/cncf/kubernetes/decorators/kubernetes.py:
##
@@ -0,0 +1,145 @@
+# 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
+
+# Set defaults for name and namespace.
+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)

Review Comment:
   Unused?



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

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

2022-08-04 Thread GitBox


uranusjr commented on code in PR #21077:
URL: https://github.com/apache/airflow/pull/21077#discussion_r938405620


##
airflow/providers/cncf/kubernetes/decorators/kubernetes.py:
##
@@ -0,0 +1,145 @@
+# 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
+
+# Set defaults for name and namespace.
+if 'name' not in kwargs:
+kwargs['name'] = f'k8s_airflow_pod_{uuid.uuid4().hex}'
+
+if 'namespace' not in kwargs:
+kwargs['namespace'] = 'default'

Review Comment:
   Nit, it’s easier to use `setdefault` for these



-- 
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-unsubscr...@airflow.apache.org

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



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

2022-08-04 Thread GitBox


uranusjr commented on code in PR #21077:
URL: https://github.com/apache/airflow/pull/21077#discussion_r938405255


##
airflow/decorators/__init__.pyi:
##
@@ -239,5 +238,119 @@ class TaskDecoratorCollection:
 :param cap_add: Include container capabilities
 """
 # [END decorator_signature]
+def kubernetes(
+self,
+*,
+multiple_outputs: Optional[bool] = None,
+namespace: Optional[str] = None,
+image: str,
+name: Optional[str] = None,
+random_name_suffix: 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,

Review Comment:
   What can go into `kwargs` here? It seems unnecessary.



-- 
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-unsubscr...@airflow.apache.org

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



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

2022-04-11 Thread GitBox


uranusjr commented on code in PR #21077:
URL: https://github.com/apache/airflow/pull/21077#discussion_r847743501


##
airflow/decorators/__init__.pyi:
##
@@ -188,8 +188,7 @@ class TaskDecoratorCollection:
 Dict will unroll to XCom values with keys as XCom keys. Defaults 
to False.
 :param use_dill: Whether to use dill or pickle for serialization
 :param python_command: Python command for executing functions, 
Default: python3
-:param image: Docker image from which to create the container.
-If image tag is omitted, "latest" will be used.
+:param image: Docker image from which to create the 
container.(Required)

Review Comment:
   ```suggestion
   :param image: Docker image from which to create the container. 
(Required)
   ```



-- 
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-unsubscr...@airflow.apache.org

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



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

2022-04-10 Thread GitBox


uranusjr commented on code in PR #21077:
URL: https://github.com/apache/airflow/pull/21077#discussion_r846958672


##
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 import AirflowException
+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:

Review Comment:
   ```python
   def __init__(self, image: str, **kwargs):
   ```
   
   Also mention `image` is required in the docstring.



##
airflow/decorators/__init__.pyi:
##
@@ -245,5 +245,121 @@ class TaskDecoratorCollection:
 :param cap_add: Include container capabilities
 """
 # [END decorator_signature]
+def kubernetes(
+self,
+*,
+python_callable: Optional[Callable] = None,

Review Comment:
   Remove this since the argument-less form is not possible. See the `docker` 
decorator for an example.



##
airflow/decorators/__init__.pyi:
##
@@ -245,5 +245,121 @@ 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,

Review Comment:
   ```suggestion
   image: str,
   ```



##
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
+# spec