You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/10 16:28:28 UTC

[GitHub] [airflow] dimberman commented on a change in pull request #8009: Airflow kubernetes cli command for generating pod templates

dimberman commented on a change in pull request #8009:
URL: https://github.com/apache/airflow/pull/8009#discussion_r502807166



##########
File path: airflow/cli/cli_parser.py
##########
@@ -1026,6 +1026,14 @@ class GroupCommand(NamedTuple):
         args=(ARG_ROLES,),
     ),
 )
+KUBERNETES_COMMANDS = (
+    ActionCommand(
+        name='pod-preview',
+        help='Generate a k8s template',

Review comment:
       ```suggestion
           help='Generate a k8s template for KubernetesPodOperator tasks',
   ```

##########
File path: airflow/cli/commands/kubernetes_command.py
##########
@@ -0,0 +1,91 @@
+#
+# 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 yaml
+from kubernetes.client import ApiClient
+
+from airflow.kubernetes import pod_generator
+from airflow.kubernetes.k8s_model import append_to_pod
+from airflow.kubernetes.pod_generator import PodGenerator
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils.cli import get_dag
+
+
+def is_a_kubernetes_pod_operator_instance(task):
+    """Return true if the object is instance of the kubernetes pod generator"""
+    return isinstance(task, KubernetesPodOperator)
+
+
+def get_kubernetes_pod_attributes(kubernetes_task_kwargs):
+    """Return a dictionoary for all the attributes of a
+       kubernetespodoperator task"""
+    all_instances = inspect.getmro(KubernetesPodOperator)
+    attrs = []
+    for instance in all_instances:
+        attrs.extend(list(inspect.signature(instance.__init__).parameters.keys()))
+
+    return {arg: value for arg, value in kubernetes_task_kwargs.items() if arg in attrs}
+
+
+def generate_pod_preview(kubernetes_tasks_args):
+    """Returns a dictionary with a kubernetes pod template for a given kubernetes
+       pod operator task"""
+    pod_generator_attrs = inspect.getfullargspec(PodGenerator).args
+    pod_args = {attr: value for attr, value in kubernetes_tasks_args.items()
+                if attr in pod_generator_attrs}
+
+    # TODO: This step should not be done there is naming conversion in all
+    # kubernetes classes args to arguments and env_vars to envs.
+    if 'args' not in pod_args:
+        pod_args['args'] = kubernetes_tasks_args.get('arguments', None)
+    if 'envs' not in pod_args:
+        pod_args['envs'] = kubernetes_tasks_args.get('env_vars', None)
+    if 'extract_xcom' not in pod_args:
+        pod_args['extract_xcom'] = kubernetes_tasks_args.get('do_xcom_push', None)
+
+    pod = pod_generator.PodGenerator(**(pod_args)).gen_pod()
+    pod = append_to_pod(pod,

Review comment:
       Why not just use this existing function in the KubernetesPodOperator?
   
   https://github.com/apache/airflow/blob/master/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py#L354




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

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