You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by fo...@apache.org on 2018/05/02 13:44:24 UTC

incubator-airflow git commit: [AIRFLOW-2394] default cmds and arguments in kubernetes operator

Repository: incubator-airflow
Updated Branches:
  refs/heads/master 96d00da5b -> 12ab796b1


[AIRFLOW-2394] default cmds and arguments in kubernetes operator

Commands aand arguments to docker image in kubernetes operator

Closes #3289 from ese/k8soperator


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/12ab796b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/12ab796b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/12ab796b

Branch: refs/heads/master
Commit: 12ab796b11c001f5cc7c5bd294616200b4159dea
Parents: 96d00da
Author: Sergio Ballesteros <sn...@locolandia.net>
Authored: Wed May 2 15:43:40 2018 +0200
Committer: Fokko Driesprong <fo...@godatadriven.com>
Committed: Wed May 2 15:43:51 2018 +0200

----------------------------------------------------------------------
 airflow/contrib/kubernetes/pod_generator.py     |  6 ++---
 .../operators/kubernetes_pod_operator.py        | 23 ++++++++++-------
 docs/kubernetes.rst                             | 26 +++++++++-----------
 3 files changed, 28 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/12ab796b/airflow/contrib/kubernetes/pod_generator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/kubernetes/pod_generator.py b/airflow/contrib/kubernetes/pod_generator.py
index d75ba22..78d3926 100644
--- a/airflow/contrib/kubernetes/pod_generator.py
+++ b/airflow/contrib/kubernetes/pod_generator.py
@@ -26,11 +26,9 @@ class PodGenerator:
 
     def __init__(self, kube_config=None):
         self.kube_config = kube_config
-        self.env_vars = {}
         self.volumes = []
         self.volume_mounts = []
         self.init_containers = []
-        self.secrets = []
 
     def add_init_container(self,
                            name,
@@ -129,8 +127,8 @@ class PodGenerator:
             cmds=cmds,
             args=arguments,
             labels=labels,
-            envs=self.env_vars,
-            secrets={},
+            envs={},
+            secrets=[],
             # service_account_name=self.kube_config.worker_service_account_name,
             # image_pull_secrets=self.kube_config.image_pull_secrets,
             init_containers=worker_init_container_spec,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/12ab796b/airflow/contrib/operators/kubernetes_pod_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/kubernetes_pod_operator.py b/airflow/contrib/operators/kubernetes_pod_operator.py
index 06c0c5a..32ad582 100644
--- a/airflow/contrib/operators/kubernetes_pod_operator.py
+++ b/airflow/contrib/operators/kubernetes_pod_operator.py
@@ -30,11 +30,13 @@ class KubernetesPodOperator(BaseOperator):
     """
     Execute a task in a Kubernetes Pod
 
-    :param image: Docker image name
+    :param image: Docker image you wish to launch. Defaults to dockerhub.io,
+        but fully qualified URLS will point to custom repositories
     :type image: str
-    :param: namespace: namespace name where run the Pod
+    :param: namespace: the namespace to run within kubernetes
     :type: namespace: str
-    :param cmds: entrypoint of the container
+    :param cmds: entrypoint of the container.
+        The docker images's entrypoint is used if this is not provide.
     :type cmds: list
     :param arguments: arguments of to the entrypoint.
         The docker image's CMD is used if this is not provided.
@@ -43,15 +45,18 @@ class KubernetesPodOperator(BaseOperator):
     :type labels: dict
     :param startup_timeout_seconds: timeout in seconds to startup the pod
     :type startup_timeout_seconds: int
-    :param name: name for the pod
+    :param name: name of the task you want to run,
+        will be used to generate a pod id
     :type name: str
     :param env_vars: Environment variables initialized in the container
     :type env_vars: dict
-    :param secrets: Secrets to attach to the container
+    :param secrets: Kubernetes secrets to inject in the container,
+        They can be exposed as environment vars or files in a volume.
     :type secrets: list
     :param in_cluster: run kubernetes client with in_cluster configuration
     :type in_cluster: bool
     :param get_logs: get the stdout of the container as logs of the tasks
+    :type get_logs: bool
     """
 
     def execute(self, context):
@@ -85,9 +90,9 @@ class KubernetesPodOperator(BaseOperator):
     def __init__(self,
                  namespace,
                  image,
-                 cmds,
-                 arguments,
                  name,
+                 cmds=None,
+                 arguments=None,
                  env_vars=None,
                  secrets=None,
                  in_cluster=False,
@@ -99,8 +104,8 @@ class KubernetesPodOperator(BaseOperator):
         super(KubernetesPodOperator, self).__init__(*args, **kwargs)
         self.image = image
         self.namespace = namespace
-        self.cmds = cmds
-        self.arguments = arguments
+        self.cmds = cmds or []
+        self.arguments = arguments or []
         self.labels = labels or {}
         self.startup_timeout_seconds = startup_timeout_seconds
         self.name = name

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/12ab796b/docs/kubernetes.rst
----------------------------------------------------------------------
diff --git a/docs/kubernetes.rst b/docs/kubernetes.rst
index 1dd77cb..4b83fc0 100644
--- a/docs/kubernetes.rst
+++ b/docs/kubernetes.rst
@@ -1,5 +1,5 @@
 Kubernetes Executor
-===================
+^^^^^^^^^^^^^^^^^^^
 
 The kubernetes executor is introduced in Apache Airflow 1.10.0. The Kubernetes executor will create a new pod for every task instance.
 
@@ -9,31 +9,29 @@ Example helm charts are available at `scripts/ci/kubernetes/kube/{airflow,volume
 
 
 Kubernetes Operator
-===================
+^^^^^^^^^^^^^^^^^^^
 
+.. code:: python
 
+    from airflow.contrib.operators import KubernetesOperator
+    from airflow.contrib.operators.kubernetes_pod_operator import KubernetesPodOperator
+    from airflow.contrib.kubernetes.secret import Secret
 
-.. code:: python
+    secret_file = Secret('volume', '/etc/sql_conn', 'airflow-secrets', 'sql_alchemy_conn')
+    secret_env  = Secret('env', 'SQL_CONN', 'airflow-secrets', 'sql_alchemy_conn')
 
-    from airflow.comtrib.operators import KubernetesOperator
     k = KubernetesPodOperator(namespace='default',
                               image="ubuntu:16.04",
                               cmds=["bash", "-cx"],
                               arguments=["echo", "10"],
                               labels={"foo": "bar"},
+                              secrets=[secret_file,secret_env]
                               name="test",
                               task_id="task"
                               )
 
 
+.. autoclass:: airflow.contrib.operators.kubernetes_pod_operator.KubernetesPodOperator
+
+.. autoclass:: airflow.contrib.operators.secret.Secret
 
-=================================   ====================================
-Variable                            Description
-=================================   ====================================
-``@namespace``                      The namespace is your isolated work environment within kubernetes
-``@image``                          docker image you wish to launch. Defaults to dockerhub.io, but fully qualified URLS will point to custom repositories
-``@cmds``                           To start a task in a docker image, we need to tell it what to do. the cmds array is the space seperated bash command that will define the task completed by the container
-``arguments``                       arguments for your bash command
-``@labels``                         Labels are an important element of launching kubernetes pods, as it tells kubernetes what pods a service can route to. For example, if you launch 5 postgres pods with the label  {'postgres':'foo'} and create a postgres service with the same label, kubernetes will know that any time that service is queried, it can pick any of those 5 postgres instances as the endpoint for that service.
-``@name``                           name of the task you want to run, will be used to generate a pod id
-=================================   ====================================