From 4e57a6862560ea21fc45a8a36f1f3f1969a94107 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Sat, 13 Aug 2022 00:42:59 -0500 Subject: [PATCH 01/43] initial job tasks --- docs/credentials.md | 1 + docs/deployment.md | 1 + docs/pod.md | 1 + docs/secrets.md | 1 + docs/service.md | 1 + docs/utilities.md | 1 + prefect_kubernetes/credentials.py | 47 +++++ prefect_kubernetes/deployment.py | 0 prefect_kubernetes/job.py | 338 ++++++++++++++++++++++++++++++ prefect_kubernetes/pod.py | 0 prefect_kubernetes/secrets.py | 0 prefect_kubernetes/service.py | 0 prefect_kubernetes/utilities.py | 109 ++++++++++ 13 files changed, 500 insertions(+) create mode 100644 docs/credentials.md create mode 100644 docs/deployment.md create mode 100644 docs/pod.md create mode 100644 docs/secrets.md create mode 100644 docs/service.md create mode 100644 docs/utilities.md create mode 100644 prefect_kubernetes/credentials.py create mode 100644 prefect_kubernetes/deployment.py create mode 100644 prefect_kubernetes/job.py create mode 100644 prefect_kubernetes/pod.py create mode 100644 prefect_kubernetes/secrets.py create mode 100644 prefect_kubernetes/service.py create mode 100644 prefect_kubernetes/utilities.py diff --git a/docs/credentials.md b/docs/credentials.md new file mode 100644 index 0000000..7078caa --- /dev/null +++ b/docs/credentials.md @@ -0,0 +1 @@ +:::prefect_kubernetes.credentials \ No newline at end of file diff --git a/docs/deployment.md b/docs/deployment.md new file mode 100644 index 0000000..d327e07 --- /dev/null +++ b/docs/deployment.md @@ -0,0 +1 @@ +:::prefect_kubernetes.deployment \ No newline at end of file diff --git a/docs/pod.md b/docs/pod.md new file mode 100644 index 0000000..5808b21 --- /dev/null +++ b/docs/pod.md @@ -0,0 +1 @@ +:::prefect_kubernetes.pod \ No newline at end of file diff --git a/docs/secrets.md b/docs/secrets.md new file mode 100644 index 0000000..18a73fc --- /dev/null +++ b/docs/secrets.md @@ -0,0 +1 @@ +:::prefect_kubernetes.secrets \ No newline at end of file diff --git a/docs/service.md b/docs/service.md new file mode 100644 index 0000000..904ab4b --- /dev/null +++ b/docs/service.md @@ -0,0 +1 @@ +:::prefect_kubernetes.service \ No newline at end of file diff --git a/docs/utilities.md b/docs/utilities.md new file mode 100644 index 0000000..7c5645f --- /dev/null +++ b/docs/utilities.md @@ -0,0 +1 @@ +:::prefect_kubernetes.utilities \ No newline at end of file diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py new file mode 100644 index 0000000..af4c0b3 --- /dev/null +++ b/prefect_kubernetes/credentials.py @@ -0,0 +1,47 @@ +from prefect.blocks.core import Block +from prefect_kubernetes.utilities import KubernetesClient, get_kubernetes_client +from pydantic import SecretStr + +class KubernetesApiKey(Block): + """Credentials block for API client generation across prefect-kubernetes tasks and flows. + + Args: + api_key (SecretStr): API key to authenticate with the Kubernetes API. + + Examples: + Load a stored kubernetes API key: + ```python + from prefect_kubernetes import KubernetesApiKey + + kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") + ``` + + Create a kubernetes API client from KubernetesApiKey and inferred cluster configuration: + ```python + from prefect_kubernetes import KubernetesApiKey + from prefect_kubernetes.utilities import get_kubernetes_client + + kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") + kubernetes_api_client = get_kubernetes_client("pod", kubernetes_api_key) + ``` + + Create a namespaced kubernetes job: + ```python + from prefect_kubernetes import KubernetesApiKey + from prefect_kubernetes.job import create_namespaced_job + + kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") + + create_namespaced_job( + namespace="default", body={"Marvin": "42"}, **kube_kwargs + ) + ``` + """ + + _block_type_name = "Kubernetes Api Key" + _logo_url = "https://kubernetes-security.info/assets/img/logo.png?h=250" # noqa + + api_key: SecretStr + + def get_client(self, resource: str) -> KubernetesClient: + return get_kubernetes_client(resource, kubernetes_api_key=self.api_key) \ No newline at end of file diff --git a/prefect_kubernetes/deployment.py b/prefect_kubernetes/deployment.py new file mode 100644 index 0000000..e69de29 diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py new file mode 100644 index 0000000..b8f2b5d --- /dev/null +++ b/prefect_kubernetes/job.py @@ -0,0 +1,338 @@ +from typing import Dict, Optional + +from concurrent.futures import ThreadPoolExecutor +from kubernetes import client +from prefect import task +from prefect_kubernetes.credentials import KubernetesApiKey +from prefect_kubernetes.utilities import get_kubernetes_client, KubernetesClient + +@task +def create_namespaced_job( + body: Dict = None, + namespace: Optional[str] = "default", + kubernetes_api_key: Optional[KubernetesApiKey] = None, + api_client: Optional[KubernetesClient] = None, + kube_kwargs: Optional[Dict] = {} +): + """Task for creating a namespaced Kubernetes job. + + Args: + body (dict): A dictionary representation of a Kubernetes V1Job + specification. Defaults to None. + namespace (str, optional): The Kubernetes namespace to create this job in, + defaults to the `default` namespace. + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to create a job. Defaults to None. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. + + Raises: + ValueError: if `body` is `None` + """ + if not body: + raise ValueError("A dictionary representing a V1Job must be provided.") + + api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + + body = {**body, **(body or {})} + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + + api_client.create_namespaced_job(namespace=namespace, body=body, **kube_kwargs) + +@task +def delete_namespaced_job( + job_name: str, + namespace: Optional[str] = "default", + kubernetes_api_key: Optional[KubernetesApiKey] = None, + api_client: Optional[KubernetesClient] = None, + kube_kwargs: Optional[Dict] = {} +): + """Task for deleting a namespaced Kubernetes job. + + Args: + job_name (str): The name of a job to delete. Defaults to None. + namespace (str, optional): The Kubernetes namespace to delete this job in, + defaults to the `default` namespace. + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + holding a Kubernetes API Key. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to delete a job. Defaults to None. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + + Raises: + ValueError: if `job_name` is `None` + """ + if not job_name: + raise ValueError("The name of a Kubernetes job must be provided.") + + api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + delete_option_kwargs = delete_option_kwargs or {} + + api_client.delete_namespaced_job( + name=job_name, + namespace=namespace, + body=client.V1DeleteOptions(**delete_option_kwargs), + **kube_kwargs, + ) + +@task +def list_namespaced_job( + namespace: Optional[str] = "default", + kubernetes_api_key: Optional[KubernetesApiKey] = None, + api_client: Optional[KubernetesClient] = None, + kube_kwargs: Optional[Dict] = {} +): + """Task for listing namespaced Kubernetes jobs. + + Args: + namespace (str, optional): The Kubernetes namespace to list jobs from, + defaults to the `default` namespace. + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + holding a Kubernetes API Key. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to delete a job. Defaults to None. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + """ + api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + + api_client.list_namespaced_job( + namespace=namespace, + **kube_kwargs, + ) + +@task +def patch_namespaced_job( + job_name: str = None, + namespace: Optional[str] = "default", + kubernetes_api_key: Optional[KubernetesApiKey] = None, + api_client: Optional[KubernetesClient] = None, + kube_kwargs: Optional[Dict] = {} +): + """Task for deleting a namespaced Kubernetes job. + + Args: + job_name (str): The name of a job to patch. Defaults to None. + namespace (str, optional): The Kubernetes namespace to patch this job in, + defaults to the `default` namespace. + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + holding a Kubernetes API Key. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to patch a job. Defaults to None. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + + Raises: + ValueError: if `job_name` is `None` + """ + if not body: + raise ValueError( + "A dictionary representing a V1Job patch must be provided." + ) + + if not job_name: + raise ValueError("The name of a Kubernetes job must be provided.") + + api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + + body = {**body, **(body or {})} + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + + api_client.patch_namespaced_job( + name=job_name, namespace=namespace, body=body, **kube_kwargs + ) + +@task +def read_namespaced_job( + job_name: str = None, + namespace: Optional[str] = "default", + kubernetes_api_key: Optional[KubernetesApiKey] = None, + api_client: Optional[KubernetesClient] = None, + kube_kwargs: Optional[Dict] = {}, +): + """Task for reading a namespaced kubernetes job. + + Args: + job_name (str): The name of a job to read. Defaults to None. + namespace (str, optional): The Kubernetes namespace to read this job in, + defaults to the `default` namespace. + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + holding a Kubernetes API Key. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to read a job. Defaults to None. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + + Raises: + ValueError: if `job_name` is `None` + """ + if not job_name: + raise ValueError("The name of a Kubernetes job must be provided.") + + api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + + return api_client.read_namespaced_job( + name=job_name, namespace=namespace, **kube_kwargs + ) + +@task +def replace_namespaced_job( + job_name: str = None, + namespace: Optional[str] = "default", + kubernetes_api_key: Optional[KubernetesApiKey] = None, + api_client: Optional[KubernetesClient] = None, + kube_kwargs: Optional[Dict] = {}, +): + """Task for replacing a namespaced kubernetes job. + + Args: + job_name (str): The name of a job to replace. Defaults to None. + namespace (str, optional): The Kubernetes namespace to replace this job in, + defaults to the `default` namespace. + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + holding a Kubernetes API Key. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to replace a job. Defaults to None. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + + Raises: + ValueError: if `job_name` is `None` + """ + if not job_name: + raise ValueError("The name of a Kubernetes job must be provided.") + + api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + + return api_client.replace_namespaced_job( + name=job_name, namespace=namespace, **kube_kwargs + ) + +@task +def run_namespaced_job( + job_name: str = None, + namespace: Optional[str] = "default", + kubernetes_api_key: Optional[KubernetesApiKey] = None, + kube_kwargs: Optional[Dict] = {}, + job_status_poll_interval: Optional[int] = 5, + log_level: Optional[str] = None, + delete_job_after_completion: Optional[bool] = True +): + """Task for running a namespaced kubernetes job. + + Args: + job_name (str): The name of a kubernetes job to run. Defaults to None. + namespace (str, optional): The Kubernetes namespace to run this job in, + defaults to the `default` namespace. + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + holding a Kubernetes API Key. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to run a job. Defaults to None. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + job_status_poll_interval (int, optional): The interval given in seconds + indicating how often the Kubernetes API will be requested about the status + of the job being performed, defaults to `5` seconds. + log_level (str, optional): Log level used when outputting logs from the job + should be one of `debug`, `info`, `warn`, `error`, 'critical' or `None` to + disable output completely. Defaults to `None`. + delete_job_after_completion (bool, optional): boolean value determining whether + resources related to a given job will be removed from the Kubernetes cluster + after completion, defaults to the `True` value + Raises: + ValueError: if `body` is `None` + ValueError: if `body["metadata"]["name"] is `None` + """ + if not body: + raise ValueError("A dictionary representing a V1Job must be provided.") + + # if log_level is not None and getattr(logger, log_level, None) is None: + # raise ValueError("A valid log_level must be provided.") + + body = {**body, **(body or {})} + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + + job_name = body.get("metadata", {}).get("name") + if not job_name: + raise ValueError( + "The job name must be defined in the body under the metadata key." + ) + + api_client_job = kubernetes_api_key.get_client(resource="job") + + api_client_pod = kubernetes_api_key.get_client(resource="pod") + + api_client_job.create_namespaced_job( + namespace=namespace, body=body, **kube_kwargs + ) + print(f"Job {job_name} has been created.") + + pod_log_streams = {} + + # Context is thread-local and isn't automatically copied + # to the threads spawned by ThreadPoolExecutor. + # Add an initializer which updates the thread's Context with + # values from the current Context. + # context_copy = prefect.context.copy() + + # def initialize_thread(context): + # prefect.context.update(context) + + # with ThreadPoolExecutor( + # initializer=initialize_thread, initargs=(context_copy,) + # ) as pool: + # completed = False + # while not completed: + # job = api_client_job.read_namespaced_job_status( + # name=job_name, namespace=namespace + # ) + + # if log_level is not None: + # func_log = getattr(self.logger, log_level) + + # pod_selector = ( + # f"controller-uid={job.metadata.labels['controller-uid']}" + # ) + # pods_list = api_client_pod.list_namespaced_pod( + # namespace=namespace, label_selector=pod_selector + # ) + + # for pod in pods_list.items: + # pod_name = pod.metadata.name + + # # Can't start logs when phase is pending + # if pod.status.phase == "Pending": + # continue + # if pod_name in pod_log_streams: + # continue + + # read_pod_logs = ReadNamespacedPodLogs( + # pod_name=pod_name, + # namespace=namespace, + # kubernetes_api_key=kubernetes_api_key, + # on_log_entry=lambda log: func_log(f"{pod_name}: {log}"), + # ) + + # self.logger.info(f"Started following logs for {pod_name}") + # pod_log_streams[pod_name] = pool.submit(read_pod_logs.run) + + # if job.status.active: + # time.sleep(job_status_poll_interval) + # elif job.status.failed: + # raise signals.FAIL( + # f"Job {job_name} failed, check Kubernetes pod logs for more information." + # ) + # elif job.status.succeeded: + # self.logger.info(f"Job {job_name} has been completed.") + # break + + # if delete_job_after_completion: + # api_client_job.delete_namespaced_job( + # name=job_name, + # namespace=namespace, + # body=client.V1DeleteOptions(propagation_policy="Foreground"), + # ) + # self.logger.info(f"Job {job_name} has been deleted.") \ No newline at end of file diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py new file mode 100644 index 0000000..e69de29 diff --git a/prefect_kubernetes/secrets.py b/prefect_kubernetes/secrets.py new file mode 100644 index 0000000..e69de29 diff --git a/prefect_kubernetes/service.py b/prefect_kubernetes/service.py new file mode 100644 index 0000000..e69de29 diff --git a/prefect_kubernetes/utilities.py b/prefect_kubernetes/utilities.py new file mode 100644 index 0000000..fd66ca3 --- /dev/null +++ b/prefect_kubernetes/utilities.py @@ -0,0 +1,109 @@ +import sys +from typing import Optional, Union + +from kubernetes import client, config as kube_config +from kubernetes.config.config_exception import ConfigException +from prefect.blocks.kubernetes import KubernetesClusterConfig +from prefect_kubernetes.credentials import KubernetesApiKey + +KubernetesClient = Union[client.BatchV1Api, client.CoreV1Api, client.AppsV1Api, client.ApiClient] + +K8S_CLIENTS = { + "job": client.BatchV1Api, + "pod": client.CoreV1Api, + "service": client.CoreV1Api, + "deployment": client.AppsV1Api, + "secret": client.CoreV1Api, +} + +def get_kubernetes_client( + resource: str, + kubernetes_api_key: Optional[KubernetesApiKey] = None, + kubernetes_cluster_config: Optional[KubernetesClusterConfig] = None + +) -> KubernetesClient: + """ + Utility function for loading kubernetes client object for a given resource. + It will attempt to connect to a Kubernetes cluster in three steps with + the first successful connection attempt becoming the mode of communication with a + cluster. + 1. Attempt to use a KubernetesApiKey block containing a Kubernetes API Key. If + `kubernetes_api_key` = `None` then it will attempt the next two connection + methods. + 2. Attempt in-cluster connection (will only work when running on a Pod in a cluster) + 3. Attempt out-of-cluster connection using the default location for a kube config file + In some cases connections to the kubernetes server are dropped after being idle for some time + (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in + ReadTimeoutErrors. + In order to prevent that a periodic keep-alive message can be sent to the server to keep the + connection open. + Args: + - resource (str): the name of the resource to retrieve a client for. Currently + you can use one of these values: `job`, `pod`, `service`, `deployment`, `secret` + - kubernetes_api_key (SecretStr): the value of a kubernetes api key in BearerToken format + Returns: + - KubernetesClient: an initialized and authenticated Kubernetes Client + """ + if (resource or kubernetes_api_key) and kubernetes_cluster_config: + raise ValueError("Please specify either a cluster config block or an API key to generate an API client") + + + # KubernetesClusterConfig.get_api_client() returns a k8s api client that can be used to interact with any resource type + if kubernetes_cluster_config: + return kubernetes_cluster_config.get_api_client() + + client_type = K8S_CLIENTS[resource] + + if kubernetes_api_key: + + configuration = client.Configuration() + configuration.api_key["authorization"] = kubernetes_api_key + k8s_client = client_type(client.ApiClient(configuration)) + else: + try: + print("Trying to load in-cluster configuration...") + kube_config.load_incluster_config() + except ConfigException as exc: + print("{} | Using out of cluster configuration option.".format(exc)) + print("Loading out-of-cluster configuration...") + kube_config.load_kube_config() + + k8s_client = client_type() + + # if config.cloud.agent.kubernetes_keep_alive: + # _keep_alive(client=k8s_client) + + return k8s_client + +def _keep_alive(client: KubernetesClient) -> None: + """ + Setting the keep-alive flags on the kubernetes client object. + Unfortunately neither the kubernetes library nor the urllib3 library which kubernetes is using + internally offer the functionality to enable keep-alive messages. Thus the flags are added to + be used on the underlying sockets. + Args: + - client (KubernetesClient): the kubernetes client object on which the keep-alive should be + enabled + """ + import socket + + socket_options = [(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1)] + + if hasattr(socket, "TCP_KEEPINTVL"): + socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 30)) + + if hasattr(socket, "TCP_KEEPCNT"): + socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPCNT, 6)) + + if hasattr(socket, "TCP_KEEPIDLE"): + socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 6)) + + if sys.platform == "darwin": + # TCP_KEEP_ALIVE not available on socket module in macOS, but defined in + # https://github.com/apple/darwin-xnu/blob/2ff845c2e033bd0ff64b5b6aa6063a1f8f65aa32/bsd/netinet/tcp.h#L215 + TCP_KEEP_ALIVE = 0x10 + socket_options.append((socket.IPPROTO_TCP, TCP_KEEP_ALIVE, 30)) + + client.api_client.rest_client.pool_manager.connection_pool_kw[ + "socket_options" + ] = socket_options \ No newline at end of file From ef5739c236e21bcc8ccbd0b7e2f29d858c6aca79 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 15 Aug 2022 23:24:16 -0500 Subject: [PATCH 02/43] migrated logger, added pod log for run job --- docs/exceptions.md | 1 + prefect_kubernetes/exceptions.py | 9 ++ prefect_kubernetes/job.py | 161 ++++++++++++++++--------------- prefect_kubernetes/pod.py | 62 ++++++++++++ 4 files changed, 157 insertions(+), 76 deletions(-) create mode 100644 docs/exceptions.md create mode 100644 prefect_kubernetes/exceptions.py diff --git a/docs/exceptions.md b/docs/exceptions.md new file mode 100644 index 0000000..bc23728 --- /dev/null +++ b/docs/exceptions.md @@ -0,0 +1 @@ +:::prefect_kubernetes.exceptions \ No newline at end of file diff --git a/prefect_kubernetes/exceptions.py b/prefect_kubernetes/exceptions.py new file mode 100644 index 0000000..beb9801 --- /dev/null +++ b/prefect_kubernetes/exceptions.py @@ -0,0 +1,9 @@ +class KubernetesJobDefinitionError(Exception): + """An exception to raise when a Kubernetes job definition is invalid + + """ + +class KubernetesJobFailedError(Exception): + """An exception to raise when a job orchestrated by a prefect_kubernetes task fails + + """ \ No newline at end of file diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py index b8f2b5d..583c77e 100644 --- a/prefect_kubernetes/job.py +++ b/prefect_kubernetes/job.py @@ -1,10 +1,15 @@ +import prefect.context +import time + from typing import Dict, Optional from concurrent.futures import ThreadPoolExecutor from kubernetes import client -from prefect import task +from prefect import get_run_logger, task +from prefect_kubernetes import exceptions as err from prefect_kubernetes.credentials import KubernetesApiKey -from prefect_kubernetes.utilities import get_kubernetes_client, KubernetesClient +from prefect_kubernetes.pod import read_namespaced_pod_logs +from prefect_kubernetes.utilities import KubernetesClient @task def create_namespaced_job( @@ -30,7 +35,9 @@ def create_namespaced_job( ValueError: if `body` is `None` """ if not body: - raise ValueError("A dictionary representing a V1Job must be provided.") + raise err.KubernetesJobDefinitionError( + "A dictionary representing a V1Job must be provided." + ) api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client @@ -128,8 +135,8 @@ def patch_namespaced_job( ValueError: if `job_name` is `None` """ if not body: - raise ValueError( - "A dictionary representing a V1Job patch must be provided." + raise err.KubernetesJobDefinitionError( + "A dictionary representing a V1Job must be provided." ) if not job_name: @@ -230,7 +237,8 @@ def run_namespaced_job( defaults to the `default` namespace. kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to run a job. Defaults to None. + api_client (KubernetesClient, optional): An initialized Kubernetes API client + to use to run a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. job_status_poll_interval (int, optional): The interval given in seconds @@ -243,11 +251,15 @@ def run_namespaced_job( resources related to a given job will be removed from the Kubernetes cluster after completion, defaults to the `True` value Raises: - ValueError: if `body` is `None` - ValueError: if `body["metadata"]["name"] is `None` + KubernetesJobDefinitionError: if `body` is `None` + KubernetesJobDefinitionError: if `body["metadata"]["name"] is `None` """ + logger = get_run_logger(log_level=log_level) + if not body: - raise ValueError("A dictionary representing a V1Job must be provided.") + raise err.KubernetesJobDefinitionError( + "A dictionary representing a V1Job must be provided." + ) # if log_level is not None and getattr(logger, log_level, None) is None: # raise ValueError("A valid log_level must be provided.") @@ -256,83 +268,80 @@ def run_namespaced_job( kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} job_name = body.get("metadata", {}).get("name") + if not job_name: - raise ValueError( + raise err.KubernetesJobDefinitionError( "The job name must be defined in the body under the metadata key." ) api_client_job = kubernetes_api_key.get_client(resource="job") - api_client_pod = kubernetes_api_key.get_client(resource="pod") api_client_job.create_namespaced_job( namespace=namespace, body=body, **kube_kwargs ) - print(f"Job {job_name} has been created.") + logger.info(f"Job {job_name} has been created.") pod_log_streams = {} - # Context is thread-local and isn't automatically copied - # to the threads spawned by ThreadPoolExecutor. - # Add an initializer which updates the thread's Context with - # values from the current Context. - # context_copy = prefect.context.copy() - - # def initialize_thread(context): - # prefect.context.update(context) - - # with ThreadPoolExecutor( - # initializer=initialize_thread, initargs=(context_copy,) - # ) as pool: - # completed = False - # while not completed: - # job = api_client_job.read_namespaced_job_status( - # name=job_name, namespace=namespace - # ) - - # if log_level is not None: - # func_log = getattr(self.logger, log_level) - - # pod_selector = ( - # f"controller-uid={job.metadata.labels['controller-uid']}" - # ) - # pods_list = api_client_pod.list_namespaced_pod( - # namespace=namespace, label_selector=pod_selector - # ) - - # for pod in pods_list.items: - # pod_name = pod.metadata.name - - # # Can't start logs when phase is pending - # if pod.status.phase == "Pending": - # continue - # if pod_name in pod_log_streams: - # continue - - # read_pod_logs = ReadNamespacedPodLogs( - # pod_name=pod_name, - # namespace=namespace, - # kubernetes_api_key=kubernetes_api_key, - # on_log_entry=lambda log: func_log(f"{pod_name}: {log}"), - # ) - - # self.logger.info(f"Started following logs for {pod_name}") - # pod_log_streams[pod_name] = pool.submit(read_pod_logs.run) - - # if job.status.active: - # time.sleep(job_status_poll_interval) - # elif job.status.failed: - # raise signals.FAIL( - # f"Job {job_name} failed, check Kubernetes pod logs for more information." - # ) - # elif job.status.succeeded: - # self.logger.info(f"Job {job_name} has been completed.") - # break - - # if delete_job_after_completion: - # api_client_job.delete_namespaced_job( - # name=job_name, - # namespace=namespace, - # body=client.V1DeleteOptions(propagation_policy="Foreground"), - # ) - # self.logger.info(f"Job {job_name} has been deleted.") \ No newline at end of file + # Context is thread-local and isn't automatically copied to the threads spawned by ThreadPoolExecutor. + # Add an initializer which updates the thread's Context with values from the current Context. + context_copy = prefect.context.copy() + + def initialize_thread(context): + prefect.context.update(context) + + with ThreadPoolExecutor( + initializer=initialize_thread, initargs=(context_copy,) + ) as pool: + completed = False + while not completed: + job = api_client_job.read_namespaced_job_status( + name=job_name, namespace=namespace + ) + if log_level is not None: + func_log = getattr(logger, log_level) + + pod_selector = ( + f"controller-uid={job.metadata.labels['controller-uid']}" + ) + pods_list = api_client_pod.list_namespaced_pod( + namespace=namespace, label_selector=pod_selector + ) + + for pod in pods_list.items: + pod_name = pod.metadata.name + + # Can't start logs when phase is pending + if pod.status.phase == "Pending": + continue + if pod_name in pod_log_streams: + continue + + read_pod_logs = read_namespaced_pod_logs( + pod_name=pod_name, + namespace=namespace, + kubernetes_api_key=kubernetes_api_key, + on_log_entry=lambda log: func_log(f"{pod_name}: {log}"), + ) + + logger.info(f"Started following logs for {pod_name}") + pod_log_streams[pod_name] = pool.submit(read_pod_logs.run) + + if job.status.active: + time.sleep(job_status_poll_interval) + elif job.status.failed: + raise err.KubernetesJobFailedError( + f"Job {job_name} failed, check Kubernetes pod logs for more information." + ) + elif job.status.succeeded: + logger.info(f"Job {job_name} has been completed.") + break + + if delete_job_after_completion: + api_client_job.delete_namespaced_job( + name=job_name, + namespace=namespace, + body=client.V1DeleteOptions(propagation_policy="Foreground"), + ) + logger.info(f"Job {job_name} has been deleted.") \ No newline at end of file diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index e69de29..034f5b9 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -0,0 +1,62 @@ +from typing import Callable + +from kubernetes.watch import Watch +from kubernetes.client.rest import ApiException + +from prefect import task +from prefect_kubernetes.credentials import KubernetesApiKey + +@task +def read_namespaced_pod_logs( + pod_name: str = None, + namespace: str = "default", + on_log_entry: Callable = None, + kubernetes_api_key: KubernetesApiKey = None, + container: str = None, + ) -> None: + """ + Task run method. + Args: + - pod_name (str, optional): The name of a pod to replace + - namespace (str, optional): The Kubernetes namespace to read pod logs in, + defaults to the `default` namespace + - on_log_entry (Callable, optional): If provided, will stream the pod logs + calling the callback for every line (and the task returns `None`). If not + provided, the current pod logs will be returned immediately from the task. + - kubernetes_api_key (KubernetesApiKey, optional): name of the KubernetesApiKey block + containing your Kubernetes API Key; value must be a string in BearerToken format + - container (str, optional): The name of the container to read logs from + Raises: + - ValueError: if `pod_name` is `None` + """ + if not pod_name: + raise ValueError("The name of a Kubernetes pod must be provided.") + + api_client = kubernetes_api_key.get_client(resource="pod") + + if on_log_entry is None: + return api_client.read_namespaced_pod_log( + name=pod_name, namespace=namespace, container=container + ) + + # From the kubernetes.watch documentation: + # Note that watching an API resource can expire. The method tries to + # resume automatically once from the last result, but if that last result + # is too old as well, an `ApiException` exception will be thrown with + # ``code`` 410. + while True: + try: + stream = Watch().stream( + api_client.read_namespaced_pod_log, + name=pod_name, + namespace=namespace, + container=container, + ) + + for log in stream: + on_log_entry(log) + + return + except ApiException as exception: + if exception.status != 410: + raise From 80cd96aa275e3008ef5adf4fa9bd5794c075495e Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 26 Aug 2022 12:40:19 -0500 Subject: [PATCH 03/43] add initial test coverage for job tasks --- prefect_kubernetes/credentials.py | 126 +++++++++++++++++++++++--- prefect_kubernetes/exceptions.py | 9 +- prefect_kubernetes/job.py | 142 ++++++++++++++++-------------- prefect_kubernetes/pod.py | 101 ++++++++++----------- prefect_kubernetes/utilities.py | 109 ----------------------- tests/conftest.py | 55 ++++++++++++ tests/test_job.py | 116 ++++++++++++++++++++++++ 7 files changed, 412 insertions(+), 246 deletions(-) delete mode 100644 prefect_kubernetes/utilities.py create mode 100644 tests/conftest.py create mode 100644 tests/test_job.py diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index af4c0b3..c4b616e 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -1,37 +1,56 @@ +from typing import TYPE_CHECKING, Optional, Union + +from kubernetes import client +from kubernetes import config as kube_config +from kubernetes.config.config_exception import ConfigException from prefect.blocks.core import Block -from prefect_kubernetes.utilities import KubernetesClient, get_kubernetes_client from pydantic import SecretStr +if TYPE_CHECKING: + from prefect.blocks.kubernetes import KubernetesClusterConfig + + +KubernetesClient = Union[ + client.BatchV1Api, client.CoreV1Api, client.AppsV1Api, client.ApiClient +] + +K8S_CLIENTS = { + "job": client.BatchV1Api, + "core": client.CoreV1Api, + "deployment": client.AppsV1Api, +} + + class KubernetesApiKey(Block): """Credentials block for API client generation across prefect-kubernetes tasks and flows. Args: - api_key (SecretStr): API key to authenticate with the Kubernetes API. - + api_key (SecretStr): API key to authenticate with the Kubernetes API. + Examples: Load a stored kubernetes API key: ```python - from prefect_kubernetes import KubernetesApiKey - + from prefect_kubernetes.credentials import KubernetesApiKey + kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") ``` - + Create a kubernetes API client from KubernetesApiKey and inferred cluster configuration: ```python from prefect_kubernetes import KubernetesApiKey from prefect_kubernetes.utilities import get_kubernetes_client - + kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") kubernetes_api_client = get_kubernetes_client("pod", kubernetes_api_key) ``` - + Create a namespaced kubernetes job: ```python from prefect_kubernetes import KubernetesApiKey from prefect_kubernetes.job import create_namespaced_job - + kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") - + create_namespaced_job( namespace="default", body={"Marvin": "42"}, **kube_kwargs ) @@ -40,8 +59,87 @@ class KubernetesApiKey(Block): _block_type_name = "Kubernetes Api Key" _logo_url = "https://kubernetes-security.info/assets/img/logo.png?h=250" # noqa - + api_key: SecretStr - - def get_client(self, resource: str) -> KubernetesClient: - return get_kubernetes_client(resource, kubernetes_api_key=self.api_key) \ No newline at end of file + + def get_core_client(self) -> client.CoreV1Api: + """Convenience method for retrieving a kubernetes api client for core resources + + Returns: + client.CoreV1Api: Kubernetes api client to interact with "pod", "service" and "secret" resources + """ + return get_kubernetes_client(resource="core", kubernetes_api_key=self.api_key) + + def get_batch_client(self) -> client.BatchV1Api: + """Convenience method for retrieving a kubernetes api client for job resources + + Returns: + client.BatchV1Api: Kubernetes api client to interact with "job" resources + """ + return get_kubernetes_client(resource="job", kubernetes_api_key=self.api_key) + + def get_app_client(self) -> client.AppsV1Api: + """Convenience method for retrieving a kubernetes api client for deployment resources + + Returns: + client.AppsV1Api: Kubernetes api client to interact with "deployment" resources + """ + return get_kubernetes_client( + resource="deployment", kubernetes_api_key=self.api_key + ) + + +def get_kubernetes_client( + resource: str, + kubernetes_api_key: Optional[KubernetesApiKey] = None, + kubernetes_cluster_config: "Optional[KubernetesClusterConfig]" = None, +) -> KubernetesClient: + """ + Utility function for loading kubernetes client object for a given resource. + It will attempt to connect to a Kubernetes cluster in three steps with + the first successful connection attempt becoming the mode of communication with a + cluster. + 1. Attempt to use a KubernetesApiKey block containing a Kubernetes API Key. If + `kubernetes_api_key` = `None` then it will attempt the next two connection + methods. + 2. Attempt in-cluster connection (will only work when running on a Pod in a cluster) + 3. Attempt out-of-cluster connection using the default location for a kube config file + In some cases connections to the kubernetes server are dropped after being idle for some time + (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in + ReadTimeoutErrors. + In order to prevent that a periodic keep-alive message can be sent to the server to keep the + connection open. + Args: + - resource (str): the name of the resource to retrieve a client for. Currently + you can use one of these values: `job`, `pod`, `service`, `deployment`, `secret` + - kubernetes_api_key (SecretStr): the value of a kubernetes api key in BearerToken format + Returns: + - KubernetesClient: an initialized, authenticated Kubernetes Client + """ + if kubernetes_api_key and kubernetes_cluster_config: + raise ValueError( + "Please provide EITHER a cluster config block OR an API key to generate an API client" + ) + + # KubernetesClusterConfig.get_api_client() returns k8s api client, usable with any resource type + if kubernetes_cluster_config: + return kubernetes_cluster_config.get_api_client() + + client_type = K8S_CLIENTS[resource] + + if kubernetes_api_key: + configuration = client.Configuration() + configuration.api_key["authorization"] = f"Bearer {kubernetes_api_key}" + k8s_client = client_type(client.ApiClient(configuration)) + else: + try: + print("Trying to load in-cluster configuration...") + kube_config.load_incluster_config() + except ConfigException as exc: + print("{} | Using out of cluster configuration option.".format(exc)) + print("Loading out-of-cluster configuration...") + kube_config.load_kube_config() + + k8s_client = client_type() + + return k8s_client diff --git a/prefect_kubernetes/exceptions.py b/prefect_kubernetes/exceptions.py index beb9801..149a137 100644 --- a/prefect_kubernetes/exceptions.py +++ b/prefect_kubernetes/exceptions.py @@ -1,9 +1,6 @@ class KubernetesJobDefinitionError(Exception): - """An exception to raise when a Kubernetes job definition is invalid + """An exception to raise when a Kubernetes job definition is invalid""" - """ - -class KubernetesJobFailedError(Exception): - """An exception to raise when a job orchestrated by a prefect_kubernetes task fails - """ \ No newline at end of file +class KubernetesJobFailedError(Exception): + """An exception to raise when a job orchestrated by a prefect_kubernetes task fails""" diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py index 583c77e..744d016 100644 --- a/prefect_kubernetes/job.py +++ b/prefect_kubernetes/job.py @@ -1,23 +1,22 @@ -import prefect.context import time - +from concurrent.futures import ThreadPoolExecutor from typing import Dict, Optional -from concurrent.futures import ThreadPoolExecutor +import prefect.context from kubernetes import client from prefect import get_run_logger, task + from prefect_kubernetes import exceptions as err from prefect_kubernetes.credentials import KubernetesApiKey from prefect_kubernetes.pod import read_namespaced_pod_logs -from prefect_kubernetes.utilities import KubernetesClient + @task def create_namespaced_job( - body: Dict = None, + body: Dict, + kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", - kubernetes_api_key: Optional[KubernetesApiKey] = None, - api_client: Optional[KubernetesClient] = None, - kube_kwargs: Optional[Dict] = {} + kube_kwargs: Optional[Dict] = {}, ): """Task for creating a namespaced Kubernetes job. @@ -27,7 +26,6 @@ def create_namespaced_job( namespace (str, optional): The Kubernetes namespace to create this job in, defaults to the `default` namespace. kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to create a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. @@ -39,20 +37,21 @@ def create_namespaced_job( "A dictionary representing a V1Job must be provided." ) - api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client - + api_client = kubernetes_api_key.get_batch_client() + body = {**body, **(body or {})} kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} api_client.create_namespaced_job(namespace=namespace, body=body, **kube_kwargs) - + + @task def delete_namespaced_job( job_name: str, + kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", - kubernetes_api_key: Optional[KubernetesApiKey] = None, - api_client: Optional[KubernetesClient] = None, - kube_kwargs: Optional[Dict] = {} + kube_kwargs: Optional[Dict] = {}, + delete_option_kwargs: Optional[Dict] = {}, ): """Task for deleting a namespaced Kubernetes job. @@ -60,11 +59,13 @@ def delete_namespaced_job( job_name (str): The name of a job to delete. Defaults to None. namespace (str, optional): The Kubernetes namespace to delete this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to delete a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + - delete_option_kwargs (dict, optional): Optional keyword arguments to pass to + the V1DeleteOptions object (e.g. {"propagation_policy": "...", + "grace_period_seconds": "..."}. Raises: ValueError: if `job_name` is `None` @@ -72,7 +73,7 @@ def delete_namespaced_job( if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + api_client = kubernetes_api_key.get_batch_client() kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} delete_option_kwargs = delete_option_kwargs or {} @@ -84,50 +85,51 @@ def delete_namespaced_job( **kube_kwargs, ) + @task def list_namespaced_job( + kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", - kubernetes_api_key: Optional[KubernetesApiKey] = None, - api_client: Optional[KubernetesClient] = None, - kube_kwargs: Optional[Dict] = {} + kube_kwargs: Optional[Dict] = {}, ): """Task for listing namespaced Kubernetes jobs. Args: namespace (str, optional): The Kubernetes namespace to list jobs from, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to delete a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. """ - api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + api_client = kubernetes_api_key.get_batch_client() kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} - + api_client.list_namespaced_job( namespace=namespace, **kube_kwargs, ) + @task def patch_namespaced_job( - job_name: str = None, + job_name: str, + body: dict, + kubernetes_api_key: KubernetesApiKey = None, namespace: Optional[str] = "default", - kubernetes_api_key: Optional[KubernetesApiKey] = None, - api_client: Optional[KubernetesClient] = None, - kube_kwargs: Optional[Dict] = {} + kube_kwargs: Optional[Dict] = {}, ): """Task for deleting a namespaced Kubernetes job. Args: job_name (str): The name of a job to patch. Defaults to None. + body (dict): A dictionary representation of a Kubernetes V1Job + specification. Defaults to None. namespace (str, optional): The Kubernetes namespace to patch this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to patch a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -142,7 +144,7 @@ def patch_namespaced_job( if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + api_client = kubernetes_api_key.get_batch_client() body = {**body, **(body or {})} kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} @@ -151,12 +153,12 @@ def patch_namespaced_job( name=job_name, namespace=namespace, body=body, **kube_kwargs ) + @task def read_namespaced_job( - job_name: str = None, + job_name: str, + kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", - kubernetes_api_key: Optional[KubernetesApiKey] = None, - api_client: Optional[KubernetesClient] = None, kube_kwargs: Optional[Dict] = {}, ): """Task for reading a namespaced kubernetes job. @@ -165,9 +167,8 @@ def read_namespaced_job( job_name (str): The name of a job to read. Defaults to None. namespace (str, optional): The Kubernetes namespace to read this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to read a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -177,7 +178,7 @@ def read_namespaced_job( if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + api_client = kubernetes_api_key.get_batch_client() kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} @@ -185,60 +186,69 @@ def read_namespaced_job( name=job_name, namespace=namespace, **kube_kwargs ) + @task def replace_namespaced_job( - job_name: str = None, + body: dict, + job_name: str, + kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", - kubernetes_api_key: Optional[KubernetesApiKey] = None, - api_client: Optional[KubernetesClient] = None, kube_kwargs: Optional[Dict] = {}, ): """Task for replacing a namespaced kubernetes job. Args: + body (dict, optional): A dictionary representation of a Kubernetes V1Job + specification job_name (str): The name of a job to replace. Defaults to None. namespace (str, optional): The Kubernetes namespace to replace this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client to use to replace a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. Raises: + ValueError: if `body` is `None` ValueError: if `job_name` is `None` """ + if not body: + raise err.KubernetesJobDefinitionError( + "A dictionary representing a V1Job must be provided." + ) + if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_client(resource="job") if not api_client else api_client + api_client = kubernetes_api_key.get_batch_client() + body = {**body, **(body or {})} kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} return api_client.replace_namespaced_job( - name=job_name, namespace=namespace, **kube_kwargs + name=job_name, body=body, namespace=namespace, **kube_kwargs ) + @task def run_namespaced_job( - job_name: str = None, + body: str, + kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", - kubernetes_api_key: Optional[KubernetesApiKey] = None, kube_kwargs: Optional[Dict] = {}, job_status_poll_interval: Optional[int] = 5, log_level: Optional[str] = None, - delete_job_after_completion: Optional[bool] = True + delete_job_after_completion: Optional[bool] = True, ): """Task for running a namespaced kubernetes job. Args: - job_name (str): The name of a kubernetes job to run. Defaults to None. + body (str): A dictionary representation of a Kubernetes V1Job + specification. Defaults to None. namespace (str, optional): The Kubernetes namespace to run this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. - api_client (KubernetesClient, optional): An initialized Kubernetes API client - to use to run a job. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. job_status_poll_interval (int, optional): The interval given in seconds @@ -254,13 +264,13 @@ def run_namespaced_job( KubernetesJobDefinitionError: if `body` is `None` KubernetesJobDefinitionError: if `body["metadata"]["name"] is `None` """ - logger = get_run_logger(log_level=log_level) - + logger = get_run_logger().setLevel(level=log_level) + if not body: raise err.KubernetesJobDefinitionError( "A dictionary representing a V1Job must be provided." ) - + # if log_level is not None and getattr(logger, log_level, None) is None: # raise ValueError("A valid log_level must be provided.") @@ -268,16 +278,16 @@ def run_namespaced_job( kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} job_name = body.get("metadata", {}).get("name") - + if not job_name: raise err.KubernetesJobDefinitionError( "The job name must be defined in the body under the metadata key." ) - api_client_job = kubernetes_api_key.get_client(resource="job") - api_client_pod = kubernetes_api_key.get_client(resource="pod") + api_batch_client = kubernetes_api_key.get_batch_client() + api_core_client = kubernetes_api_key.get_core_client() - api_client_job.create_namespaced_job( + api_batch_client.create_namespaced_job( namespace=namespace, body=body, **kube_kwargs ) logger.info(f"Job {job_name} has been created.") @@ -296,16 +306,14 @@ def initialize_thread(context): ) as pool: completed = False while not completed: - job = api_client_job.read_namespaced_job_status( + job = api_batch_client.read_namespaced_job_status( name=job_name, namespace=namespace ) if log_level is not None: func_log = getattr(logger, log_level) - - pod_selector = ( - f"controller-uid={job.metadata.labels['controller-uid']}" - ) - pods_list = api_client_pod.list_namespaced_pod( + + pod_selector = f"controller-uid={job.metadata.labels['controller-uid']}" + pods_list = api_core_client.list_namespaced_pod( namespace=namespace, label_selector=pod_selector ) @@ -339,9 +347,9 @@ def initialize_thread(context): break if delete_job_after_completion: - api_client_job.delete_namespaced_job( + api_batch_client.delete_namespaced_job( name=job_name, namespace=namespace, body=client.V1DeleteOptions(propagation_policy="Foreground"), ) - logger.info(f"Job {job_name} has been deleted.") \ No newline at end of file + logger.info(f"Job {job_name} has been deleted.") diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index 034f5b9..4defc0e 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -6,57 +6,58 @@ from prefect import task from prefect_kubernetes.credentials import KubernetesApiKey + @task def read_namespaced_pod_logs( - pod_name: str = None, - namespace: str = "default", - on_log_entry: Callable = None, - kubernetes_api_key: KubernetesApiKey = None, - container: str = None, - ) -> None: - """ - Task run method. - Args: - - pod_name (str, optional): The name of a pod to replace - - namespace (str, optional): The Kubernetes namespace to read pod logs in, - defaults to the `default` namespace - - on_log_entry (Callable, optional): If provided, will stream the pod logs - calling the callback for every line (and the task returns `None`). If not - provided, the current pod logs will be returned immediately from the task. - - kubernetes_api_key (KubernetesApiKey, optional): name of the KubernetesApiKey block - containing your Kubernetes API Key; value must be a string in BearerToken format - - container (str, optional): The name of the container to read logs from - Raises: - - ValueError: if `pod_name` is `None` - """ - if not pod_name: - raise ValueError("The name of a Kubernetes pod must be provided.") - - api_client = kubernetes_api_key.get_client(resource="pod") - - if on_log_entry is None: - return api_client.read_namespaced_pod_log( - name=pod_name, namespace=namespace, container=container + pod_name: str = None, + namespace: str = "default", + on_log_entry: Callable = None, + kubernetes_api_key: KubernetesApiKey = None, + container: str = None, +) -> None: + """ + Task run method. + Args: + - pod_name (str, optional): The name of a pod to replace + - namespace (str, optional): The Kubernetes namespace to read pod logs in, + defaults to the `default` namespace + - on_log_entry (Callable, optional): If provided, will stream the pod logs + calling the callback for every line (and the task returns `None`). If not + provided, the current pod logs will be returned immediately from the task. + - kubernetes_api_key (KubernetesApiKey, optional): name of the KubernetesApiKey block + containing your Kubernetes API Key; value must be a string in BearerToken format + - container (str, optional): The name of the container to read logs from + Raises: + - ValueError: if `pod_name` is `None` + """ + if not pod_name: + raise ValueError("The name of a Kubernetes pod must be provided.") + + api_core_client = kubernetes_api_key.get_core_client() + + if on_log_entry is None: + return api_core_client.read_namespaced_pod_log( + name=pod_name, namespace=namespace, container=container + ) + + # From the kubernetes.watch documentation: + # Note that watching an API resource can expire. The method tries to + # resume automatically once from the last result, but if that last result + # is too old as well, an `ApiException` exception will be thrown with + # ``code`` 410. + while True: + try: + stream = Watch().stream( + api_core_client.read_namespaced_pod_log, + name=pod_name, + namespace=namespace, + container=container, ) - # From the kubernetes.watch documentation: - # Note that watching an API resource can expire. The method tries to - # resume automatically once from the last result, but if that last result - # is too old as well, an `ApiException` exception will be thrown with - # ``code`` 410. - while True: - try: - stream = Watch().stream( - api_client.read_namespaced_pod_log, - name=pod_name, - namespace=namespace, - container=container, - ) - - for log in stream: - on_log_entry(log) - - return - except ApiException as exception: - if exception.status != 410: - raise + for log in stream: + on_log_entry(log) + + return + except ApiException as exception: + if exception.status != 410: + raise diff --git a/prefect_kubernetes/utilities.py b/prefect_kubernetes/utilities.py deleted file mode 100644 index fd66ca3..0000000 --- a/prefect_kubernetes/utilities.py +++ /dev/null @@ -1,109 +0,0 @@ -import sys -from typing import Optional, Union - -from kubernetes import client, config as kube_config -from kubernetes.config.config_exception import ConfigException -from prefect.blocks.kubernetes import KubernetesClusterConfig -from prefect_kubernetes.credentials import KubernetesApiKey - -KubernetesClient = Union[client.BatchV1Api, client.CoreV1Api, client.AppsV1Api, client.ApiClient] - -K8S_CLIENTS = { - "job": client.BatchV1Api, - "pod": client.CoreV1Api, - "service": client.CoreV1Api, - "deployment": client.AppsV1Api, - "secret": client.CoreV1Api, -} - -def get_kubernetes_client( - resource: str, - kubernetes_api_key: Optional[KubernetesApiKey] = None, - kubernetes_cluster_config: Optional[KubernetesClusterConfig] = None - -) -> KubernetesClient: - """ - Utility function for loading kubernetes client object for a given resource. - It will attempt to connect to a Kubernetes cluster in three steps with - the first successful connection attempt becoming the mode of communication with a - cluster. - 1. Attempt to use a KubernetesApiKey block containing a Kubernetes API Key. If - `kubernetes_api_key` = `None` then it will attempt the next two connection - methods. - 2. Attempt in-cluster connection (will only work when running on a Pod in a cluster) - 3. Attempt out-of-cluster connection using the default location for a kube config file - In some cases connections to the kubernetes server are dropped after being idle for some time - (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in - ReadTimeoutErrors. - In order to prevent that a periodic keep-alive message can be sent to the server to keep the - connection open. - Args: - - resource (str): the name of the resource to retrieve a client for. Currently - you can use one of these values: `job`, `pod`, `service`, `deployment`, `secret` - - kubernetes_api_key (SecretStr): the value of a kubernetes api key in BearerToken format - Returns: - - KubernetesClient: an initialized and authenticated Kubernetes Client - """ - if (resource or kubernetes_api_key) and kubernetes_cluster_config: - raise ValueError("Please specify either a cluster config block or an API key to generate an API client") - - - # KubernetesClusterConfig.get_api_client() returns a k8s api client that can be used to interact with any resource type - if kubernetes_cluster_config: - return kubernetes_cluster_config.get_api_client() - - client_type = K8S_CLIENTS[resource] - - if kubernetes_api_key: - - configuration = client.Configuration() - configuration.api_key["authorization"] = kubernetes_api_key - k8s_client = client_type(client.ApiClient(configuration)) - else: - try: - print("Trying to load in-cluster configuration...") - kube_config.load_incluster_config() - except ConfigException as exc: - print("{} | Using out of cluster configuration option.".format(exc)) - print("Loading out-of-cluster configuration...") - kube_config.load_kube_config() - - k8s_client = client_type() - - # if config.cloud.agent.kubernetes_keep_alive: - # _keep_alive(client=k8s_client) - - return k8s_client - -def _keep_alive(client: KubernetesClient) -> None: - """ - Setting the keep-alive flags on the kubernetes client object. - Unfortunately neither the kubernetes library nor the urllib3 library which kubernetes is using - internally offer the functionality to enable keep-alive messages. Thus the flags are added to - be used on the underlying sockets. - Args: - - client (KubernetesClient): the kubernetes client object on which the keep-alive should be - enabled - """ - import socket - - socket_options = [(socket.SOL_SOCKET, socket.SO_KEEPALIVE, 1)] - - if hasattr(socket, "TCP_KEEPINTVL"): - socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPINTVL, 30)) - - if hasattr(socket, "TCP_KEEPCNT"): - socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPCNT, 6)) - - if hasattr(socket, "TCP_KEEPIDLE"): - socket_options.append((socket.IPPROTO_TCP, socket.TCP_KEEPIDLE, 6)) - - if sys.platform == "darwin": - # TCP_KEEP_ALIVE not available on socket module in macOS, but defined in - # https://github.com/apple/darwin-xnu/blob/2ff845c2e033bd0ff64b5b6aa6063a1f8f65aa32/bsd/netinet/tcp.h#L215 - TCP_KEEP_ALIVE = 0x10 - socket_options.append((socket.IPPROTO_TCP, TCP_KEEP_ALIVE, 30)) - - client.api_client.rest_client.pool_manager.connection_pool_kw[ - "socket_options" - ] = socket_options \ No newline at end of file diff --git a/tests/conftest.py b/tests/conftest.py new file mode 100644 index 0000000..769a38e --- /dev/null +++ b/tests/conftest.py @@ -0,0 +1,55 @@ +from unittest.mock import MagicMock + +import pytest + +from prefect_kubernetes.credentials import KubernetesApiKey + + +@pytest.fixture +def successful_job_status(): + job_status = MagicMock() + job_status.status.active = None + job_status.status.failed = None + job_status.status.succeeded = 1 + return job_status + + +@pytest.fixture +def kubernetes_api_key(): + return KubernetesApiKey(api_key="XXXX") + + +@pytest.fixture +def api_app_client(monkeypatch): + app_client = MagicMock() + + monkeypatch.setattr( + "prefect_kubernetes.credentials.KubernetesApiKey.get_app_client", + MagicMock(return_value=app_client), + ) + + return app_client + + +@pytest.fixture +def api_batch_client(monkeypatch): + batch_client = MagicMock() + + monkeypatch.setattr( + "prefect_kubernetes.credentials.KubernetesApiKey.get_batch_client", + MagicMock(return_value=batch_client), + ) + + return batch_client + + +@pytest.fixture +def api_core_client(monkeypatch): + core_client = MagicMock() + + monkeypatch.setattr( + "prefect_kubernetes.credentials.KubernetesApiKey.get_core_client", + MagicMock(return_value=core_client), + ) + + return core_client diff --git a/tests/test_job.py b/tests/test_job.py new file mode 100644 index 0000000..4177459 --- /dev/null +++ b/tests/test_job.py @@ -0,0 +1,116 @@ +import pytest +from prefect import flow +from prefect_kubernetes import exceptions as err +from prefect_kubernetes.job import ( + create_namespaced_job, + delete_namespaced_job, + list_namespaced_job, + patch_namespaced_job, + read_namespaced_job, + replace_namespaced_job, + run_namespaced_job, +) + + +async def test_invalid_body_raises_error(kubernetes_api_key): + with pytest.raises(err.KubernetesJobDefinitionError): + await create_namespaced_job.fn(body=None, kubernetes_api_key=kubernetes_api_key) + with pytest.raises(err.KubernetesJobDefinitionError): + await patch_namespaced_job.fn( + body=None, job_name="", kubernetes_api_key=kubernetes_api_key + ) + + +async def test_create_namespaced_job(kubernetes_api_key, api_batch_client): + create_namespaced_job.fn( + body={"test": "a"}, + kube_kwargs={"a": "test"}, + kubernetes_api_key=kubernetes_api_key, + ) + + assert api_batch_client.create_namespaced_job.call_args[1]["body"] == {"test": "a"} + assert api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" + + +async def test_delete_namespaced_job(kubernetes_api_key, api_batch_client): + delete_namespaced_job.fn( + job_name="test_job", + kube_kwargs={"a": "test"}, + kubernetes_api_key=kubernetes_api_key, + ) + assert api_batch_client.delete_namespaced_job.call_args[1]["name"] == "test_job" + assert api_batch_client.delete_namespaced_job.call_args[1]["a"] == "test" + + +async def test_list_namespaced_job(kubernetes_api_key, api_batch_client): + list_namespaced_job.fn( + namespace="ns", + kube_kwargs={"a": "test"}, + kubernetes_api_key=kubernetes_api_key, + ) + assert api_batch_client.list_namespaced_job.call_args[1]["namespace"] == "ns" + assert api_batch_client.list_namespaced_job.call_args[1]["a"] == "test" + + +async def test_patch_namespaced_job(kubernetes_api_key, api_batch_client): + patch_namespaced_job.fn( + body={"test": "a"}, + job_name="test_job", + kube_kwargs={"a": "test"}, + kubernetes_api_key=kubernetes_api_key, + ) + assert api_batch_client.patch_namespaced_job.call_args[1]["body"] == {"test": "a"} + assert api_batch_client.patch_namespaced_job.call_args[1]["name"] == "test_job" + assert api_batch_client.patch_namespaced_job.call_args[1]["a"] == "test" + + +async def test_read_namespaced_job(kubernetes_api_key, api_batch_client): + read_namespaced_job.fn( + job_name="test_job", + namespace="ns", + kube_kwargs={"a": "test"}, + kubernetes_api_key=kubernetes_api_key, + ) + assert api_batch_client.read_namespaced_job.call_args[1]["name"] == "test_job" + assert api_batch_client.read_namespaced_job.call_args[1]["namespace"] == "ns" + assert api_batch_client.read_namespaced_job.call_args[1]["a"] == "test" + + +async def test_replace_namespaced_job(kubernetes_api_key, api_batch_client): + replace_namespaced_job.fn( + job_name="test_job", + body={"test": "a"}, + namespace="ns", + kube_kwargs={"a": "test"}, + kubernetes_api_key=kubernetes_api_key, + ) + assert api_batch_client.replace_namespaced_job.call_args[1]["name"] == "test_job" + assert api_batch_client.replace_namespaced_job.call_args[1]["namespace"] == "ns" + assert api_batch_client.replace_namespaced_job.call_args[1]["body"] == {"test": "a"} + assert api_batch_client.replace_namespaced_job.call_args[1]["a"] == "test" + + +async def test_run_namespaced_job( + kubernetes_api_key, api_batch_client, api_core_client +): + @flow + def test_flow(): + run_namespaced_job( + body={"metadata": {"name": "test"}}, + namespace="ns", + kube_kwargs={"a": "test"}, + kubernetes_api_key=kubernetes_api_key, + log_level='DEBUG' + ) + test_flow() + + assert api_batch_client.create_namespaced_job.call_args[1]["body"] == {"test": "a"} + assert api_batch_client.create_namespaced_job.call_args[1]["namespace"] == "ns" + assert api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" + + assert api_batch_client.read_namespaced_job_status.call_args[1]["body"] == { + "test": "a" + } + assert api_batch_client.read_namespaced_job_status.call_args[1]["namespace"] == "ns" + + assert api_core_client.list_namespaced_pod.call_args[1]["namespace"] == "ns" From f263366125c9f18b92127df48f42ca56f1f57ef5 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 2 Sep 2022 12:32:20 -0500 Subject: [PATCH 04/43] add connect_get_namespaced_pod_exec and initial test --- prefect_kubernetes/job.py | 16 ++++---- prefect_kubernetes/pod.py | 78 ++++++++++++++++++++++++++++++++++++--- tests/conftest.py | 13 +++++++ tests/test_job.py | 58 +++++++++++++++-------------- tests/test_pod.py | 17 +++++++++ 5 files changed, 141 insertions(+), 41 deletions(-) create mode 100644 tests/test_pod.py diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py index 744d016..5e7f545 100644 --- a/prefect_kubernetes/job.py +++ b/prefect_kubernetes/job.py @@ -12,7 +12,7 @@ @task -def create_namespaced_job( +async def create_namespaced_job( body: Dict, kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", @@ -46,7 +46,7 @@ def create_namespaced_job( @task -def delete_namespaced_job( +async def delete_namespaced_job( job_name: str, kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", @@ -87,7 +87,7 @@ def delete_namespaced_job( @task -def list_namespaced_job( +async def list_namespaced_job( kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = {}, @@ -98,7 +98,7 @@ def list_namespaced_job( namespace (str, optional): The Kubernetes namespace to list jobs from, defaults to the `default` namespace. kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block - holding a Kubernetes API Key. Defaults to None. + holding a Kubernetes API Key kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. """ @@ -113,7 +113,7 @@ def list_namespaced_job( @task -def patch_namespaced_job( +async def patch_namespaced_job( job_name: str, body: dict, kubernetes_api_key: KubernetesApiKey = None, @@ -155,7 +155,7 @@ def patch_namespaced_job( @task -def read_namespaced_job( +async def read_namespaced_job( job_name: str, kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", @@ -188,7 +188,7 @@ def read_namespaced_job( @task -def replace_namespaced_job( +async def replace_namespaced_job( body: dict, job_name: str, kubernetes_api_key: KubernetesApiKey, @@ -231,7 +231,7 @@ def replace_namespaced_job( @task -def run_namespaced_job( +async def run_namespaced_job( body: str, kubernetes_api_key: KubernetesApiKey, namespace: Optional[str] = "default", diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index 4defc0e..83e8e9e 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -1,14 +1,14 @@ -from typing import Callable +from typing import Any, Callable, Dict, List -from kubernetes.watch import Watch from kubernetes.client.rest import ApiException - +from kubernetes.stream import stream +from kubernetes.watch import Watch from prefect import task from prefect_kubernetes.credentials import KubernetesApiKey @task -def read_namespaced_pod_logs( +async def read_namespaced_pod_logs( pod_name: str = None, namespace: str = "default", on_log_entry: Callable = None, @@ -33,7 +33,10 @@ def read_namespaced_pod_logs( if not pod_name: raise ValueError("The name of a Kubernetes pod must be provided.") - api_core_client = kubernetes_api_key.get_core_client() + if kubernetes_api_key: + api_core_client = kubernetes_api_key.get_core_client() + else: + raise ValueError("A `KubernetesApiKey` block must be provided") if on_log_entry is None: return api_core_client.read_namespaced_pod_log( @@ -61,3 +64,68 @@ def read_namespaced_pod_logs( except ApiException as exception: if exception.status != 410: raise + + +@task(name="Run a command in a namespaced pod on Kubernetes") +async def connect_get_namespaced_pod_exec( + pod_name: str = None, + container_name: str = None, + exec_command: List = None, + kubernetes_api_key: KubernetesApiKey = None, + namespace: str = "default", + kube_kwargs: Dict = {}, +) -> Callable: + """Task for running a command in a namespaced pod on Kubernetes. + + This task requires a `KubernetesApiKey` to generate a`CoreV1Api` Kubernetes + client to stream the overridden `api_response` to `connect_get_namespaced_pod_exec`. + + Args: + pod_name (str): The name of a pod in which the command is to be run + container_name (str): The name of a container to use in the pod. + Defaults to `None` + exec_command (List): The command to run in `pod_name` + Defaults to `None` + kubernetes_api_key (KubernetesApiKey): A block that stores a Kubernetes api, + has methods to generate resource specific client + namespace (str, optional): The Kubernetes namespace of the pod. + Defaults to `default` + kube_kwargs (Dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "exact": "..."}`) + + Returns: + Callable: A Kubernetes websocket `stream` according to supplied Kubernetes API method + and kwarg overrides + + + Raises: + - ValueError: if `pod_name` is `None` or `container_name` is `None` + - TypeError: `exec_command` is not a list + """ + if not (pod_name and container_name): + raise ValueError("The name of a Kubernetes pod and container must be provided.") + + if not isinstance(exec_command, List): + raise TypeError("The `exec_command` argument must be provided as a list") + + if not kubernetes_api_key: + raise ValueError("An existing `KubernetesApiKey` block must be provided.") + + api_client = kubernetes_api_key.get_core_client() + + kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + + api_response = stream( + api_client.connect_get_namespaced_pod_exec, + name=pod_name, + namespace=namespace, + container=container_name, + command=exec_command, + stderr=True, + stdin=True, + stdout=True, + tty=False, + **kube_kwargs + ) + + return api_response diff --git a/tests/conftest.py b/tests/conftest.py index 769a38e..ad05baf 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -53,3 +53,16 @@ def api_core_client(monkeypatch): ) return core_client + + +@pytest.fixture +def api_request_method(monkeypatch): + method = MagicMock() + method.__self__ = api_core_client + + monkeypatch.setattr( + "kubernetes.client.api_client.ApiClient.request", + "method", + method, + ) + return method diff --git a/tests/test_job.py b/tests/test_job.py index 4177459..7e2351c 100644 --- a/tests/test_job.py +++ b/tests/test_job.py @@ -1,5 +1,6 @@ import pytest from prefect import flow + from prefect_kubernetes import exceptions as err from prefect_kubernetes.job import ( create_namespaced_job, @@ -22,7 +23,7 @@ async def test_invalid_body_raises_error(kubernetes_api_key): async def test_create_namespaced_job(kubernetes_api_key, api_batch_client): - create_namespaced_job.fn( + await create_namespaced_job.fn( body={"test": "a"}, kube_kwargs={"a": "test"}, kubernetes_api_key=kubernetes_api_key, @@ -33,7 +34,7 @@ async def test_create_namespaced_job(kubernetes_api_key, api_batch_client): async def test_delete_namespaced_job(kubernetes_api_key, api_batch_client): - delete_namespaced_job.fn( + await delete_namespaced_job.fn( job_name="test_job", kube_kwargs={"a": "test"}, kubernetes_api_key=kubernetes_api_key, @@ -43,7 +44,7 @@ async def test_delete_namespaced_job(kubernetes_api_key, api_batch_client): async def test_list_namespaced_job(kubernetes_api_key, api_batch_client): - list_namespaced_job.fn( + await list_namespaced_job.fn( namespace="ns", kube_kwargs={"a": "test"}, kubernetes_api_key=kubernetes_api_key, @@ -53,7 +54,7 @@ async def test_list_namespaced_job(kubernetes_api_key, api_batch_client): async def test_patch_namespaced_job(kubernetes_api_key, api_batch_client): - patch_namespaced_job.fn( + await patch_namespaced_job.fn( body={"test": "a"}, job_name="test_job", kube_kwargs={"a": "test"}, @@ -65,7 +66,7 @@ async def test_patch_namespaced_job(kubernetes_api_key, api_batch_client): async def test_read_namespaced_job(kubernetes_api_key, api_batch_client): - read_namespaced_job.fn( + await read_namespaced_job.fn( job_name="test_job", namespace="ns", kube_kwargs={"a": "test"}, @@ -77,7 +78,7 @@ async def test_read_namespaced_job(kubernetes_api_key, api_batch_client): async def test_replace_namespaced_job(kubernetes_api_key, api_batch_client): - replace_namespaced_job.fn( + await replace_namespaced_job.fn( job_name="test_job", body={"test": "a"}, namespace="ns", @@ -90,27 +91,28 @@ async def test_replace_namespaced_job(kubernetes_api_key, api_batch_client): assert api_batch_client.replace_namespaced_job.call_args[1]["a"] == "test" -async def test_run_namespaced_job( - kubernetes_api_key, api_batch_client, api_core_client -): - @flow - def test_flow(): - run_namespaced_job( - body={"metadata": {"name": "test"}}, - namespace="ns", - kube_kwargs={"a": "test"}, - kubernetes_api_key=kubernetes_api_key, - log_level='DEBUG' - ) - test_flow() - - assert api_batch_client.create_namespaced_job.call_args[1]["body"] == {"test": "a"} - assert api_batch_client.create_namespaced_job.call_args[1]["namespace"] == "ns" - assert api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" +# async def test_run_namespaced_job( +# kubernetes_api_key, api_batch_client, api_core_client +# ): +# @flow +# def test_flow(): +# run_namespaced_job( +# body={"metadata": {"name": "test"}}, +# namespace="ns", +# kube_kwargs={"a": "test"}, +# kubernetes_api_key=kubernetes_api_key, +# log_level="DEBUG", +# ) + +# test_flow() + +# assert api_batch_client.create_namespaced_job.call_args[1]["body"] == {"test": "a"} +# assert api_batch_client.create_namespaced_job.call_args[1]["namespace"] == "ns" +# assert api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" - assert api_batch_client.read_namespaced_job_status.call_args[1]["body"] == { - "test": "a" - } - assert api_batch_client.read_namespaced_job_status.call_args[1]["namespace"] == "ns" +# assert api_batch_client.read_namespaced_job_status.call_args[1]["body"] == { +# "test": "a" +# } +# assert api_batch_client.read_namespaced_job_status.call_args[1]["namespace"] == "ns" - assert api_core_client.list_namespaced_pod.call_args[1]["namespace"] == "ns" +# assert api_core_client.list_namespaced_pod.call_args[1]["namespace"] == "ns" diff --git a/tests/test_pod.py b/tests/test_pod.py new file mode 100644 index 0000000..3fe4d05 --- /dev/null +++ b/tests/test_pod.py @@ -0,0 +1,17 @@ +import pytest + +from prefect_kubernetes.pod import connect_get_namespaced_pod_exec + + +async def test_connect_get_namespaced_pod_exec(kubernetes_api_key): + + response = await connect_get_namespaced_pod_exec.fn( + pod_name="test_pod", + container_name="test_container", + exec_command=["whoami"], + namespace="test_ns", + kubernetes_api_key=kubernetes_api_key, + kube_kwargs={}, + ) + + assert 1 == response.splitlines() From e6ba8b5edbcc35a85b24cae1c4665614f36a1fc9 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Sun, 4 Sep 2022 20:00:50 -0500 Subject: [PATCH 05/43] add initial connect_get_namespaced_pod_exec coverage --- .gitignore | 3 + prefect_kubernetes/credentials.py | 136 ++++++++++++++---------------- prefect_kubernetes/job.py | 73 ++++++++-------- prefect_kubernetes/pod.py | 95 +++++++++------------ tests/conftest.py | 74 +++++++++++----- tests/kube_config.yaml | 13 +++ tests/test_pod.py | 44 +++++++--- 7 files changed, 246 insertions(+), 192 deletions(-) create mode 100644 tests/kube_config.yaml diff --git a/.gitignore b/.gitignore index b96a3be..8fe1688 100644 --- a/.gitignore +++ b/.gitignore @@ -136,3 +136,6 @@ dmypy.json # Jupyter notebook *.ipynb + +# testing +testing_stuff.py \ No newline at end of file diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index c4b616e..54918d8 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -1,14 +1,14 @@ +from distutils.command.config import config from typing import TYPE_CHECKING, Optional, Union from kubernetes import client from kubernetes import config as kube_config from kubernetes.config.config_exception import ConfigException from prefect.blocks.core import Block -from pydantic import SecretStr - -if TYPE_CHECKING: - from prefect.blocks.kubernetes import KubernetesClusterConfig +# if TYPE_CHECKING: +from prefect.blocks.kubernetes import KubernetesClusterConfig +from pydantic import SecretStr KubernetesClient = Union[ client.BatchV1Api, client.CoreV1Api, client.AppsV1Api, client.ApiClient @@ -21,7 +21,7 @@ } -class KubernetesApiKey(Block): +class KubernetesCredentials(Block): """Credentials block for API client generation across prefect-kubernetes tasks and flows. Args: @@ -30,18 +30,18 @@ class KubernetesApiKey(Block): Examples: Load a stored kubernetes API key: ```python - from prefect_kubernetes.credentials import KubernetesApiKey + from prefect_kubernetes.credentials import KubernetesCredentials - kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") + kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") ``` - Create a kubernetes API client from KubernetesApiKey and inferred cluster configuration: + Create a kubernetes API client from KubernetesCredentials and inferred cluster configuration: ```python - from prefect_kubernetes import KubernetesApiKey + from prefect_kubernetes import KubernetesCredentials from prefect_kubernetes.utilities import get_kubernetes_client - kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") - kubernetes_api_client = get_kubernetes_client("pod", kubernetes_api_key) + kubernetes_credentials = KubernetesCredentials.load("my-k8s-api-key") + kubernetes_api_client = kubernetes_credentials.get_core_client() ``` Create a namespaced kubernetes job: @@ -49,7 +49,7 @@ class KubernetesApiKey(Block): from prefect_kubernetes import KubernetesApiKey from prefect_kubernetes.job import create_namespaced_job - kubernetes_api_key = KubernetesApiKey.load("my-k8s-api-key") + kubernetes_credentials = KubernetesApiKey.load("my-k8s-api-key") create_namespaced_job( namespace="default", body={"Marvin": "42"}, **kube_kwargs @@ -57,10 +57,11 @@ class KubernetesApiKey(Block): ``` """ - _block_type_name = "Kubernetes Api Key" + _block_type_name = "Kubernetes Credentials" _logo_url = "https://kubernetes-security.info/assets/img/logo.png?h=250" # noqa - api_key: SecretStr + api_key: SecretStr = None + cluster_config: KubernetesClusterConfig = None def get_core_client(self) -> client.CoreV1Api: """Convenience method for retrieving a kubernetes api client for core resources @@ -68,7 +69,7 @@ def get_core_client(self) -> client.CoreV1Api: Returns: client.CoreV1Api: Kubernetes api client to interact with "pod", "service" and "secret" resources """ - return get_kubernetes_client(resource="core", kubernetes_api_key=self.api_key) + return self.get_kubernetes_client(resource="core") def get_batch_client(self) -> client.BatchV1Api: """Convenience method for retrieving a kubernetes api client for job resources @@ -76,7 +77,7 @@ def get_batch_client(self) -> client.BatchV1Api: Returns: client.BatchV1Api: Kubernetes api client to interact with "job" resources """ - return get_kubernetes_client(resource="job", kubernetes_api_key=self.api_key) + return self.get_kubernetes_client(resource="job") def get_app_client(self) -> client.AppsV1Api: """Convenience method for retrieving a kubernetes api client for deployment resources @@ -84,62 +85,53 @@ def get_app_client(self) -> client.AppsV1Api: Returns: client.AppsV1Api: Kubernetes api client to interact with "deployment" resources """ - return get_kubernetes_client( - resource="deployment", kubernetes_api_key=self.api_key - ) - - -def get_kubernetes_client( - resource: str, - kubernetes_api_key: Optional[KubernetesApiKey] = None, - kubernetes_cluster_config: "Optional[KubernetesClusterConfig]" = None, -) -> KubernetesClient: - """ - Utility function for loading kubernetes client object for a given resource. - It will attempt to connect to a Kubernetes cluster in three steps with - the first successful connection attempt becoming the mode of communication with a - cluster. - 1. Attempt to use a KubernetesApiKey block containing a Kubernetes API Key. If - `kubernetes_api_key` = `None` then it will attempt the next two connection - methods. - 2. Attempt in-cluster connection (will only work when running on a Pod in a cluster) - 3. Attempt out-of-cluster connection using the default location for a kube config file - In some cases connections to the kubernetes server are dropped after being idle for some time - (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in - ReadTimeoutErrors. - In order to prevent that a periodic keep-alive message can be sent to the server to keep the - connection open. - Args: - - resource (str): the name of the resource to retrieve a client for. Currently - you can use one of these values: `job`, `pod`, `service`, `deployment`, `secret` - - kubernetes_api_key (SecretStr): the value of a kubernetes api key in BearerToken format - Returns: - - KubernetesClient: an initialized, authenticated Kubernetes Client - """ - if kubernetes_api_key and kubernetes_cluster_config: - raise ValueError( - "Please provide EITHER a cluster config block OR an API key to generate an API client" - ) - - # KubernetesClusterConfig.get_api_client() returns k8s api client, usable with any resource type - if kubernetes_cluster_config: - return kubernetes_cluster_config.get_api_client() + return self.get_kubernetes_client(resource="deployment") - client_type = K8S_CLIENTS[resource] - - if kubernetes_api_key: - configuration = client.Configuration() - configuration.api_key["authorization"] = f"Bearer {kubernetes_api_key}" - k8s_client = client_type(client.ApiClient(configuration)) - else: - try: - print("Trying to load in-cluster configuration...") - kube_config.load_incluster_config() - except ConfigException as exc: - print("{} | Using out of cluster configuration option.".format(exc)) - print("Loading out-of-cluster configuration...") - kube_config.load_kube_config() + def get_kubernetes_client(self, resource: str) -> KubernetesClient: + """ + Utility function for loading kubernetes client object for a given resource. + It will attempt to connect to a Kubernetes cluster in three steps with + the first successful connection attempt becoming the mode of communication with a + cluster. + 1. Attempt to use a KubernetesCredentials block containing a Kubernetes API Key. If + `kubernetes_credentials` = `None` then it will attempt the next two connection + methods. + 2. Attempt in-cluster connection (will only work when running on a Pod in a cluster) + 3. Attempt out-of-cluster connection using the default location for a kube config file + In some cases connections to the kubernetes server are dropped after being idle for some time + (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in + ReadTimeoutErrors. + In order to prevent that a periodic keep-alive message can be sent to the server to keep the + connection open. + Args: + - resource (str): the name of the resource to retrieve a client for. Currently + you can use one of these values: `job`, `pod`, `service`, `deployment`, `secret` - k8s_client = client_type() + Returns: + - KubernetesClient: an initialized, authenticated Kubernetes Client + """ - return k8s_client + resource_specific_client = K8S_CLIENTS[resource] + + if self.api_key: + configuration = client.Configuration() + configuration.api_key["authorization"] = self.api_key.get_secret_value() + configuration.api_key_prefix["authorization"] = "Bearer" + k8s_client = resource_specific_client( + api_client=client.ApiClient(configuration=configuration) + ) + elif self.cluster_config: + self.cluster_config.configure_client() + k8s_client = resource_specific_client() + else: + try: + print("Trying to load in-cluster configuration...") + kube_config.load_incluster_config() + except ConfigException as exc: + print("{} | Using out of cluster configuration option.".format(exc)) + print("Loading out-of-cluster configuration...") + kube_config.load_kube_config() + + k8s_client = resource_specific_client() + + return k8s_client diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py index 5e7f545..bd4f921 100644 --- a/prefect_kubernetes/job.py +++ b/prefect_kubernetes/job.py @@ -7,16 +7,16 @@ from prefect import get_run_logger, task from prefect_kubernetes import exceptions as err -from prefect_kubernetes.credentials import KubernetesApiKey +from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.pod import read_namespaced_pod_logs @task async def create_namespaced_job( body: Dict, - kubernetes_api_key: KubernetesApiKey, + kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = {}, + kube_kwargs: Optional[Dict] = None, ): """Task for creating a namespaced Kubernetes job. @@ -25,7 +25,7 @@ async def create_namespaced_job( specification. Defaults to None. namespace (str, optional): The Kubernetes namespace to create this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block holding a Kubernetes API Key. Defaults to None. + kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. @@ -37,7 +37,7 @@ async def create_namespaced_job( "A dictionary representing a V1Job must be provided." ) - api_client = kubernetes_api_key.get_batch_client() + api_client = kubernetes_credentials.get_batch_client() body = {**body, **(body or {})} kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} @@ -48,10 +48,10 @@ async def create_namespaced_job( @task async def delete_namespaced_job( job_name: str, - kubernetes_api_key: KubernetesApiKey, + kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = {}, - delete_option_kwargs: Optional[Dict] = {}, + kube_kwargs: Optional[Dict] = None, + delete_option_kwargs: Optional[Dict] = None, ): """Task for deleting a namespaced Kubernetes job. @@ -59,7 +59,7 @@ async def delete_namespaced_job( job_name (str): The name of a job to delete. Defaults to None. namespace (str, optional): The Kubernetes namespace to delete this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -73,42 +73,43 @@ async def delete_namespaced_job( if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_batch_client() + api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} - delete_option_kwargs = delete_option_kwargs or {} + method_kwargs = {} + + if delete_option_kwargs: + method_kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) + if kube_kwargs: + method_kwargs.update(**kube_kwargs) api_client.delete_namespaced_job( - name=job_name, - namespace=namespace, - body=client.V1DeleteOptions(**delete_option_kwargs), - **kube_kwargs, + name=job_name, namespace=namespace, **method_kwargs ) @task async def list_namespaced_job( - kubernetes_api_key: KubernetesApiKey, + kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = {}, + kube_kwargs: Optional[Dict] = None, ): """Task for listing namespaced Kubernetes jobs. Args: namespace (str, optional): The Kubernetes namespace to list jobs from, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. """ - api_client = kubernetes_api_key.get_batch_client() + api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + method_kwargs = kube_kwargs or {} api_client.list_namespaced_job( namespace=namespace, - **kube_kwargs, + **method_kwargs, ) @@ -116,7 +117,7 @@ async def list_namespaced_job( async def patch_namespaced_job( job_name: str, body: dict, - kubernetes_api_key: KubernetesApiKey = None, + kubernetes_credentials: KubernetesCredentials = None, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = {}, ): @@ -128,7 +129,7 @@ async def patch_namespaced_job( specification. Defaults to None. namespace (str, optional): The Kubernetes namespace to patch this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -144,7 +145,7 @@ async def patch_namespaced_job( if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_batch_client() + api_client = kubernetes_credentials.get_batch_client() body = {**body, **(body or {})} kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} @@ -157,7 +158,7 @@ async def patch_namespaced_job( @task async def read_namespaced_job( job_name: str, - kubernetes_api_key: KubernetesApiKey, + kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = {}, ): @@ -167,7 +168,7 @@ async def read_namespaced_job( job_name (str): The name of a job to read. Defaults to None. namespace (str, optional): The Kubernetes namespace to read this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -178,7 +179,7 @@ async def read_namespaced_job( if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_batch_client() + api_client = kubernetes_credentials.get_batch_client() kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} @@ -191,7 +192,7 @@ async def read_namespaced_job( async def replace_namespaced_job( body: dict, job_name: str, - kubernetes_api_key: KubernetesApiKey, + kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = {}, ): @@ -203,7 +204,7 @@ async def replace_namespaced_job( job_name (str): The name of a job to replace. Defaults to None. namespace (str, optional): The Kubernetes namespace to replace this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -220,7 +221,7 @@ async def replace_namespaced_job( if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_api_key.get_batch_client() + api_client = kubernetes_credentials.get_batch_client() body = {**body, **(body or {})} kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} @@ -233,7 +234,7 @@ async def replace_namespaced_job( @task async def run_namespaced_job( body: str, - kubernetes_api_key: KubernetesApiKey, + kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = {}, job_status_poll_interval: Optional[int] = 5, @@ -247,7 +248,7 @@ async def run_namespaced_job( specification. Defaults to None. namespace (str, optional): The Kubernetes namespace to run this job in, defaults to the `default` namespace. - kubernetes_api_key (KubernetesApiKey, optional): KubernetesApiKey block + kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key. Defaults to None. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -284,8 +285,8 @@ async def run_namespaced_job( "The job name must be defined in the body under the metadata key." ) - api_batch_client = kubernetes_api_key.get_batch_client() - api_core_client = kubernetes_api_key.get_core_client() + api_batch_client = kubernetes_credentials.get_batch_client() + api_core_client = kubernetes_credentials.get_core_client() api_batch_client.create_namespaced_job( namespace=namespace, body=body, **kube_kwargs @@ -329,7 +330,7 @@ def initialize_thread(context): read_pod_logs = read_namespaced_pod_logs( pod_name=pod_name, namespace=namespace, - kubernetes_api_key=kubernetes_api_key, + kubernetes_credentials=kubernetes_credentials, on_log_entry=lambda log: func_log(f"{pod_name}: {log}"), ) diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index 83e8e9e..822c192 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -1,42 +1,35 @@ -from typing import Any, Callable, Dict, List +from typing import Any, Callable, Dict, List, Optional from kubernetes.client.rest import ApiException from kubernetes.stream import stream from kubernetes.watch import Watch from prefect import task -from prefect_kubernetes.credentials import KubernetesApiKey + +from prefect_kubernetes.credentials import KubernetesCredentials @task async def read_namespaced_pod_logs( - pod_name: str = None, + pod_name: str, + kubernetes_credentials: KubernetesCredentials, namespace: str = "default", on_log_entry: Callable = None, - kubernetes_api_key: KubernetesApiKey = None, container: str = None, ) -> None: """ Task run method. Args: - pod_name (str, optional): The name of a pod to replace + - kubernetes_credentials (KubernetesCredentials): name of the KubernetesCredentials block - namespace (str, optional): The Kubernetes namespace to read pod logs in, defaults to the `default` namespace - on_log_entry (Callable, optional): If provided, will stream the pod logs calling the callback for every line (and the task returns `None`). If not provided, the current pod logs will be returned immediately from the task. - - kubernetes_api_key (KubernetesApiKey, optional): name of the KubernetesApiKey block - containing your Kubernetes API Key; value must be a string in BearerToken format - container (str, optional): The name of the container to read logs from - Raises: - - ValueError: if `pod_name` is `None` """ - if not pod_name: - raise ValueError("The name of a Kubernetes pod must be provided.") - if kubernetes_api_key: - api_core_client = kubernetes_api_key.get_core_client() - else: - raise ValueError("A `KubernetesApiKey` block must be provided") + api_core_client = kubernetes_credentials.get_core_client() if on_log_entry is None: return api_core_client.read_namespaced_pod_log( @@ -66,66 +59,62 @@ async def read_namespaced_pod_logs( raise -@task(name="Run a command in a namespaced pod on Kubernetes") +@task async def connect_get_namespaced_pod_exec( - pod_name: str = None, - container_name: str = None, - exec_command: List = None, - kubernetes_api_key: KubernetesApiKey = None, - namespace: str = "default", - kube_kwargs: Dict = {}, -) -> Callable: + name: str, + container: str, + command: List[str], + kubernetes_credentials: KubernetesCredentials, + namespace: Optional[str] = "default", + **kwargs, +) -> str: """Task for running a command in a namespaced pod on Kubernetes. - This task requires a `KubernetesApiKey` to generate a`CoreV1Api` Kubernetes + This task requires `KubernetesCredentials` to generate a`CoreV1Api` Kubernetes client to stream the overridden `api_response` to `connect_get_namespaced_pod_exec`. + User-provided `kwargs` will overwrite `default_kwargs` if key values from `default_kwargs` + are set in `kwargs`. + Args: - pod_name (str): The name of a pod in which the command is to be run - container_name (str): The name of a container to use in the pod. - Defaults to `None` - exec_command (List): The command to run in `pod_name` - Defaults to `None` - kubernetes_api_key (KubernetesApiKey): A block that stores a Kubernetes api, - has methods to generate resource specific client + name (str): The name of the pod in which the command is to be run + container (str): The name of a container to use in the pod. + command (List): The command to run in `pod_name` + kubernetes_credentials (KubernetesCredentials): A block that stores a Kubernetes credentials, + has methods to generate resource-specific client namespace (str, optional): The Kubernetes namespace of the pod. Defaults to `default` - kube_kwargs (Dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "exact": "..."}`) + kwargs (Dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API method (e.g. `{"stderr": "False", "tty": "True"}`) Returns: - Callable: A Kubernetes websocket `stream` according to supplied Kubernetes API method - and kwarg overrides - + str: The string output of `command` on `container`, will be empty if `stdout=False` Raises: - - ValueError: if `pod_name` is `None` or `container_name` is `None` - - TypeError: `exec_command` is not a list + - TypeError: `command` is not a list """ - if not (pod_name and container_name): - raise ValueError("The name of a Kubernetes pod and container must be provided.") - if not isinstance(exec_command, List): - raise TypeError("The `exec_command` argument must be provided as a list") + if not isinstance(command, List): + raise TypeError("The `command` argument must be provided as a list") - if not kubernetes_api_key: - raise ValueError("An existing `KubernetesApiKey` block must be provided.") + api_client = kubernetes_credentials.get_core_client() - api_client = kubernetes_api_key.get_core_client() + default_kwargs = dict( + stderr=True, + stdin=True, + stdout=True, + tty=False, + ) - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + method_kwargs = {**default_kwargs, **kwargs} api_response = stream( api_client.connect_get_namespaced_pod_exec, - name=pod_name, + name=name, namespace=namespace, - container=container_name, - command=exec_command, - stderr=True, - stdin=True, - stdout=True, - tty=False, - **kube_kwargs + container=container, + command=command, + **method_kwargs, ) return api_response diff --git a/tests/conftest.py b/tests/conftest.py index ad05baf..8c585a7 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,13 +1,27 @@ -from unittest.mock import MagicMock +import json +from pathlib import Path +from unittest import mock import pytest +import yaml +from prefect.blocks.kubernetes import KubernetesClusterConfig -from prefect_kubernetes.credentials import KubernetesApiKey +from prefect_kubernetes.credentials import KubernetesCredentials + +CERT_DUMMY = b"dummy" + +BASEDIR = Path("tests") +GOOD_CONFIG_FILE_PATH = BASEDIR / "kube_config.yaml" + + +@pytest.fixture +def kube_config_dict(): + return yaml.safe_load(GOOD_CONFIG_FILE_PATH.read_text()) @pytest.fixture def successful_job_status(): - job_status = MagicMock() + job_status = mock.MagicMock() job_status.status.active = None job_status.status.failed = None job_status.status.succeeded = 1 @@ -15,17 +29,24 @@ def successful_job_status(): @pytest.fixture -def kubernetes_api_key(): - return KubernetesApiKey(api_key="XXXX") +def kubernetes_credentials(kube_config_dict): + + return KubernetesCredentials( + # cluster_config=KubernetesClusterConfig( + # context_name="test", + # config=kube_config_dict + # ) + cluster_config=KubernetesClusterConfig.load("kube-config") + ) @pytest.fixture def api_app_client(monkeypatch): - app_client = MagicMock() + app_client = mock.MagicMock() monkeypatch.setattr( - "prefect_kubernetes.credentials.KubernetesApiKey.get_app_client", - MagicMock(return_value=app_client), + "prefect_kubernetes.credentials.KubernetesCredentials.get_app_client", + mock.MagicMock(return_value=app_client), ) return app_client @@ -33,11 +54,11 @@ def api_app_client(monkeypatch): @pytest.fixture def api_batch_client(monkeypatch): - batch_client = MagicMock() + batch_client = mock.MagicMock() monkeypatch.setattr( - "prefect_kubernetes.credentials.KubernetesApiKey.get_batch_client", - MagicMock(return_value=batch_client), + "prefect_kubernetes.credentials.KubernetesCredentials.get_batch_client", + mock.MagicMock(return_value=batch_client), ) return batch_client @@ -45,24 +66,35 @@ def api_batch_client(monkeypatch): @pytest.fixture def api_core_client(monkeypatch): - core_client = MagicMock() + core_client = mock.MagicMock() monkeypatch.setattr( - "prefect_kubernetes.credentials.KubernetesApiKey.get_core_client", - MagicMock(return_value=core_client), + "prefect_kubernetes.credentials.KubernetesCredentials.get_core_client", + mock.MagicMock(return_value=core_client), ) return core_client @pytest.fixture -def api_request_method(monkeypatch): - method = MagicMock() - method.__self__ = api_core_client +def mock_ApiClient(monkeypatch): + + mock_ApiClient = mock.MagicMock() + mock_response = mock.MagicMock() + + mock_response.status = mock.PropertyMock(return_value=200) + + mock_response.data = mock.PropertyMock( + return_value=json.dumps( + {"token_endpoint": "https://example.org/identity/token"} + ) + ) + + mock_ApiClient.return_value = mock_response monkeypatch.setattr( - "kubernetes.client.api_client.ApiClient.request", - "method", - method, + "kubernetes.config.kube_config.ApiClient.request", + mock_ApiClient, ) - return method + + return mock_ApiClient diff --git a/tests/kube_config.yaml b/tests/kube_config.yaml new file mode 100644 index 0000000..b9c1f58 --- /dev/null +++ b/tests/kube_config.yaml @@ -0,0 +1,13 @@ +apiVersion: v1 +clusters: +- cluster: {server: 'https://localhost:9443'} + name: test +contexts: +- context: {cluster: test, user: test} + name: test +current-context: test +kind: Config +preferences: {} +users: +- name: test + user: {token: testtoken} \ No newline at end of file diff --git a/tests/test_pod.py b/tests/test_pod.py index 3fe4d05..0e27e93 100644 --- a/tests/test_pod.py +++ b/tests/test_pod.py @@ -1,17 +1,41 @@ -import pytest - from prefect_kubernetes.pod import connect_get_namespaced_pod_exec -async def test_connect_get_namespaced_pod_exec(kubernetes_api_key): +async def test_connect_get_namespaced_pod_exec(kubernetes_credentials, mock_ApiClient): + + response = await connect_get_namespaced_pod_exec.fn( + name="demo-pod", + container="app-container", + command=["whoami"], + kubernetes_credentials=kubernetes_credentials, + ) + + assert response == "root\n" + + +async def test_connect_get_namespaced_pod_exec_multiline_cmd( + kubernetes_credentials, mock_ApiClient +): + + response = await connect_get_namespaced_pod_exec.fn( + name="demo-pod", + container="app-container", + command=["ls", "/"], + kubernetes_credentials=kubernetes_credentials, + ) + assert len(response.splitlines()) == 20 + + +async def test_connect_get_namespaced_pod_exec_no_stdout( + kubernetes_credentials, mock_ApiClient +): response = await connect_get_namespaced_pod_exec.fn( - pod_name="test_pod", - container_name="test_container", - exec_command=["whoami"], - namespace="test_ns", - kubernetes_api_key=kubernetes_api_key, - kube_kwargs={}, + name="demo-pod", + container="app-container", + command=["whoami"], + stdout=False, + kubernetes_credentials=kubernetes_credentials, ) - assert 1 == response.splitlines() + assert response == "" From 7b876abde0a3fee01fcead865ec1c8c5e3bb2aaa Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 5 Sep 2022 15:30:56 -0500 Subject: [PATCH 06/43] add better type hinting for connect_get_namespaced_pod_exec and exception handling --- prefect_kubernetes/exceptions.py | 7 +++ prefect_kubernetes/pod.py | 85 +++++++++++++++++++------ tests/test_job.py | 37 +++++------ tests/test_pod.py | 105 +++++++++++++++++++++++-------- 4 files changed, 170 insertions(+), 64 deletions(-) diff --git a/prefect_kubernetes/exceptions.py b/prefect_kubernetes/exceptions.py index 149a137..52959ec 100644 --- a/prefect_kubernetes/exceptions.py +++ b/prefect_kubernetes/exceptions.py @@ -1,6 +1,13 @@ +from kubernetes.client.exceptions import ApiException + + class KubernetesJobDefinitionError(Exception): """An exception to raise when a Kubernetes job definition is invalid""" class KubernetesJobFailedError(Exception): """An exception to raise when a job orchestrated by a prefect_kubernetes task fails""" + + +class KubernetesResourceNotFoundError(ApiException): + """An exception to raise when a Kubernetes resource cannot be found by an api client""" diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index 822c192..916987b 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -1,11 +1,16 @@ -from typing import Any, Callable, Dict, List, Optional +import asyncio +from typing import Callable, List, Optional, Union from kubernetes.client.rest import ApiException from kubernetes.stream import stream +from kubernetes.stream.ws_client import WSClient from kubernetes.watch import Watch -from prefect import task +from prefect import get_run_logger, task from prefect_kubernetes.credentials import KubernetesCredentials +from prefect_kubernetes.exceptions import KubernetesResourceNotFoundError + +KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE = 0 @task @@ -67,14 +72,22 @@ async def connect_get_namespaced_pod_exec( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", **kwargs, -) -> str: - """Task for running a command in a namespaced pod on Kubernetes. +) -> Union[str, WSClient]: + """Task for running and/or streaming commands in a namespaced pod on Kubernetes. This task requires `KubernetesCredentials` to generate a`CoreV1Api` Kubernetes - client to stream the overridden `api_response` to `connect_get_namespaced_pod_exec`. + client to run / stream commands on the specified pod's `container`. + + User-provided `kwargs` will overwrite `default_kwargs` if keys exist in both. + + The `kubernetes.stream.stream` object accepts a `_preload_content` kwarg (defualts to True) which + determines this task's return value type. + + If `_preload_content=True`, `api_response` will be the `str` output of `command` on `container`. + Otherwise if `_preload_content=False`, `api_response` will be an interactive `WSClient` object. - User-provided `kwargs` will overwrite `default_kwargs` if key values from `default_kwargs` - are set in `kwargs`. + Note that since `WSClient` is a non-pickleable object-type, it cannot be used as the `return` value + of a @flow-decorated function definition. Args: name (str): The name of the pod in which the command is to be run @@ -83,17 +96,22 @@ async def connect_get_namespaced_pod_exec( kubernetes_credentials (KubernetesCredentials): A block that stores a Kubernetes credentials, has methods to generate resource-specific client namespace (str, optional): The Kubernetes namespace of the pod. - Defaults to `default` + Defaults to `default` kwargs (Dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API method (e.g. `{"stderr": "False", "tty": "True"}`) + Kubernetes API method (e.g. `{"stderr": "False", "tty": "True"}`) Returns: - str: The string output of `command` on `container`, will be empty if `stdout=False` + Union[str, WSClient]: This task either returns the `str` output of `command`, or if + `_preload_content=False`, then an interactive `WSClient` object is returned. Raises: - - TypeError: `command` is not a list + - TypeError: `command` is not a list, or `api_response` is of unexpected type + - KubernetesResourceNotFoundError: if `api_response` has KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE + - ApiException: if bad `api_response` status and is not KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE """ + logger = get_run_logger() + if not isinstance(command, List): raise TypeError("The `command` argument must be provided as a list") @@ -108,13 +126,40 @@ async def connect_get_namespaced_pod_exec( method_kwargs = {**default_kwargs, **kwargs} - api_response = stream( - api_client.connect_get_namespaced_pod_exec, - name=name, - namespace=namespace, - container=container, - command=command, - **method_kwargs, - ) + try: + api_response = stream( + api_client.connect_get_namespaced_pod_exec, + name=name, + namespace=namespace, + container=container, + command=command, + **method_kwargs, + ) - return api_response + if isinstance(api_response, str): + logger.info( + f"Returning `str` output of '{' '.join(command)}' as executed on {container}..." + ) + elif isinstance(api_response, WSClient): + logger.info( + f"Returning an interactive `kubernetes.stream.ws_client.WSClient` object..." + ) + else: + raise TypeError( + f"Unexpected API response object-type: {type(api_response)}" + ) + + return api_response + + except ApiException as err: + if err.status == KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE: + raise KubernetesResourceNotFoundError( + status=404, + reason=( + f"{err.reason}" + " - Your Kubernetes API client cannot find a resource you specified." + ), + ) + else: + logger.error(f"{err.reason}") + raise diff --git a/tests/test_job.py b/tests/test_job.py index 7e2351c..f1ec251 100644 --- a/tests/test_job.py +++ b/tests/test_job.py @@ -1,5 +1,4 @@ import pytest -from prefect import flow from prefect_kubernetes import exceptions as err from prefect_kubernetes.job import ( @@ -13,77 +12,79 @@ ) -async def test_invalid_body_raises_error(kubernetes_api_key): +async def test_invalid_body_raises_error(kubernetes_credentials): with pytest.raises(err.KubernetesJobDefinitionError): - await create_namespaced_job.fn(body=None, kubernetes_api_key=kubernetes_api_key) + await create_namespaced_job.fn( + body=None, kubernetes_credentials=kubernetes_credentials + ) with pytest.raises(err.KubernetesJobDefinitionError): await patch_namespaced_job.fn( - body=None, job_name="", kubernetes_api_key=kubernetes_api_key + body=None, job_name="", kubernetes_credentials=kubernetes_credentials ) -async def test_create_namespaced_job(kubernetes_api_key, api_batch_client): +async def test_create_namespaced_job(kubernetes_credentials, api_batch_client): await create_namespaced_job.fn( body={"test": "a"}, kube_kwargs={"a": "test"}, - kubernetes_api_key=kubernetes_api_key, + kubernetes_credentials=kubernetes_credentials, ) assert api_batch_client.create_namespaced_job.call_args[1]["body"] == {"test": "a"} assert api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" -async def test_delete_namespaced_job(kubernetes_api_key, api_batch_client): +async def test_delete_namespaced_job(kubernetes_credentials, api_batch_client): await delete_namespaced_job.fn( job_name="test_job", kube_kwargs={"a": "test"}, - kubernetes_api_key=kubernetes_api_key, + kubernetes_credentials=kubernetes_credentials, ) assert api_batch_client.delete_namespaced_job.call_args[1]["name"] == "test_job" assert api_batch_client.delete_namespaced_job.call_args[1]["a"] == "test" -async def test_list_namespaced_job(kubernetes_api_key, api_batch_client): +async def test_list_namespaced_job(kubernetes_credentials, api_batch_client): await list_namespaced_job.fn( namespace="ns", kube_kwargs={"a": "test"}, - kubernetes_api_key=kubernetes_api_key, + kubernetes_credentials=kubernetes_credentials, ) assert api_batch_client.list_namespaced_job.call_args[1]["namespace"] == "ns" assert api_batch_client.list_namespaced_job.call_args[1]["a"] == "test" -async def test_patch_namespaced_job(kubernetes_api_key, api_batch_client): +async def test_patch_namespaced_job(kubernetes_credentials, api_batch_client): await patch_namespaced_job.fn( body={"test": "a"}, job_name="test_job", kube_kwargs={"a": "test"}, - kubernetes_api_key=kubernetes_api_key, + kubernetes_credentials=kubernetes_credentials, ) assert api_batch_client.patch_namespaced_job.call_args[1]["body"] == {"test": "a"} assert api_batch_client.patch_namespaced_job.call_args[1]["name"] == "test_job" assert api_batch_client.patch_namespaced_job.call_args[1]["a"] == "test" -async def test_read_namespaced_job(kubernetes_api_key, api_batch_client): +async def test_read_namespaced_job(kubernetes_credentials, api_batch_client): await read_namespaced_job.fn( job_name="test_job", namespace="ns", kube_kwargs={"a": "test"}, - kubernetes_api_key=kubernetes_api_key, + kubernetes_credentials=kubernetes_credentials, ) assert api_batch_client.read_namespaced_job.call_args[1]["name"] == "test_job" assert api_batch_client.read_namespaced_job.call_args[1]["namespace"] == "ns" assert api_batch_client.read_namespaced_job.call_args[1]["a"] == "test" -async def test_replace_namespaced_job(kubernetes_api_key, api_batch_client): +async def test_replace_namespaced_job(kubernetes_credentials, api_batch_client): await replace_namespaced_job.fn( job_name="test_job", body={"test": "a"}, namespace="ns", kube_kwargs={"a": "test"}, - kubernetes_api_key=kubernetes_api_key, + kubernetes_credentials=kubernetes_credentials, ) assert api_batch_client.replace_namespaced_job.call_args[1]["name"] == "test_job" assert api_batch_client.replace_namespaced_job.call_args[1]["namespace"] == "ns" @@ -92,7 +93,7 @@ async def test_replace_namespaced_job(kubernetes_api_key, api_batch_client): # async def test_run_namespaced_job( -# kubernetes_api_key, api_batch_client, api_core_client +# kubernetes_credentials, api_batch_client, api_core_client # ): # @flow # def test_flow(): @@ -100,7 +101,7 @@ async def test_replace_namespaced_job(kubernetes_api_key, api_batch_client): # body={"metadata": {"name": "test"}}, # namespace="ns", # kube_kwargs={"a": "test"}, -# kubernetes_api_key=kubernetes_api_key, +# kubernetes_credentials=kubernetes_credentials, # log_level="DEBUG", # ) diff --git a/tests/test_pod.py b/tests/test_pod.py index 0e27e93..4ededd5 100644 --- a/tests/test_pod.py +++ b/tests/test_pod.py @@ -1,41 +1,94 @@ +from urllib import response + +import pytest +from kubernetes.client.exceptions import ApiException +from kubernetes.stream.ws_client import WSClient +from prefect import flow +from prefect.orion.schemas.states import Completed + from prefect_kubernetes.pod import connect_get_namespaced_pod_exec -async def test_connect_get_namespaced_pod_exec(kubernetes_credentials, mock_ApiClient): +async def test_connect_get_namespaced_pod_exec_str_return(kubernetes_credentials): + @flow + async def test_flow(): + return await connect_get_namespaced_pod_exec( + name="demo-pod", + container="app-container", + command=["whoami"], + kubernetes_credentials=kubernetes_credentials, + ) - response = await connect_get_namespaced_pod_exec.fn( - name="demo-pod", - container="app-container", - command=["whoami"], - kubernetes_credentials=kubernetes_credentials, - ) + response = await test_flow() + assert isinstance(response, str) assert response == "root\n" -async def test_connect_get_namespaced_pod_exec_multiline_cmd( - kubernetes_credentials, mock_ApiClient -): +async def test_connect_get_namespaced_pod_exec_stream_return(kubernetes_credentials): + @flow + async def test_flow(): + response = await connect_get_namespaced_pod_exec( + name="demo-pod", + container="app-container", + command=["/bin/bash"], + kubernetes_credentials=kubernetes_credentials, + _preload_content=False, + ) + # test_flow` failing because cannot `return` non-pickleable type `SSLSocket` ? + assert isinstance(response, WSClient) + return None + + try: + await test_flow() + except TypeError as e: + if "cannot pickle 'SSLSocket' object" in str(e): + pass + else: + raise + + +async def test_connect_get_namespaced_pod_exec_multiline_cmd(kubernetes_credentials): + @flow + async def test_flow(): + return await connect_get_namespaced_pod_exec( + name="demo-pod", + container="app-container", + command=["ls", "/"], + kubernetes_credentials=kubernetes_credentials, + ) + + response = await test_flow() - response = await connect_get_namespaced_pod_exec.fn( - name="demo-pod", - container="app-container", - command=["ls", "/"], - kubernetes_credentials=kubernetes_credentials, - ) assert len(response.splitlines()) == 20 -async def test_connect_get_namespaced_pod_exec_no_stdout( - kubernetes_credentials, mock_ApiClient -): +async def test_connect_get_namespaced_pod_exec_no_stdout(kubernetes_credentials): + @flow + async def test_flow(): + return await connect_get_namespaced_pod_exec( + name="demo-pod", + container="app-container", + command=["whoami"], + stdout=False, + kubernetes_credentials=kubernetes_credentials, + ) - response = await connect_get_namespaced_pod_exec.fn( - name="demo-pod", - container="app-container", - command=["whoami"], - stdout=False, - kubernetes_credentials=kubernetes_credentials, - ) + response = await test_flow() assert response == "" + + +async def test_connect_get_namespaced_pod_exec_fake_pod(kubernetes_credentials): + @flow + async def test_flow(): + return await connect_get_namespaced_pod_exec( + name="non-existent-pod", + container="app-container", + command=["whoami"], + stdout=False, + kubernetes_credentials=kubernetes_credentials, + ) + + with pytest.raises(ApiException): + await test_flow() From e3a273ae9533c973869dd0d6f1185e76b2b79a3f Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 5 Sep 2022 16:09:59 -0500 Subject: [PATCH 07/43] allow selection of KubernetesClusterConfig in KubernetesCredentials specification --- docs/utilities.md | 1 - prefect_kubernetes/credentials.py | 21 +++++++++++---------- prefect_kubernetes/pod.py | 6 +++--- tests/test_pod.py | 10 +++++----- 4 files changed, 19 insertions(+), 19 deletions(-) delete mode 100644 docs/utilities.md diff --git a/docs/utilities.md b/docs/utilities.md deleted file mode 100644 index 7c5645f..0000000 --- a/docs/utilities.md +++ /dev/null @@ -1 +0,0 @@ -:::prefect_kubernetes.utilities \ No newline at end of file diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 54918d8..1f0ff76 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -22,13 +22,15 @@ class KubernetesCredentials(Block): - """Credentials block for API client generation across prefect-kubernetes tasks and flows. + """Credentials block for authenticated Kubernetes API client generation. Args: - api_key (SecretStr): API key to authenticate with the Kubernetes API. + api_key (SecretStr): API key to authenticate with the Kubernetes API + cluster_config (KubernetesClusterConfig, optional): an instance of `KubernetesClusterConfig` + holding a JSON kube config for a specific kubernetes context Examples: - Load a stored kubernetes API key: + Load stored Kubernetes credentials: ```python from prefect_kubernetes.credentials import KubernetesCredentials @@ -38,21 +40,20 @@ class KubernetesCredentials(Block): Create a kubernetes API client from KubernetesCredentials and inferred cluster configuration: ```python from prefect_kubernetes import KubernetesCredentials - from prefect_kubernetes.utilities import get_kubernetes_client - kubernetes_credentials = KubernetesCredentials.load("my-k8s-api-key") + kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") kubernetes_api_client = kubernetes_credentials.get_core_client() ``` Create a namespaced kubernetes job: ```python - from prefect_kubernetes import KubernetesApiKey + from prefect_kubernetes import KubernetesCredentials from prefect_kubernetes.job import create_namespaced_job - kubernetes_credentials = KubernetesApiKey.load("my-k8s-api-key") + kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") create_namespaced_job( - namespace="default", body={"Marvin": "42"}, **kube_kwargs + body={"Marvin": "42"}, kubernetes_credentials=kubernetes_credentials ) ``` """ @@ -60,8 +61,8 @@ class KubernetesCredentials(Block): _block_type_name = "Kubernetes Credentials" _logo_url = "https://kubernetes-security.info/assets/img/logo.png?h=250" # noqa - api_key: SecretStr = None - cluster_config: KubernetesClusterConfig = None + api_key: Optional[SecretStr] = None + cluster_config: Optional[KubernetesClusterConfig] = None def get_core_client(self) -> client.CoreV1Api: """Convenience method for retrieving a kubernetes api client for core resources diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index 916987b..4f26a75 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -66,7 +66,7 @@ async def read_namespaced_pod_logs( @task async def connect_get_namespaced_pod_exec( - name: str, + pod_name: str, container: str, command: List[str], kubernetes_credentials: KubernetesCredentials, @@ -90,7 +90,7 @@ async def connect_get_namespaced_pod_exec( of a @flow-decorated function definition. Args: - name (str): The name of the pod in which the command is to be run + pod_name (str): The name of the pod in which the command is to be run container (str): The name of a container to use in the pod. command (List): The command to run in `pod_name` kubernetes_credentials (KubernetesCredentials): A block that stores a Kubernetes credentials, @@ -129,7 +129,7 @@ async def connect_get_namespaced_pod_exec( try: api_response = stream( api_client.connect_get_namespaced_pod_exec, - name=name, + name=pod_name, namespace=namespace, container=container, command=command, diff --git a/tests/test_pod.py b/tests/test_pod.py index 4ededd5..21bdd79 100644 --- a/tests/test_pod.py +++ b/tests/test_pod.py @@ -13,7 +13,7 @@ async def test_connect_get_namespaced_pod_exec_str_return(kubernetes_credentials @flow async def test_flow(): return await connect_get_namespaced_pod_exec( - name="demo-pod", + pod_name="demo-pod", container="app-container", command=["whoami"], kubernetes_credentials=kubernetes_credentials, @@ -29,7 +29,7 @@ async def test_connect_get_namespaced_pod_exec_stream_return(kubernetes_credenti @flow async def test_flow(): response = await connect_get_namespaced_pod_exec( - name="demo-pod", + pod_name="demo-pod", container="app-container", command=["/bin/bash"], kubernetes_credentials=kubernetes_credentials, @@ -52,7 +52,7 @@ async def test_connect_get_namespaced_pod_exec_multiline_cmd(kubernetes_credenti @flow async def test_flow(): return await connect_get_namespaced_pod_exec( - name="demo-pod", + pod_name="demo-pod", container="app-container", command=["ls", "/"], kubernetes_credentials=kubernetes_credentials, @@ -67,7 +67,7 @@ async def test_connect_get_namespaced_pod_exec_no_stdout(kubernetes_credentials) @flow async def test_flow(): return await connect_get_namespaced_pod_exec( - name="demo-pod", + pod_name="demo-pod", container="app-container", command=["whoami"], stdout=False, @@ -83,7 +83,7 @@ async def test_connect_get_namespaced_pod_exec_fake_pod(kubernetes_credentials): @flow async def test_flow(): return await connect_get_namespaced_pod_exec( - name="non-existent-pod", + pod_name="non-existent-pod", container="app-container", command=["whoami"], stdout=False, From 8532bccad2ed3ea8ddbd1a15c567b35bb2d65c0e Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 5 Sep 2022 16:21:30 -0500 Subject: [PATCH 08/43] docstring changes --- prefect_kubernetes/credentials.py | 22 ++++++------ prefect_kubernetes/job.py | 57 +++++++++++-------------------- prefect_kubernetes/pod.py | 4 +-- 3 files changed, 31 insertions(+), 52 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 1f0ff76..34d2d6d 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -1,4 +1,3 @@ -from distutils.command.config import config from typing import TYPE_CHECKING, Optional, Union from kubernetes import client @@ -26,8 +25,8 @@ class KubernetesCredentials(Block): Args: api_key (SecretStr): API key to authenticate with the Kubernetes API - cluster_config (KubernetesClusterConfig, optional): an instance of `KubernetesClusterConfig` - holding a JSON kube config for a specific kubernetes context + cluster_config (KubernetesClusterConfig, optional): a `KubernetesClusterConfig` + block holding a JSON kube config for a specific kubernetes context Examples: Load stored Kubernetes credentials: @@ -37,7 +36,7 @@ class KubernetesCredentials(Block): kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") ``` - Create a kubernetes API client from KubernetesCredentials and inferred cluster configuration: + Create an API client from KubernetesCredentials and inferred cluster configuration: ```python from prefect_kubernetes import KubernetesCredentials @@ -45,7 +44,7 @@ class KubernetesCredentials(Block): kubernetes_api_client = kubernetes_credentials.get_core_client() ``` - Create a namespaced kubernetes job: + Create a namespaced job: ```python from prefect_kubernetes import KubernetesCredentials from prefect_kubernetes.job import create_namespaced_job @@ -68,7 +67,8 @@ def get_core_client(self) -> client.CoreV1Api: """Convenience method for retrieving a kubernetes api client for core resources Returns: - client.CoreV1Api: Kubernetes api client to interact with "pod", "service" and "secret" resources + client.CoreV1Api: Kubernetes api client to interact with "pod", "service" + and "secret" resources """ return self.get_kubernetes_client(resource="core") @@ -114,16 +114,16 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: resource_specific_client = K8S_CLIENTS[resource] - if self.api_key: + if self.api_key: # this case is not yet working as expected configuration = client.Configuration() configuration.api_key["authorization"] = self.api_key.get_secret_value() configuration.api_key_prefix["authorization"] = "Bearer" - k8s_client = resource_specific_client( + return resource_specific_client( api_client=client.ApiClient(configuration=configuration) ) elif self.cluster_config: self.cluster_config.configure_client() - k8s_client = resource_specific_client() + return resource_specific_client() else: try: print("Trying to load in-cluster configuration...") @@ -133,6 +133,4 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: print("Loading out-of-cluster configuration...") kube_config.load_kube_config() - k8s_client = resource_specific_client() - - return k8s_client + return resource_specific_client() diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py index bd4f921..d0a1899 100644 --- a/prefect_kubernetes/job.py +++ b/prefect_kubernetes/job.py @@ -22,26 +22,16 @@ async def create_namespaced_job( Args: body (dict): A dictionary representation of a Kubernetes V1Job - specification. Defaults to None. + specification. namespace (str, optional): The Kubernetes namespace to create this job in, defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block holding a Kubernetes API Key. Defaults to None. + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - - Raises: - ValueError: if `body` is `None` + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. """ - if not body: - raise err.KubernetesJobDefinitionError( - "A dictionary representing a V1Job must be provided." - ) - api_client = kubernetes_credentials.get_batch_client() - body = {**body, **(body or {})} - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} - api_client.create_namespaced_job(namespace=namespace, body=body, **kube_kwargs) @@ -59,32 +49,23 @@ async def delete_namespaced_job( job_name (str): The name of a job to delete. Defaults to None. namespace (str, optional): The Kubernetes namespace to delete this job in, defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block - holding a Kubernetes API Key. Defaults to None. + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. - delete_option_kwargs (dict, optional): Optional keyword arguments to pass to the V1DeleteOptions object (e.g. {"propagation_policy": "...", "grace_period_seconds": "..."}. - - Raises: - ValueError: if `job_name` is `None` """ - if not job_name: - raise ValueError("The name of a Kubernetes job must be provided.") api_client = kubernetes_credentials.get_batch_client() - method_kwargs = {} + kwargs = kube_kwargs or {} if delete_option_kwargs: - method_kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) - if kube_kwargs: - method_kwargs.update(**kube_kwargs) + kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) - api_client.delete_namespaced_job( - name=job_name, namespace=namespace, **method_kwargs - ) + api_client.delete_namespaced_job(name=job_name, namespace=namespace, **kwargs) @task @@ -96,10 +77,10 @@ async def list_namespaced_job( """Task for listing namespaced Kubernetes jobs. Args: + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. namespace (str, optional): The Kubernetes namespace to list jobs from, defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block - holding a Kubernetes API Key kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. """ @@ -129,8 +110,8 @@ async def patch_namespaced_job( specification. Defaults to None. namespace (str, optional): The Kubernetes namespace to patch this job in, defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block - holding a Kubernetes API Key. Defaults to None. + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -168,8 +149,8 @@ async def read_namespaced_job( job_name (str): The name of a job to read. Defaults to None. namespace (str, optional): The Kubernetes namespace to read this job in, defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block - holding a Kubernetes API Key. Defaults to None. + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -204,8 +185,8 @@ async def replace_namespaced_job( job_name (str): The name of a job to replace. Defaults to None. namespace (str, optional): The Kubernetes namespace to replace this job in, defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block - holding a Kubernetes API Key. Defaults to None. + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. @@ -248,8 +229,8 @@ async def run_namespaced_job( specification. Defaults to None. namespace (str, optional): The Kubernetes namespace to run this job in, defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials, optional): KubernetesCredentials block - holding a Kubernetes API Key. Defaults to None. + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. job_status_poll_interval (int, optional): The interval given in seconds diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index 4f26a75..02ef9f0 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -124,7 +124,7 @@ async def connect_get_namespaced_pod_exec( tty=False, ) - method_kwargs = {**default_kwargs, **kwargs} + api_method_kwargs = {**default_kwargs, **kwargs} try: api_response = stream( @@ -133,7 +133,7 @@ async def connect_get_namespaced_pod_exec( namespace=namespace, container=container, command=command, - **method_kwargs, + **api_method_kwargs, ) if isinstance(api_response, str): From 58df11b32fb65617b00a12c1199d5b34d5b5200b Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 5 Sep 2022 16:34:06 -0500 Subject: [PATCH 09/43] rm iterable defaults, fix corresponding tests --- prefect_kubernetes/job.py | 55 ++++++++++----------------------------- tests/test_job.py | 6 ++--- tests/test_pod.py | 2 +- 3 files changed, 18 insertions(+), 45 deletions(-) diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py index d0a1899..4d13a7e 100644 --- a/prefect_kubernetes/job.py +++ b/prefect_kubernetes/job.py @@ -52,7 +52,7 @@ async def delete_namespaced_job( kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - delete_option_kwargs (dict, optional): Optional keyword arguments to pass to the V1DeleteOptions object (e.g. {"propagation_policy": "...", "grace_period_seconds": "..."}. @@ -82,7 +82,7 @@ async def list_namespaced_job( namespace (str, optional): The Kubernetes namespace to list jobs from, defaults to the `default` namespace. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. """ api_client = kubernetes_credentials.get_batch_client() @@ -100,7 +100,7 @@ async def patch_namespaced_job( body: dict, kubernetes_credentials: KubernetesCredentials = None, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = {}, + kube_kwargs: Optional[Dict] = None, ): """Task for deleting a namespaced Kubernetes job. @@ -113,23 +113,15 @@ async def patch_namespaced_job( kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. Raises: ValueError: if `job_name` is `None` """ - if not body: - raise err.KubernetesJobDefinitionError( - "A dictionary representing a V1Job must be provided." - ) - - if not job_name: - raise ValueError("The name of a Kubernetes job must be provided.") api_client = kubernetes_credentials.get_batch_client() - body = {**body, **(body or {})} - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + kube_kwargs = kube_kwargs or {} api_client.patch_namespaced_job( name=job_name, namespace=namespace, body=body, **kube_kwargs @@ -141,7 +133,7 @@ async def read_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = {}, + kube_kwargs: Optional[Dict] = None, ): """Task for reading a namespaced kubernetes job. @@ -152,7 +144,7 @@ async def read_namespaced_job( kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. Raises: ValueError: if `job_name` is `None` @@ -162,7 +154,7 @@ async def read_namespaced_job( api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + kube_kwargs = kube_kwargs or {} return api_client.read_namespaced_job( name=job_name, namespace=namespace, **kube_kwargs @@ -175,7 +167,7 @@ async def replace_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = {}, + kube_kwargs: Optional[Dict] = None, ): """Task for replacing a namespaced kubernetes job. @@ -188,24 +180,11 @@ async def replace_namespaced_job( kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. - - Raises: - ValueError: if `body` is `None` - ValueError: if `job_name` is `None` + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. """ - if not body: - raise err.KubernetesJobDefinitionError( - "A dictionary representing a V1Job must be provided." - ) - - if not job_name: - raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_credentials.get_batch_client() - body = {**body, **(body or {})} - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + kube_kwargs = kube_kwargs or {} return api_client.replace_namespaced_job( name=job_name, body=body, namespace=namespace, **kube_kwargs @@ -217,7 +196,7 @@ async def run_namespaced_job( body: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = {}, + kube_kwargs: Optional[Dict] = None, job_status_poll_interval: Optional[int] = 5, log_level: Optional[str] = None, delete_job_after_completion: Optional[bool] = True, @@ -232,7 +211,7 @@ async def run_namespaced_job( kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to {}. + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. job_status_poll_interval (int, optional): The interval given in seconds indicating how often the Kubernetes API will be requested about the status of the job being performed, defaults to `5` seconds. @@ -248,16 +227,10 @@ async def run_namespaced_job( """ logger = get_run_logger().setLevel(level=log_level) - if not body: - raise err.KubernetesJobDefinitionError( - "A dictionary representing a V1Job must be provided." - ) - # if log_level is not None and getattr(logger, log_level, None) is None: # raise ValueError("A valid log_level must be provided.") - body = {**body, **(body or {})} - kube_kwargs = {**kube_kwargs, **(kube_kwargs or {})} + kube_kwargs = kube_kwargs or {} job_name = body.get("metadata", {}).get("name") diff --git a/tests/test_job.py b/tests/test_job.py index f1ec251..c980fb6 100644 --- a/tests/test_job.py +++ b/tests/test_job.py @@ -1,6 +1,6 @@ import pytest +from kubernetes.client.exceptions import ApiValueError -from prefect_kubernetes import exceptions as err from prefect_kubernetes.job import ( create_namespaced_job, delete_namespaced_job, @@ -13,11 +13,11 @@ async def test_invalid_body_raises_error(kubernetes_credentials): - with pytest.raises(err.KubernetesJobDefinitionError): + with pytest.raises(TypeError): await create_namespaced_job.fn( body=None, kubernetes_credentials=kubernetes_credentials ) - with pytest.raises(err.KubernetesJobDefinitionError): + with pytest.raises(ApiValueError): await patch_namespaced_job.fn( body=None, job_name="", kubernetes_credentials=kubernetes_credentials ) diff --git a/tests/test_pod.py b/tests/test_pod.py index 21bdd79..4bcd2d4 100644 --- a/tests/test_pod.py +++ b/tests/test_pod.py @@ -35,7 +35,7 @@ async def test_flow(): kubernetes_credentials=kubernetes_credentials, _preload_content=False, ) - # test_flow` failing because cannot `return` non-pickleable type `SSLSocket` ? + # `test_flow` failing because cannot `return` non-pickleable type `SSLSocket` ? assert isinstance(response, WSClient) return None From 2eda391ceadf88a0b8452390eced51a56079a017 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 5 Sep 2022 21:25:12 -0500 Subject: [PATCH 10/43] cleaning up and comments --- prefect_kubernetes/credentials.py | 44 +++++++++++++++++++------------ prefect_kubernetes/pod.py | 40 +++++++++++++++++----------- tests/test_pod.py | 42 ++++++++++++++++++----------- 3 files changed, 78 insertions(+), 48 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 34d2d6d..bfb362c 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -21,13 +21,15 @@ class KubernetesCredentials(Block): - """Credentials block for authenticated Kubernetes API client generation. + """Credentials block for generating configured Kubernetes API clients. Args: - api_key (SecretStr): API key to authenticate with the Kubernetes API cluster_config (KubernetesClusterConfig, optional): a `KubernetesClusterConfig` block holding a JSON kube config for a specific kubernetes context + api_key (SecretStr): API key to authenticate with the Kubernetes API + + Examples: Load stored Kubernetes credentials: ```python @@ -36,12 +38,15 @@ class KubernetesCredentials(Block): kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") ``` - Create an API client from KubernetesCredentials and inferred cluster configuration: + Create resource-specific API clients from KubernetesCredentials: ```python from prefect_kubernetes import KubernetesCredentials kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") - kubernetes_api_client = kubernetes_credentials.get_core_client() + + kubernetes_app_v1_client = kubernetes_credentials.get_app_client() + kubernetes_batch_v1_client = kubernetes_credentials.get_batch_client() + kubernetes_core_v1_client = kubernetes_credentials.get_core_client() ``` Create a namespaced job: @@ -60,9 +65,10 @@ class KubernetesCredentials(Block): _block_type_name = "Kubernetes Credentials" _logo_url = "https://kubernetes-security.info/assets/img/logo.png?h=250" # noqa - api_key: Optional[SecretStr] = None cluster_config: Optional[KubernetesClusterConfig] = None + api_key: Optional[SecretStr] = None + def get_core_client(self) -> client.CoreV1Api: """Convenience method for retrieving a kubernetes api client for core resources @@ -94,11 +100,14 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: It will attempt to connect to a Kubernetes cluster in three steps with the first successful connection attempt becoming the mode of communication with a cluster. - 1. Attempt to use a KubernetesCredentials block containing a Kubernetes API Key. If - `kubernetes_credentials` = `None` then it will attempt the next two connection + 1. It will first attempt to use a `KubernetesCredentials` block's `cluster_config` to + configure a client using `KubernetesClusterConfig.configure_client` and then return the + `resource_specific_client`. + 2. Attempt to use a `KubernetesCredentials` block's `api_key`. If + `not self.api_key` then it will attempt the next two connection methods. - 2. Attempt in-cluster connection (will only work when running on a Pod in a cluster) - 3. Attempt out-of-cluster connection using the default location for a kube config file + 3. Attempt in-cluster connection (will only work when running on a Pod in a cluster) + 4. Attempt out-of-cluster connection using the default location for a kube config file In some cases connections to the kubernetes server are dropped after being idle for some time (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in ReadTimeoutErrors. @@ -109,21 +118,22 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: you can use one of these values: `job`, `pod`, `service`, `deployment`, `secret` Returns: - - KubernetesClient: an initialized, authenticated Kubernetes Client + - KubernetesClient: an initialized, configured Kubernetes Client """ resource_specific_client = K8S_CLIENTS[resource] - if self.api_key: # this case is not yet working as expected + if self.cluster_config: + self.cluster_config.configure_client() + return resource_specific_client() + + elif self.api_key: configuration = client.Configuration() configuration.api_key["authorization"] = self.api_key.get_secret_value() configuration.api_key_prefix["authorization"] = "Bearer" - return resource_specific_client( - api_client=client.ApiClient(configuration=configuration) - ) - elif self.cluster_config: - self.cluster_config.configure_client() - return resource_specific_client() + generic_client = client.ApiClient(configuration=configuration) + return resource_specific_client(api_client=generic_client) + else: try: print("Trying to load in-cluster configuration...") diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py index 02ef9f0..80e1fd2 100644 --- a/prefect_kubernetes/pod.py +++ b/prefect_kubernetes/pod.py @@ -1,4 +1,3 @@ -import asyncio from typing import Callable, List, Optional, Union from kubernetes.client.rest import ApiException @@ -71,43 +70,49 @@ async def connect_get_namespaced_pod_exec( command: List[str], kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", + interactive: bool = False, **kwargs, ) -> Union[str, WSClient]: """Task for running and/or streaming commands in a namespaced pod on Kubernetes. This task requires `KubernetesCredentials` to generate a`CoreV1Api` Kubernetes - client to run / stream commands on the specified pod's `container`. + client to run / stream the user `command` on a specified `container` of `pod_name`. - User-provided `kwargs` will overwrite `default_kwargs` if keys exist in both. + User-provided `kwargs` will overwrite `default_settings` if keys exist in both. The `kubernetes.stream.stream` object accepts a `_preload_content` kwarg (defualts to True) which - determines this task's return value type. + determines this task's return value type, configurable as the `interactive` bool parameter. - If `_preload_content=True`, `api_response` will be the `str` output of `command` on `container`. - Otherwise if `_preload_content=False`, `api_response` will be an interactive `WSClient` object. + If `interactive` is set to False (as is default), then `_preload_content=True`, + and `api_response` will be the str output of `command` on `container`. - Note that since `WSClient` is a non-pickleable object-type, it cannot be used as the `return` value - of a @flow-decorated function definition. + Else if `interactive` is set to True, `_preload_content=False`, and + `api_response` will be an interactive `WSClient` object. + + Note that since `WSClient` is a un-pickleable object-type, it cannot be used as the `return` value + of a `@flow`-decorated function definition. Args: pod_name (str): The name of the pod in which the command is to be run - container (str): The name of a container to use in the pod. + container (str): The name of a container to use in the pod command (List): The command to run in `pod_name` kubernetes_credentials (KubernetesCredentials): A block that stores a Kubernetes credentials, has methods to generate resource-specific client - namespace (str, optional): The Kubernetes namespace of the pod. + namespace (str, optional): The Kubernetes namespace of the pod Defaults to `default` + interactive (bool, optional): If `True` return the interactive Kubernetes websocket + client object `WSClient`, else the `str` output of `command`. Defaults to `False`. kwargs (Dict, optional): Optional extra keyword arguments to pass to the Kubernetes API method (e.g. `{"stderr": "False", "tty": "True"}`) Returns: - Union[str, WSClient]: This task either returns the `str` output of `command`, or if - `_preload_content=False`, then an interactive `WSClient` object is returned. + Union[str, WSClient]: This task either returns the str output of `command`, or if + `_preload_content=False`, then an interactive `WSClient` object is returned Raises: - TypeError: `command` is not a list, or `api_response` is of unexpected type - KubernetesResourceNotFoundError: if `api_response` has KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE - - ApiException: if bad `api_response` status and is not KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE + - ApiException: if bad `api_response` status but is not a KubernetesResourceNotFoundError """ logger = get_run_logger() @@ -117,14 +122,17 @@ async def connect_get_namespaced_pod_exec( api_client = kubernetes_credentials.get_core_client() - default_kwargs = dict( + default_settings = dict( stderr=True, stdin=True, stdout=True, tty=False, ) + # overwrite `default_settings` with user-defined `kwargs` and add any other `kwargs` provided + api_method_kwargs = {**default_settings, **kwargs} - api_method_kwargs = {**default_kwargs, **kwargs} + if interactive: + api_method_kwargs["_preload_content"] = False try: api_response = stream( @@ -146,7 +154,7 @@ async def connect_get_namespaced_pod_exec( ) else: raise TypeError( - f"Unexpected API response object-type: {type(api_response)}" + f"Unexpected API response object `{type(api_response)}`: {api_response!r}" ) return api_response diff --git a/tests/test_pod.py b/tests/test_pod.py index 4bcd2d4..ea47ffd 100644 --- a/tests/test_pod.py +++ b/tests/test_pod.py @@ -1,10 +1,7 @@ -from urllib import response - import pytest from kubernetes.client.exceptions import ApiException from kubernetes.stream.ws_client import WSClient from prefect import flow -from prefect.orion.schemas.states import Completed from prefect_kubernetes.pod import connect_get_namespaced_pod_exec @@ -28,24 +25,40 @@ async def test_flow(): async def test_connect_get_namespaced_pod_exec_stream_return(kubernetes_credentials): @flow async def test_flow(): - response = await connect_get_namespaced_pod_exec( + websocket_client = await connect_get_namespaced_pod_exec( + pod_name="demo-pod", + container="app-container", + command=["/bin/bash"], + kubernetes_credentials=kubernetes_credentials, + interactive=True, + ) + assert isinstance(websocket_client, WSClient) + assert websocket_client.is_open() + + return websocket_client + + # `test_flow` failing because cannot return a non-pickleable type `SSLSocket` ? + with pytest.raises(TypeError): + await test_flow() + + +async def test_connect_get_namespaced_pod_exec_stream_return_poweruser( + kubernetes_credentials, +): + @flow + async def test_flow(): + websocket_client = await connect_get_namespaced_pod_exec( pod_name="demo-pod", container="app-container", command=["/bin/bash"], kubernetes_credentials=kubernetes_credentials, - _preload_content=False, + _preload_content=False, # only someone who knows the client well would try this ) - # `test_flow` failing because cannot `return` non-pickleable type `SSLSocket` ? - assert isinstance(response, WSClient) - return None + assert isinstance(websocket_client, WSClient) + assert websocket_client.is_open() - try: + with pytest.raises(TypeError): await test_flow() - except TypeError as e: - if "cannot pickle 'SSLSocket' object" in str(e): - pass - else: - raise async def test_connect_get_namespaced_pod_exec_multiline_cmd(kubernetes_credentials): @@ -86,7 +99,6 @@ async def test_flow(): pod_name="non-existent-pod", container="app-container", command=["whoami"], - stdout=False, kubernetes_credentials=kubernetes_credentials, ) From a6d5e36cde5dbb0d6f2618abd6222f7c1c68284e Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 24 Oct 2022 12:49:59 -0500 Subject: [PATCH 11/43] save for checkpoint --- tests/test_pod.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/test_pod.py b/tests/test_pod.py index ea47ffd..44e9d42 100644 --- a/tests/test_pod.py +++ b/tests/test_pod.py @@ -38,8 +38,6 @@ async def test_flow(): return websocket_client # `test_flow` failing because cannot return a non-pickleable type `SSLSocket` ? - with pytest.raises(TypeError): - await test_flow() async def test_connect_get_namespaced_pod_exec_stream_return_poweruser( From 07d0dcd40c527481c4eec21e1622ca3d5e3cf19c Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 13:01:42 -0500 Subject: [PATCH 12/43] adds job tasks and tests --- docs/deployment.md | 1 - docs/jobs.md | 1 + docs/pod.md | 1 - docs/secrets.md | 1 - mkdocs.yml | 1 + prefect_kubernetes/credentials.py | 40 ++-- prefect_kubernetes/deployment.py | 0 prefect_kubernetes/exceptions.py | 8 +- prefect_kubernetes/job.py | 310 ---------------------------- prefect_kubernetes/jobs.py | 186 +++++++++++++++++ prefect_kubernetes/pod.py | 173 ---------------- prefect_kubernetes/secrets.py | 0 prefect_kubernetes/service.py | 0 tests/{test_job.py => test_jobs.py} | 30 +-- tests/test_pod.py | 104 ---------- 15 files changed, 217 insertions(+), 639 deletions(-) delete mode 100644 docs/deployment.md create mode 100644 docs/jobs.md delete mode 100644 docs/pod.md delete mode 100644 docs/secrets.md delete mode 100644 prefect_kubernetes/deployment.py delete mode 100644 prefect_kubernetes/job.py create mode 100644 prefect_kubernetes/jobs.py delete mode 100644 prefect_kubernetes/pod.py delete mode 100644 prefect_kubernetes/secrets.py delete mode 100644 prefect_kubernetes/service.py rename tests/{test_job.py => test_jobs.py} (77%) delete mode 100644 tests/test_pod.py diff --git a/docs/deployment.md b/docs/deployment.md deleted file mode 100644 index d327e07..0000000 --- a/docs/deployment.md +++ /dev/null @@ -1 +0,0 @@ -:::prefect_kubernetes.deployment \ No newline at end of file diff --git a/docs/jobs.md b/docs/jobs.md new file mode 100644 index 0000000..a8728be --- /dev/null +++ b/docs/jobs.md @@ -0,0 +1 @@ +:::prefect_kubernetes.jobs \ No newline at end of file diff --git a/docs/pod.md b/docs/pod.md deleted file mode 100644 index 5808b21..0000000 --- a/docs/pod.md +++ /dev/null @@ -1 +0,0 @@ -:::prefect_kubernetes.pod \ No newline at end of file diff --git a/docs/secrets.md b/docs/secrets.md deleted file mode 100644 index 18a73fc..0000000 --- a/docs/secrets.md +++ /dev/null @@ -1 +0,0 @@ -:::prefect_kubernetes.secrets \ No newline at end of file diff --git a/mkdocs.yml b/mkdocs.yml index 21fa38a..f289ffb 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -37,3 +37,4 @@ plugins: nav: - Home: index.md - Credentials: credentials.md + - Jobs: jobs.md diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index bfb362c..f2aba7f 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -1,11 +1,11 @@ -from typing import TYPE_CHECKING, Optional, Union +"""Module for defining Kubernetes credential handling and client generation.""" + +from typing import Optional, Union from kubernetes import client from kubernetes import config as kube_config from kubernetes.config.config_exception import ConfigException from prefect.blocks.core import Block - -# if TYPE_CHECKING: from prefect.blocks.kubernetes import KubernetesClusterConfig from pydantic import SecretStr @@ -90,7 +90,7 @@ def get_app_client(self) -> client.AppsV1Api: """Convenience method for retrieving a kubernetes api client for deployment resources Returns: - client.AppsV1Api: Kubernetes api client to interact with "deployment" resources + client.AppsV1Api: Kubernetes api client to interact with deployments """ return self.get_kubernetes_client(resource="deployment") @@ -98,24 +98,30 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: """ Utility function for loading kubernetes client object for a given resource. It will attempt to connect to a Kubernetes cluster in three steps with - the first successful connection attempt becoming the mode of communication with a - cluster. - 1. It will first attempt to use a `KubernetesCredentials` block's `cluster_config` to - configure a client using `KubernetesClusterConfig.configure_client` and then return the + the first successful connection attempt becoming the mode of communication with + a cluster: + 1. It will first attempt to use a `KubernetesCredentials` block's + `cluster_config` to configure a client using + `KubernetesClusterConfig.configure_client` and then return the `resource_specific_client`. + 2. Attempt to use a `KubernetesCredentials` block's `api_key`. If `not self.api_key` then it will attempt the next two connection methods. - 3. Attempt in-cluster connection (will only work when running on a Pod in a cluster) - 4. Attempt out-of-cluster connection using the default location for a kube config file - In some cases connections to the kubernetes server are dropped after being idle for some time - (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in - ReadTimeoutErrors. - In order to prevent that a periodic keep-alive message can be sent to the server to keep the - connection open. + + 3. Attempt in-cluster connection (will only work when running on a pod) + 4. Attempt out-of-cluster connection using the default location for a + kube config file. In some cases connections to the kubernetes server + are dropped after being idle for some time (e.g. Azure Firewall drops + idle connections after 4 minutes) which would result in ReadTimeoutErrors. + + In order to prevent that a periodic keep-alive message can be sent to the + server to keep the connection open. + Args: - - resource (str): the name of the resource to retrieve a client for. Currently - you can use one of these values: `job`, `pod`, `service`, `deployment`, `secret` + - resource (str): the name of the resource to retrieve a client for. + Currently you can use one of these values: `job`, `pod`, `service`, + `deployment`, and `secret`. Returns: - KubernetesClient: an initialized, configured Kubernetes Client diff --git a/prefect_kubernetes/deployment.py b/prefect_kubernetes/deployment.py deleted file mode 100644 index e69de29..0000000 diff --git a/prefect_kubernetes/exceptions.py b/prefect_kubernetes/exceptions.py index 52959ec..72b65ff 100644 --- a/prefect_kubernetes/exceptions.py +++ b/prefect_kubernetes/exceptions.py @@ -1,13 +1,15 @@ +"""Module to define common exceptions within `prefect_kubernetes`.""" + from kubernetes.client.exceptions import ApiException class KubernetesJobDefinitionError(Exception): - """An exception to raise when a Kubernetes job definition is invalid""" + """An exception for when a Kubernetes job definition is invalid""" class KubernetesJobFailedError(Exception): - """An exception to raise when a job orchestrated by a prefect_kubernetes task fails""" + """An exception for when a Kubernetes job fails""" class KubernetesResourceNotFoundError(ApiException): - """An exception to raise when a Kubernetes resource cannot be found by an api client""" + """An exception for when a Kubernetes resource cannot be found by a client""" diff --git a/prefect_kubernetes/job.py b/prefect_kubernetes/job.py deleted file mode 100644 index 4d13a7e..0000000 --- a/prefect_kubernetes/job.py +++ /dev/null @@ -1,310 +0,0 @@ -import time -from concurrent.futures import ThreadPoolExecutor -from typing import Dict, Optional - -import prefect.context -from kubernetes import client -from prefect import get_run_logger, task - -from prefect_kubernetes import exceptions as err -from prefect_kubernetes.credentials import KubernetesCredentials -from prefect_kubernetes.pod import read_namespaced_pod_logs - - -@task -async def create_namespaced_job( - body: Dict, - kubernetes_credentials: KubernetesCredentials, - namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, -): - """Task for creating a namespaced Kubernetes job. - - Args: - body (dict): A dictionary representation of a Kubernetes V1Job - specification. - namespace (str, optional): The Kubernetes namespace to create this job in, - defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block - holding authentication needed to generate the required API client. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - """ - api_client = kubernetes_credentials.get_batch_client() - - api_client.create_namespaced_job(namespace=namespace, body=body, **kube_kwargs) - - -@task -async def delete_namespaced_job( - job_name: str, - kubernetes_credentials: KubernetesCredentials, - namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, - delete_option_kwargs: Optional[Dict] = None, -): - """Task for deleting a namespaced Kubernetes job. - - Args: - job_name (str): The name of a job to delete. Defaults to None. - namespace (str, optional): The Kubernetes namespace to delete this job in, - defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block - holding authentication needed to generate the required API client. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - - delete_option_kwargs (dict, optional): Optional keyword arguments to pass to - the V1DeleteOptions object (e.g. {"propagation_policy": "...", - "grace_period_seconds": "..."}. - """ - - api_client = kubernetes_credentials.get_batch_client() - - kwargs = kube_kwargs or {} - - if delete_option_kwargs: - kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) - - api_client.delete_namespaced_job(name=job_name, namespace=namespace, **kwargs) - - -@task -async def list_namespaced_job( - kubernetes_credentials: KubernetesCredentials, - namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, -): - """Task for listing namespaced Kubernetes jobs. - - Args: - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block - holding authentication needed to generate the required API client. - namespace (str, optional): The Kubernetes namespace to list jobs from, - defaults to the `default` namespace. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - """ - api_client = kubernetes_credentials.get_batch_client() - - method_kwargs = kube_kwargs or {} - - api_client.list_namespaced_job( - namespace=namespace, - **method_kwargs, - ) - - -@task -async def patch_namespaced_job( - job_name: str, - body: dict, - kubernetes_credentials: KubernetesCredentials = None, - namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, -): - """Task for deleting a namespaced Kubernetes job. - - Args: - job_name (str): The name of a job to patch. Defaults to None. - body (dict): A dictionary representation of a Kubernetes V1Job - specification. Defaults to None. - namespace (str, optional): The Kubernetes namespace to patch this job in, - defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block - holding authentication needed to generate the required API client. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - - Raises: - ValueError: if `job_name` is `None` - """ - - api_client = kubernetes_credentials.get_batch_client() - - kube_kwargs = kube_kwargs or {} - - api_client.patch_namespaced_job( - name=job_name, namespace=namespace, body=body, **kube_kwargs - ) - - -@task -async def read_namespaced_job( - job_name: str, - kubernetes_credentials: KubernetesCredentials, - namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, -): - """Task for reading a namespaced kubernetes job. - - Args: - job_name (str): The name of a job to read. Defaults to None. - namespace (str, optional): The Kubernetes namespace to read this job in, - defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block - holding authentication needed to generate the required API client. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - - Raises: - ValueError: if `job_name` is `None` - """ - if not job_name: - raise ValueError("The name of a Kubernetes job must be provided.") - - api_client = kubernetes_credentials.get_batch_client() - - kube_kwargs = kube_kwargs or {} - - return api_client.read_namespaced_job( - name=job_name, namespace=namespace, **kube_kwargs - ) - - -@task -async def replace_namespaced_job( - body: dict, - job_name: str, - kubernetes_credentials: KubernetesCredentials, - namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, -): - """Task for replacing a namespaced kubernetes job. - - Args: - body (dict, optional): A dictionary representation of a Kubernetes V1Job - specification - job_name (str): The name of a job to replace. Defaults to None. - namespace (str, optional): The Kubernetes namespace to replace this job in, - defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block - holding authentication needed to generate the required API client. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - """ - api_client = kubernetes_credentials.get_batch_client() - - kube_kwargs = kube_kwargs or {} - - return api_client.replace_namespaced_job( - name=job_name, body=body, namespace=namespace, **kube_kwargs - ) - - -@task -async def run_namespaced_job( - body: str, - kubernetes_credentials: KubernetesCredentials, - namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, - job_status_poll_interval: Optional[int] = 5, - log_level: Optional[str] = None, - delete_job_after_completion: Optional[bool] = True, -): - """Task for running a namespaced kubernetes job. - - Args: - body (str): A dictionary representation of a Kubernetes V1Job - specification. Defaults to None. - namespace (str, optional): The Kubernetes namespace to run this job in, - defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block - holding authentication needed to generate the required API client. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Defaults to None. - job_status_poll_interval (int, optional): The interval given in seconds - indicating how often the Kubernetes API will be requested about the status - of the job being performed, defaults to `5` seconds. - log_level (str, optional): Log level used when outputting logs from the job - should be one of `debug`, `info`, `warn`, `error`, 'critical' or `None` to - disable output completely. Defaults to `None`. - delete_job_after_completion (bool, optional): boolean value determining whether - resources related to a given job will be removed from the Kubernetes cluster - after completion, defaults to the `True` value - Raises: - KubernetesJobDefinitionError: if `body` is `None` - KubernetesJobDefinitionError: if `body["metadata"]["name"] is `None` - """ - logger = get_run_logger().setLevel(level=log_level) - - # if log_level is not None and getattr(logger, log_level, None) is None: - # raise ValueError("A valid log_level must be provided.") - - kube_kwargs = kube_kwargs or {} - - job_name = body.get("metadata", {}).get("name") - - if not job_name: - raise err.KubernetesJobDefinitionError( - "The job name must be defined in the body under the metadata key." - ) - - api_batch_client = kubernetes_credentials.get_batch_client() - api_core_client = kubernetes_credentials.get_core_client() - - api_batch_client.create_namespaced_job( - namespace=namespace, body=body, **kube_kwargs - ) - logger.info(f"Job {job_name} has been created.") - - pod_log_streams = {} - - # Context is thread-local and isn't automatically copied to the threads spawned by ThreadPoolExecutor. - # Add an initializer which updates the thread's Context with values from the current Context. - context_copy = prefect.context.copy() - - def initialize_thread(context): - prefect.context.update(context) - - with ThreadPoolExecutor( - initializer=initialize_thread, initargs=(context_copy,) - ) as pool: - completed = False - while not completed: - job = api_batch_client.read_namespaced_job_status( - name=job_name, namespace=namespace - ) - if log_level is not None: - func_log = getattr(logger, log_level) - - pod_selector = f"controller-uid={job.metadata.labels['controller-uid']}" - pods_list = api_core_client.list_namespaced_pod( - namespace=namespace, label_selector=pod_selector - ) - - for pod in pods_list.items: - pod_name = pod.metadata.name - - # Can't start logs when phase is pending - if pod.status.phase == "Pending": - continue - if pod_name in pod_log_streams: - continue - - read_pod_logs = read_namespaced_pod_logs( - pod_name=pod_name, - namespace=namespace, - kubernetes_credentials=kubernetes_credentials, - on_log_entry=lambda log: func_log(f"{pod_name}: {log}"), - ) - - logger.info(f"Started following logs for {pod_name}") - pod_log_streams[pod_name] = pool.submit(read_pod_logs.run) - - if job.status.active: - time.sleep(job_status_poll_interval) - elif job.status.failed: - raise err.KubernetesJobFailedError( - f"Job {job_name} failed, check Kubernetes pod logs for more information." - ) - elif job.status.succeeded: - logger.info(f"Job {job_name} has been completed.") - break - - if delete_job_after_completion: - api_batch_client.delete_namespaced_job( - name=job_name, - namespace=namespace, - body=client.V1DeleteOptions(propagation_policy="Foreground"), - ) - logger.info(f"Job {job_name} has been deleted.") diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py new file mode 100644 index 0000000..6126088 --- /dev/null +++ b/prefect_kubernetes/jobs.py @@ -0,0 +1,186 @@ +"""Module to define tasks for interacting with Kubernetes jobs.""" + +from typing import Dict, Optional + +from kubernetes import client +from prefect import task + +from prefect_kubernetes.credentials import KubernetesCredentials + + +@task +async def create_namespaced_job( + body: Dict, + kubernetes_credentials: KubernetesCredentials, + namespace: Optional[str] = "default", + kube_kwargs: Optional[Dict] = None, +): + """Task for creating a namespaced Kubernetes job. + + Args: + body: A dictionary representation of a Kubernetes V1Job specification. + namespace: The Kubernetes namespace to create this job in. + kubernetes_credentials: KubernetesCredentials block + holding authentication needed to generate the required API client. + kube_kwargs: Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + """ + api_client = kubernetes_credentials.get_batch_client() + + api_client.create_namespaced_job(namespace=namespace, body=body, **kube_kwargs) + + +@task +async def delete_namespaced_job( + job_name: str, + kubernetes_credentials: KubernetesCredentials, + namespace: Optional[str] = "default", + kube_kwargs: Optional[Dict] = None, + delete_option_kwargs: Optional[Dict] = None, +): + """Task for deleting a namespaced Kubernetes job. + + Args: + job_name (str): The name of a job to delete. + namespace (str, optional): The Kubernetes namespace to delete this job in, + defaults to the `default` namespace. + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + - delete_option_kwargs (dict, optional): Optional keyword arguments to pass to + the V1DeleteOptions object (e.g. {"propagation_policy": "...", + "grace_period_seconds": "..."}. + """ + + api_client = kubernetes_credentials.get_batch_client() + + kwargs = kube_kwargs or {} + + if delete_option_kwargs: + kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) + + api_client.delete_namespaced_job(name=job_name, namespace=namespace, **kwargs) + + +@task +async def list_namespaced_job( + kubernetes_credentials: KubernetesCredentials, + namespace: Optional[str] = "default", + kube_kwargs: Optional[Dict] = None, +): + """Task for listing namespaced Kubernetes jobs. + + Args: + kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + holding authentication needed to generate the required API client. + namespace (str, optional): The Kubernetes namespace to list jobs from, + defaults to the `default` namespace. + kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + """ + api_client = kubernetes_credentials.get_batch_client() + + method_kwargs = kube_kwargs or {} + + api_client.list_namespaced_job( + namespace=namespace, + **method_kwargs, + ) + + +@task +async def patch_namespaced_job( + job_name: str, + body: dict, + kubernetes_credentials: KubernetesCredentials = None, + namespace: Optional[str] = "default", + kube_kwargs: Optional[Dict] = None, +): + """Task for deleting a namespaced Kubernetes job. + + Args: + job_name: The name of a job to patch. + body: A dictionary representation of a Kubernetes V1Job + specification. + namespace: The Kubernetes namespace to patch this job in, + defaults to the `default` namespace. + kubernetes_credentials: KubernetesCredentials block + holding authentication needed to generate the required API client. + kube_kwargs: Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + + Raises: + ValueError: if `job_name` is `None` + """ + + api_client = kubernetes_credentials.get_batch_client() + + kube_kwargs = kube_kwargs or {} + + api_client.patch_namespaced_job( + name=job_name, namespace=namespace, body=body, **kube_kwargs + ) + + +@task +async def read_namespaced_job( + job_name: str, + kubernetes_credentials: KubernetesCredentials, + namespace: Optional[str] = "default", + kube_kwargs: Optional[Dict] = None, +): + """Task for reading a namespaced kubernetes job. + + Args: + job_name: The name of a job to read. + namespace: The Kubernetes namespace to read this job in, + defaults to the `default` namespace. + kubernetes_credentials: KubernetesCredentials block + holding authentication needed to generate the required API client. + kube_kwargs: Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + + Raises: + ValueError: if `job_name` is `None` + """ + if not job_name: + raise ValueError("The name of a Kubernetes job must be provided.") + + api_client = kubernetes_credentials.get_batch_client() + + kube_kwargs = kube_kwargs or {} + + return api_client.read_namespaced_job( + name=job_name, namespace=namespace, **kube_kwargs + ) + + +@task +async def replace_namespaced_job( + body: dict, + job_name: str, + kubernetes_credentials: KubernetesCredentials, + namespace: Optional[str] = "default", + kube_kwargs: Optional[Dict] = None, +): + """Task for replacing a namespaced kubernetes job. + + Args: + body: A dictionary representation of a Kubernetes V1Job + specification + job_name: The name of a job to replace. + namespace: The Kubernetes namespace to replace this job in, + defaults to the `default` namespace. + kubernetes_credentials: KubernetesCredentials block + holding authentication needed to generate the required API client. + kube_kwargs: Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + """ + api_client = kubernetes_credentials.get_batch_client() + + kube_kwargs = kube_kwargs or {} + + return api_client.replace_namespaced_job( + name=job_name, body=body, namespace=namespace, **kube_kwargs + ) diff --git a/prefect_kubernetes/pod.py b/prefect_kubernetes/pod.py deleted file mode 100644 index 80e1fd2..0000000 --- a/prefect_kubernetes/pod.py +++ /dev/null @@ -1,173 +0,0 @@ -from typing import Callable, List, Optional, Union - -from kubernetes.client.rest import ApiException -from kubernetes.stream import stream -from kubernetes.stream.ws_client import WSClient -from kubernetes.watch import Watch -from prefect import get_run_logger, task - -from prefect_kubernetes.credentials import KubernetesCredentials -from prefect_kubernetes.exceptions import KubernetesResourceNotFoundError - -KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE = 0 - - -@task -async def read_namespaced_pod_logs( - pod_name: str, - kubernetes_credentials: KubernetesCredentials, - namespace: str = "default", - on_log_entry: Callable = None, - container: str = None, -) -> None: - """ - Task run method. - Args: - - pod_name (str, optional): The name of a pod to replace - - kubernetes_credentials (KubernetesCredentials): name of the KubernetesCredentials block - - namespace (str, optional): The Kubernetes namespace to read pod logs in, - defaults to the `default` namespace - - on_log_entry (Callable, optional): If provided, will stream the pod logs - calling the callback for every line (and the task returns `None`). If not - provided, the current pod logs will be returned immediately from the task. - - container (str, optional): The name of the container to read logs from - """ - - api_core_client = kubernetes_credentials.get_core_client() - - if on_log_entry is None: - return api_core_client.read_namespaced_pod_log( - name=pod_name, namespace=namespace, container=container - ) - - # From the kubernetes.watch documentation: - # Note that watching an API resource can expire. The method tries to - # resume automatically once from the last result, but if that last result - # is too old as well, an `ApiException` exception will be thrown with - # ``code`` 410. - while True: - try: - stream = Watch().stream( - api_core_client.read_namespaced_pod_log, - name=pod_name, - namespace=namespace, - container=container, - ) - - for log in stream: - on_log_entry(log) - - return - except ApiException as exception: - if exception.status != 410: - raise - - -@task -async def connect_get_namespaced_pod_exec( - pod_name: str, - container: str, - command: List[str], - kubernetes_credentials: KubernetesCredentials, - namespace: Optional[str] = "default", - interactive: bool = False, - **kwargs, -) -> Union[str, WSClient]: - """Task for running and/or streaming commands in a namespaced pod on Kubernetes. - - This task requires `KubernetesCredentials` to generate a`CoreV1Api` Kubernetes - client to run / stream the user `command` on a specified `container` of `pod_name`. - - User-provided `kwargs` will overwrite `default_settings` if keys exist in both. - - The `kubernetes.stream.stream` object accepts a `_preload_content` kwarg (defualts to True) which - determines this task's return value type, configurable as the `interactive` bool parameter. - - If `interactive` is set to False (as is default), then `_preload_content=True`, - and `api_response` will be the str output of `command` on `container`. - - Else if `interactive` is set to True, `_preload_content=False`, and - `api_response` will be an interactive `WSClient` object. - - Note that since `WSClient` is a un-pickleable object-type, it cannot be used as the `return` value - of a `@flow`-decorated function definition. - - Args: - pod_name (str): The name of the pod in which the command is to be run - container (str): The name of a container to use in the pod - command (List): The command to run in `pod_name` - kubernetes_credentials (KubernetesCredentials): A block that stores a Kubernetes credentials, - has methods to generate resource-specific client - namespace (str, optional): The Kubernetes namespace of the pod - Defaults to `default` - interactive (bool, optional): If `True` return the interactive Kubernetes websocket - client object `WSClient`, else the `str` output of `command`. Defaults to `False`. - kwargs (Dict, optional): Optional extra keyword arguments to pass to the - Kubernetes API method (e.g. `{"stderr": "False", "tty": "True"}`) - - Returns: - Union[str, WSClient]: This task either returns the str output of `command`, or if - `_preload_content=False`, then an interactive `WSClient` object is returned - - Raises: - - TypeError: `command` is not a list, or `api_response` is of unexpected type - - KubernetesResourceNotFoundError: if `api_response` has KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE - - ApiException: if bad `api_response` status but is not a KubernetesResourceNotFoundError - """ - - logger = get_run_logger() - - if not isinstance(command, List): - raise TypeError("The `command` argument must be provided as a list") - - api_client = kubernetes_credentials.get_core_client() - - default_settings = dict( - stderr=True, - stdin=True, - stdout=True, - tty=False, - ) - # overwrite `default_settings` with user-defined `kwargs` and add any other `kwargs` provided - api_method_kwargs = {**default_settings, **kwargs} - - if interactive: - api_method_kwargs["_preload_content"] = False - - try: - api_response = stream( - api_client.connect_get_namespaced_pod_exec, - name=pod_name, - namespace=namespace, - container=container, - command=command, - **api_method_kwargs, - ) - - if isinstance(api_response, str): - logger.info( - f"Returning `str` output of '{' '.join(command)}' as executed on {container}..." - ) - elif isinstance(api_response, WSClient): - logger.info( - f"Returning an interactive `kubernetes.stream.ws_client.WSClient` object..." - ) - else: - raise TypeError( - f"Unexpected API response object `{type(api_response)}`: {api_response!r}" - ) - - return api_response - - except ApiException as err: - if err.status == KUBERNETES_RESOURCE_NOT_FOUND_STATUS_CODE: - raise KubernetesResourceNotFoundError( - status=404, - reason=( - f"{err.reason}" - " - Your Kubernetes API client cannot find a resource you specified." - ), - ) - else: - logger.error(f"{err.reason}") - raise diff --git a/prefect_kubernetes/secrets.py b/prefect_kubernetes/secrets.py deleted file mode 100644 index e69de29..0000000 diff --git a/prefect_kubernetes/service.py b/prefect_kubernetes/service.py deleted file mode 100644 index e69de29..0000000 diff --git a/tests/test_job.py b/tests/test_jobs.py similarity index 77% rename from tests/test_job.py rename to tests/test_jobs.py index c980fb6..9a99abd 100644 --- a/tests/test_job.py +++ b/tests/test_jobs.py @@ -1,14 +1,13 @@ import pytest from kubernetes.client.exceptions import ApiValueError -from prefect_kubernetes.job import ( +from prefect_kubernetes.jobs import ( create_namespaced_job, delete_namespaced_job, list_namespaced_job, patch_namespaced_job, read_namespaced_job, replace_namespaced_job, - run_namespaced_job, ) @@ -90,30 +89,3 @@ async def test_replace_namespaced_job(kubernetes_credentials, api_batch_client): assert api_batch_client.replace_namespaced_job.call_args[1]["namespace"] == "ns" assert api_batch_client.replace_namespaced_job.call_args[1]["body"] == {"test": "a"} assert api_batch_client.replace_namespaced_job.call_args[1]["a"] == "test" - - -# async def test_run_namespaced_job( -# kubernetes_credentials, api_batch_client, api_core_client -# ): -# @flow -# def test_flow(): -# run_namespaced_job( -# body={"metadata": {"name": "test"}}, -# namespace="ns", -# kube_kwargs={"a": "test"}, -# kubernetes_credentials=kubernetes_credentials, -# log_level="DEBUG", -# ) - -# test_flow() - -# assert api_batch_client.create_namespaced_job.call_args[1]["body"] == {"test": "a"} -# assert api_batch_client.create_namespaced_job.call_args[1]["namespace"] == "ns" -# assert api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" - -# assert api_batch_client.read_namespaced_job_status.call_args[1]["body"] == { -# "test": "a" -# } -# assert api_batch_client.read_namespaced_job_status.call_args[1]["namespace"] == "ns" - -# assert api_core_client.list_namespaced_pod.call_args[1]["namespace"] == "ns" diff --git a/tests/test_pod.py b/tests/test_pod.py deleted file mode 100644 index 44e9d42..0000000 --- a/tests/test_pod.py +++ /dev/null @@ -1,104 +0,0 @@ -import pytest -from kubernetes.client.exceptions import ApiException -from kubernetes.stream.ws_client import WSClient -from prefect import flow - -from prefect_kubernetes.pod import connect_get_namespaced_pod_exec - - -async def test_connect_get_namespaced_pod_exec_str_return(kubernetes_credentials): - @flow - async def test_flow(): - return await connect_get_namespaced_pod_exec( - pod_name="demo-pod", - container="app-container", - command=["whoami"], - kubernetes_credentials=kubernetes_credentials, - ) - - response = await test_flow() - - assert isinstance(response, str) - assert response == "root\n" - - -async def test_connect_get_namespaced_pod_exec_stream_return(kubernetes_credentials): - @flow - async def test_flow(): - websocket_client = await connect_get_namespaced_pod_exec( - pod_name="demo-pod", - container="app-container", - command=["/bin/bash"], - kubernetes_credentials=kubernetes_credentials, - interactive=True, - ) - assert isinstance(websocket_client, WSClient) - assert websocket_client.is_open() - - return websocket_client - - # `test_flow` failing because cannot return a non-pickleable type `SSLSocket` ? - - -async def test_connect_get_namespaced_pod_exec_stream_return_poweruser( - kubernetes_credentials, -): - @flow - async def test_flow(): - websocket_client = await connect_get_namespaced_pod_exec( - pod_name="demo-pod", - container="app-container", - command=["/bin/bash"], - kubernetes_credentials=kubernetes_credentials, - _preload_content=False, # only someone who knows the client well would try this - ) - assert isinstance(websocket_client, WSClient) - assert websocket_client.is_open() - - with pytest.raises(TypeError): - await test_flow() - - -async def test_connect_get_namespaced_pod_exec_multiline_cmd(kubernetes_credentials): - @flow - async def test_flow(): - return await connect_get_namespaced_pod_exec( - pod_name="demo-pod", - container="app-container", - command=["ls", "/"], - kubernetes_credentials=kubernetes_credentials, - ) - - response = await test_flow() - - assert len(response.splitlines()) == 20 - - -async def test_connect_get_namespaced_pod_exec_no_stdout(kubernetes_credentials): - @flow - async def test_flow(): - return await connect_get_namespaced_pod_exec( - pod_name="demo-pod", - container="app-container", - command=["whoami"], - stdout=False, - kubernetes_credentials=kubernetes_credentials, - ) - - response = await test_flow() - - assert response == "" - - -async def test_connect_get_namespaced_pod_exec_fake_pod(kubernetes_credentials): - @flow - async def test_flow(): - return await connect_get_namespaced_pod_exec( - pod_name="non-existent-pod", - container="app-container", - command=["whoami"], - kubernetes_credentials=kubernetes_credentials, - ) - - with pytest.raises(ApiException): - await test_flow() From 19d71fa069a0767b24c515ba2a7862a277f6a487 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 13:13:23 -0500 Subject: [PATCH 13/43] return client models from job tasks --- prefect_kubernetes/jobs.py | 75 ++++++++++++++++++++++++-------------- 1 file changed, 47 insertions(+), 28 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 6126088..04e8b9a 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -3,6 +3,7 @@ from typing import Dict, Optional from kubernetes import client +from kubernetes.client.models import V1Job, V1JobList, V1Status from prefect import task from prefect_kubernetes.credentials import KubernetesCredentials @@ -14,20 +15,26 @@ async def create_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, -): +) -> V1Job: """Task for creating a namespaced Kubernetes job. Args: - body: A dictionary representation of a Kubernetes V1Job specification. + body: A dictionary representing a Kubernetes V1Job specification. namespace: The Kubernetes namespace to create this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + + + Returns: + A Kubernetes `V1Job` object. """ api_client = kubernetes_credentials.get_batch_client() - api_client.create_namespaced_job(namespace=namespace, body=body, **kube_kwargs) + return api_client.create_namespaced_job( + namespace=namespace, body=body, **kube_kwargs + ) @task @@ -37,20 +44,22 @@ async def delete_namespaced_job( namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, delete_option_kwargs: Optional[Dict] = None, -): +) -> V1Status: """Task for deleting a namespaced Kubernetes job. Args: - job_name (str): The name of a job to delete. - namespace (str, optional): The Kubernetes namespace to delete this job in, - defaults to the `default` namespace. - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + job_name: The name of a job to delete. + namespace: The Kubernetes namespace to delete this job in. + kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). - - delete_option_kwargs (dict, optional): Optional keyword arguments to pass to + - delete_option_kwargs: Optional keyword arguments to pass to the V1DeleteOptions object (e.g. {"propagation_policy": "...", "grace_period_seconds": "..."}. + + Returns: + A Kubernetes `V1Status` object. """ api_client = kubernetes_credentials.get_batch_client() @@ -60,7 +69,9 @@ async def delete_namespaced_job( if delete_option_kwargs: kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) - api_client.delete_namespaced_job(name=job_name, namespace=namespace, **kwargs) + return api_client.delete_namespaced_job( + name=job_name, namespace=namespace, **kwargs + ) @task @@ -68,22 +79,24 @@ async def list_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, -): +) -> V1JobList: """Task for listing namespaced Kubernetes jobs. Args: - kubernetes_credentials (KubernetesCredentials): KubernetesCredentials block + kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - namespace (str, optional): The Kubernetes namespace to list jobs from, - defaults to the `default` namespace. - kube_kwargs (dict, optional): Optional extra keyword arguments to pass to the + namespace: The Kubernetes namespace to list jobs from. + kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + + Returns: + A Kubernetes `V1JobList` object. """ api_client = kubernetes_credentials.get_batch_client() method_kwargs = kube_kwargs or {} - api_client.list_namespaced_job( + return api_client.list_namespaced_job( namespace=namespace, **method_kwargs, ) @@ -92,19 +105,18 @@ async def list_namespaced_job( @task async def patch_namespaced_job( job_name: str, - body: dict, + body: Dict, kubernetes_credentials: KubernetesCredentials = None, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, -): +) -> V1Job: """Task for deleting a namespaced Kubernetes job. Args: job_name: The name of a job to patch. body: A dictionary representation of a Kubernetes V1Job specification. - namespace: The Kubernetes namespace to patch this job in, - defaults to the `default` namespace. + namespace: The Kubernetes namespace to patch this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs: Optional extra keyword arguments to pass to the @@ -112,13 +124,16 @@ async def patch_namespaced_job( Raises: ValueError: if `job_name` is `None` + + Returns: + A Kubernetes `V1Job` object. """ api_client = kubernetes_credentials.get_batch_client() kube_kwargs = kube_kwargs or {} - api_client.patch_namespaced_job( + return api_client.patch_namespaced_job( name=job_name, namespace=namespace, body=body, **kube_kwargs ) @@ -129,13 +144,12 @@ async def read_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, -): +) -> V1Job: """Task for reading a namespaced kubernetes job. Args: job_name: The name of a job to read. - namespace: The Kubernetes namespace to read this job in, - defaults to the `default` namespace. + namespace: The Kubernetes namespace to read this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs: Optional extra keyword arguments to pass to the @@ -143,6 +157,9 @@ async def read_namespaced_job( Raises: ValueError: if `job_name` is `None` + + Returns: + A Kubernetes `V1Job` object. """ if not job_name: raise ValueError("The name of a Kubernetes job must be provided.") @@ -163,19 +180,21 @@ async def replace_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, -): +) -> V1Job: """Task for replacing a namespaced kubernetes job. Args: body: A dictionary representation of a Kubernetes V1Job specification job_name: The name of a job to replace. - namespace: The Kubernetes namespace to replace this job in, - defaults to the `default` namespace. + namespace: The Kubernetes namespace to replace this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + + Returns: + A Kubernetes `V1Job` object. """ api_client = kubernetes_credentials.get_batch_client() From b64d6ebed78d61dd312bfcda8ede20fdac4058c0 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 13:25:36 -0500 Subject: [PATCH 14/43] clean up docstrings --- .gitignore | 5 +---- prefect_kubernetes/credentials.py | 8 ++++---- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/.gitignore b/.gitignore index 8fe1688..430d66c 100644 --- a/.gitignore +++ b/.gitignore @@ -135,7 +135,4 @@ dmypy.json .vscode # Jupyter notebook -*.ipynb - -# testing -testing_stuff.py \ No newline at end of file +*.ipynb \ No newline at end of file diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index f2aba7f..6dca7fb 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -24,10 +24,10 @@ class KubernetesCredentials(Block): """Credentials block for generating configured Kubernetes API clients. Args: - cluster_config (KubernetesClusterConfig, optional): a `KubernetesClusterConfig` - block holding a JSON kube config for a specific kubernetes context + cluster_config: a `KubernetesClusterConfig` block holding a JSON kube + config for a specific kubernetes context - api_key (SecretStr): API key to authenticate with the Kubernetes API + api_key: API key to authenticate with the Kubernetes API Examples: @@ -119,7 +119,7 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: server to keep the connection open. Args: - - resource (str): the name of the resource to retrieve a client for. + - resource: the name of the resource to retrieve a client for. Currently you can use one of these values: `job`, `pod`, `service`, `deployment`, and `secret`. From ad18fa8ec0c9d41a77e239fc76d55335e43dbb3f Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 13:32:22 -0500 Subject: [PATCH 15/43] mock credentials --- tests/conftest.py | 35 +++-------------------------------- 1 file changed, 3 insertions(+), 32 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 8c585a7..1f17c0b 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,4 +1,3 @@ -import json from pathlib import Path from unittest import mock @@ -8,8 +7,6 @@ from prefect_kubernetes.credentials import KubernetesCredentials -CERT_DUMMY = b"dummy" - BASEDIR = Path("tests") GOOD_CONFIG_FILE_PATH = BASEDIR / "kube_config.yaml" @@ -32,11 +29,9 @@ def successful_job_status(): def kubernetes_credentials(kube_config_dict): return KubernetesCredentials( - # cluster_config=KubernetesClusterConfig( - # context_name="test", - # config=kube_config_dict - # ) - cluster_config=KubernetesClusterConfig.load("kube-config") + cluster_config=KubernetesClusterConfig( + context_name="test", config=kube_config_dict + ) ) @@ -74,27 +69,3 @@ def api_core_client(monkeypatch): ) return core_client - - -@pytest.fixture -def mock_ApiClient(monkeypatch): - - mock_ApiClient = mock.MagicMock() - mock_response = mock.MagicMock() - - mock_response.status = mock.PropertyMock(return_value=200) - - mock_response.data = mock.PropertyMock( - return_value=json.dumps( - {"token_endpoint": "https://example.org/identity/token"} - ) - ) - - mock_ApiClient.return_value = mock_response - - monkeypatch.setattr( - "kubernetes.config.kube_config.ApiClient.request", - mock_ApiClient, - ) - - return mock_ApiClient From 2125e24c95f9618779bf1e239e6e3e4ed3d6f3bf Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 13:50:09 -0500 Subject: [PATCH 16/43] add tests for client creation methods --- tests/test_credentials.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 tests/test_credentials.py diff --git a/tests/test_credentials.py b/tests/test_credentials.py new file mode 100644 index 0000000..8040ba6 --- /dev/null +++ b/tests/test_credentials.py @@ -0,0 +1,15 @@ +import pytest +from kubernetes.client import AppsV1Api, BatchV1Api, CoreV1Api + + +@pytest.mark.parametrize( + "resource_type_method,client_type", + [ + ("get_app_client", AppsV1Api), + ("get_batch_client", BatchV1Api), + ("get_core_client", CoreV1Api), + ], +) +def test_client_return_type(kubernetes_credentials, resource_type_method, client_type): + resource_specific_client = getattr(kubernetes_credentials, resource_type_method)() + assert isinstance(resource_specific_client, client_type) From fa5b334b4202a8d44c72e4f99cbaa7099f127355 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 13:53:08 -0500 Subject: [PATCH 17/43] update changelog --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index dcee7ed..7933f7c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,6 +8,8 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## Unreleased ### Added +- Credentials block for generating authenticated Kubernetes clients - [#19](https://github.com/PrefectHQ/prefect-kubernetes/pull/19) +- Tasks for interacting with `job` resources - [#19](https://github.com/PrefectHQ/prefect-kubernetes/pull/19) ### Changed From ad686fc58a9f15de03aa11f3ce9949a12595a691 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 14:01:06 -0500 Subject: [PATCH 18/43] rm boilerplate in changelog --- CHANGELOG.md | 8 -------- 1 file changed, 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7933f7c..531fcf3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,11 +20,3 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ### Fixed ### Security - -## 0.1.0 - -Released on ????? ?th, 20??. - -### Added - -- `task_name` task - [#1](https://github.com/PrefectHQ/prefect-kubernetes/pull/1) From 7195927bdb2dddcd51680699af13dcde0894cf3a Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 14:03:53 -0500 Subject: [PATCH 19/43] rm api key for now --- prefect_kubernetes/credentials.py | 21 ++------------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 6dca7fb..62b3cb5 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -7,7 +7,6 @@ from kubernetes.config.config_exception import ConfigException from prefect.blocks.core import Block from prefect.blocks.kubernetes import KubernetesClusterConfig -from pydantic import SecretStr KubernetesClient = Union[ client.BatchV1Api, client.CoreV1Api, client.AppsV1Api, client.ApiClient @@ -27,9 +26,6 @@ class KubernetesCredentials(Block): cluster_config: a `KubernetesClusterConfig` block holding a JSON kube config for a specific kubernetes context - api_key: API key to authenticate with the Kubernetes API - - Examples: Load stored Kubernetes credentials: ```python @@ -67,8 +63,6 @@ class KubernetesCredentials(Block): cluster_config: Optional[KubernetesClusterConfig] = None - api_key: Optional[SecretStr] = None - def get_core_client(self) -> client.CoreV1Api: """Convenience method for retrieving a kubernetes api client for core resources @@ -105,12 +99,8 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: `KubernetesClusterConfig.configure_client` and then return the `resource_specific_client`. - 2. Attempt to use a `KubernetesCredentials` block's `api_key`. If - `not self.api_key` then it will attempt the next two connection - methods. - - 3. Attempt in-cluster connection (will only work when running on a pod) - 4. Attempt out-of-cluster connection using the default location for a + 2. Attempt in-cluster connection (will only work when running on a pod) + 3. Attempt out-of-cluster connection using the default location for a kube config file. In some cases connections to the kubernetes server are dropped after being idle for some time (e.g. Azure Firewall drops idle connections after 4 minutes) which would result in ReadTimeoutErrors. @@ -133,13 +123,6 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: self.cluster_config.configure_client() return resource_specific_client() - elif self.api_key: - configuration = client.Configuration() - configuration.api_key["authorization"] = self.api_key.get_secret_value() - configuration.api_key_prefix["authorization"] = "Bearer" - generic_client = client.ApiClient(configuration=configuration) - return resource_specific_client(api_client=generic_client) - else: try: print("Trying to load in-cluster configuration...") From edffc7d92a428d1902586d2f76015c427a93bff4 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 14:06:58 -0500 Subject: [PATCH 20/43] rm service doc page until it exists --- docs/service.md | 1 - 1 file changed, 1 deletion(-) delete mode 100644 docs/service.md diff --git a/docs/service.md b/docs/service.md deleted file mode 100644 index 904ab4b..0000000 --- a/docs/service.md +++ /dev/null @@ -1 +0,0 @@ -:::prefect_kubernetes.service \ No newline at end of file From 3d5b5852f4e26dd5297aca9b41b3519a7f801ca6 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 14:09:03 -0500 Subject: [PATCH 21/43] use consistent names for kube_kwargs --- prefect_kubernetes/jobs.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 04e8b9a..dbc65f1 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -64,13 +64,13 @@ async def delete_namespaced_job( api_client = kubernetes_credentials.get_batch_client() - kwargs = kube_kwargs or {} + kube_kwargs = kube_kwargs or {} if delete_option_kwargs: - kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) + kube_kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) return api_client.delete_namespaced_job( - name=job_name, namespace=namespace, **kwargs + name=job_name, namespace=namespace, **kube_kwargs ) @@ -94,11 +94,11 @@ async def list_namespaced_job( """ api_client = kubernetes_credentials.get_batch_client() - method_kwargs = kube_kwargs or {} + kube_kwargs = kube_kwargs or {} return api_client.list_namespaced_job( namespace=namespace, - **method_kwargs, + **kube_kwargs, ) From 0a434834f76b3db2806ed755ab5f6456a609faa3 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 14:36:28 -0500 Subject: [PATCH 22/43] fix formatting things --- mkdocs.yml | 1 + prefect_kubernetes/credentials.py | 11 +++++++---- prefect_kubernetes/jobs.py | 9 +++------ 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/mkdocs.yml b/mkdocs.yml index f289ffb..80e9529 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -37,4 +37,5 @@ plugins: nav: - Home: index.md - Credentials: credentials.md + - Exceptions: exceptions.md - Jobs: jobs.md diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 62b3cb5..f60fa82 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -47,14 +47,17 @@ class KubernetesCredentials(Block): Create a namespaced job: ```python + from prefect import flow from prefect_kubernetes import KubernetesCredentials from prefect_kubernetes.job import create_namespaced_job kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") - create_namespaced_job( - body={"Marvin": "42"}, kubernetes_credentials=kubernetes_credentials - ) + @flow + def kubernetes_orchestrator(): + create_namespaced_job( + body={"Marvin": "42"}, kubernetes_credentials=kubernetes_credentials + ) ``` """ @@ -114,7 +117,7 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: `deployment`, and `secret`. Returns: - - KubernetesClient: an initialized, configured Kubernetes Client + KubernetesClient: an initialized, configured Kubernetes Client """ resource_specific_client = K8S_CLIENTS[resource] diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index dbc65f1..2393008 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -145,7 +145,7 @@ async def read_namespaced_job( namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, ) -> V1Job: - """Task for reading a namespaced kubernetes job. + """Task for reading a namespaced Kubernetes job. Args: job_name: The name of a job to read. @@ -156,14 +156,11 @@ async def read_namespaced_job( Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Raises: - ValueError: if `job_name` is `None` + ValueError: if `job_name` is `None`. Returns: A Kubernetes `V1Job` object. """ - if not job_name: - raise ValueError("The name of a Kubernetes job must be provided.") - api_client = kubernetes_credentials.get_batch_client() kube_kwargs = kube_kwargs or {} @@ -181,7 +178,7 @@ async def replace_namespaced_job( namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, ) -> V1Job: - """Task for replacing a namespaced kubernetes job. + """Task for replacing a namespaced Kubernetes job. Args: body: A dictionary representation of a Kubernetes V1Job From 092cdfeb0d7c6d912ae26e532c22110c40f9d4a2 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 14:39:21 -0500 Subject: [PATCH 23/43] update mock name --- tests/conftest.py | 6 ++--- tests/test_jobs.py | 58 ++++++++++++++++++++++++++++------------------ 2 files changed, 39 insertions(+), 25 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 1f17c0b..28b0e83 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -36,7 +36,7 @@ def kubernetes_credentials(kube_config_dict): @pytest.fixture -def api_app_client(monkeypatch): +def _mock_api_app_client(monkeypatch): app_client = mock.MagicMock() monkeypatch.setattr( @@ -48,7 +48,7 @@ def api_app_client(monkeypatch): @pytest.fixture -def api_batch_client(monkeypatch): +def _mock_api_batch_client(monkeypatch): batch_client = mock.MagicMock() monkeypatch.setattr( @@ -60,7 +60,7 @@ def api_batch_client(monkeypatch): @pytest.fixture -def api_core_client(monkeypatch): +def _mock_api_core_client(monkeypatch): core_client = mock.MagicMock() monkeypatch.setattr( diff --git a/tests/test_jobs.py b/tests/test_jobs.py index 9a99abd..6425d53 100644 --- a/tests/test_jobs.py +++ b/tests/test_jobs.py @@ -22,62 +22,70 @@ async def test_invalid_body_raises_error(kubernetes_credentials): ) -async def test_create_namespaced_job(kubernetes_credentials, api_batch_client): +async def test_create_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await create_namespaced_job.fn( body={"test": "a"}, kube_kwargs={"a": "test"}, kubernetes_credentials=kubernetes_credentials, ) - assert api_batch_client.create_namespaced_job.call_args[1]["body"] == {"test": "a"} - assert api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" + assert _mock_api_batch_client.create_namespaced_job.call_args[1]["body"] == { + "test": "a" + } + assert _mock_api_batch_client.create_namespaced_job.call_args[1]["a"] == "test" -async def test_delete_namespaced_job(kubernetes_credentials, api_batch_client): +async def test_delete_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await delete_namespaced_job.fn( job_name="test_job", kube_kwargs={"a": "test"}, kubernetes_credentials=kubernetes_credentials, ) - assert api_batch_client.delete_namespaced_job.call_args[1]["name"] == "test_job" - assert api_batch_client.delete_namespaced_job.call_args[1]["a"] == "test" + assert ( + _mock_api_batch_client.delete_namespaced_job.call_args[1]["name"] == "test_job" + ) + assert _mock_api_batch_client.delete_namespaced_job.call_args[1]["a"] == "test" -async def test_list_namespaced_job(kubernetes_credentials, api_batch_client): +async def test_list_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await list_namespaced_job.fn( namespace="ns", kube_kwargs={"a": "test"}, kubernetes_credentials=kubernetes_credentials, ) - assert api_batch_client.list_namespaced_job.call_args[1]["namespace"] == "ns" - assert api_batch_client.list_namespaced_job.call_args[1]["a"] == "test" + assert _mock_api_batch_client.list_namespaced_job.call_args[1]["namespace"] == "ns" + assert _mock_api_batch_client.list_namespaced_job.call_args[1]["a"] == "test" -async def test_patch_namespaced_job(kubernetes_credentials, api_batch_client): +async def test_patch_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await patch_namespaced_job.fn( body={"test": "a"}, job_name="test_job", kube_kwargs={"a": "test"}, kubernetes_credentials=kubernetes_credentials, ) - assert api_batch_client.patch_namespaced_job.call_args[1]["body"] == {"test": "a"} - assert api_batch_client.patch_namespaced_job.call_args[1]["name"] == "test_job" - assert api_batch_client.patch_namespaced_job.call_args[1]["a"] == "test" + assert _mock_api_batch_client.patch_namespaced_job.call_args[1]["body"] == { + "test": "a" + } + assert ( + _mock_api_batch_client.patch_namespaced_job.call_args[1]["name"] == "test_job" + ) + assert _mock_api_batch_client.patch_namespaced_job.call_args[1]["a"] == "test" -async def test_read_namespaced_job(kubernetes_credentials, api_batch_client): +async def test_read_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await read_namespaced_job.fn( job_name="test_job", namespace="ns", kube_kwargs={"a": "test"}, kubernetes_credentials=kubernetes_credentials, ) - assert api_batch_client.read_namespaced_job.call_args[1]["name"] == "test_job" - assert api_batch_client.read_namespaced_job.call_args[1]["namespace"] == "ns" - assert api_batch_client.read_namespaced_job.call_args[1]["a"] == "test" + assert _mock_api_batch_client.read_namespaced_job.call_args[1]["name"] == "test_job" + assert _mock_api_batch_client.read_namespaced_job.call_args[1]["namespace"] == "ns" + assert _mock_api_batch_client.read_namespaced_job.call_args[1]["a"] == "test" -async def test_replace_namespaced_job(kubernetes_credentials, api_batch_client): +async def test_replace_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await replace_namespaced_job.fn( job_name="test_job", body={"test": "a"}, @@ -85,7 +93,13 @@ async def test_replace_namespaced_job(kubernetes_credentials, api_batch_client): kube_kwargs={"a": "test"}, kubernetes_credentials=kubernetes_credentials, ) - assert api_batch_client.replace_namespaced_job.call_args[1]["name"] == "test_job" - assert api_batch_client.replace_namespaced_job.call_args[1]["namespace"] == "ns" - assert api_batch_client.replace_namespaced_job.call_args[1]["body"] == {"test": "a"} - assert api_batch_client.replace_namespaced_job.call_args[1]["a"] == "test" + assert ( + _mock_api_batch_client.replace_namespaced_job.call_args[1]["name"] == "test_job" + ) + assert ( + _mock_api_batch_client.replace_namespaced_job.call_args[1]["namespace"] == "ns" + ) + assert _mock_api_batch_client.replace_namespaced_job.call_args[1]["body"] == { + "test": "a" + } + assert _mock_api_batch_client.replace_namespaced_job.call_args[1]["a"] == "test" From ce0ac4cd610a65e8040f20c2b2b4c654573ab8e0 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 2 Nov 2022 14:51:17 -0500 Subject: [PATCH 24/43] inherit from OpenAPIException or ApiException --- prefect_kubernetes/credentials.py | 25 +++++++++---------------- prefect_kubernetes/exceptions.py | 6 +++--- requirements-dev.txt | 2 +- 3 files changed, 13 insertions(+), 20 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index f60fa82..85431ac 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -97,27 +97,23 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: It will attempt to connect to a Kubernetes cluster in three steps with the first successful connection attempt becoming the mode of communication with a cluster: + 1. It will first attempt to use a `KubernetesCredentials` block's `cluster_config` to configure a client using `KubernetesClusterConfig.configure_client` and then return the - `resource_specific_client`. + resource specific client. - 2. Attempt in-cluster connection (will only work when running on a pod) + 2. Attempt in-cluster connection (will only work when running on a pod). 3. Attempt out-of-cluster connection using the default location for a - kube config file. In some cases connections to the kubernetes server - are dropped after being idle for some time (e.g. Azure Firewall drops - idle connections after 4 minutes) which would result in ReadTimeoutErrors. - - In order to prevent that a periodic keep-alive message can be sent to the - server to keep the connection open. + kube config file. Args: - - resource: the name of the resource to retrieve a client for. - Currently you can use one of these values: `job`, `pod`, `service`, - `deployment`, and `secret`. + resource: the name of the resource to retrieve a client for. + Currently you can use one of these values: `job`, `core`, + or `deployment`. Returns: - KubernetesClient: an initialized, configured Kubernetes Client + An authenticated and configured Kubernetes Client. """ resource_specific_client = K8S_CLIENTS[resource] @@ -128,11 +124,8 @@ def get_kubernetes_client(self, resource: str) -> KubernetesClient: else: try: - print("Trying to load in-cluster configuration...") kube_config.load_incluster_config() - except ConfigException as exc: - print("{} | Using out of cluster configuration option.".format(exc)) - print("Loading out-of-cluster configuration...") + except ConfigException: kube_config.load_kube_config() return resource_specific_client() diff --git a/prefect_kubernetes/exceptions.py b/prefect_kubernetes/exceptions.py index 72b65ff..27ad2af 100644 --- a/prefect_kubernetes/exceptions.py +++ b/prefect_kubernetes/exceptions.py @@ -1,13 +1,13 @@ """Module to define common exceptions within `prefect_kubernetes`.""" -from kubernetes.client.exceptions import ApiException +from kubernetes.client.exceptions import ApiException, OpenApiException -class KubernetesJobDefinitionError(Exception): +class KubernetesJobDefinitionError(OpenApiException): """An exception for when a Kubernetes job definition is invalid""" -class KubernetesJobFailedError(Exception): +class KubernetesJobFailedError(OpenApiException): """An exception for when a Kubernetes job fails""" diff --git a/requirements-dev.txt b/requirements-dev.txt index ead4728..cee55b6 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -4,7 +4,7 @@ flake8 mypy mkdocs mkdocs-material -mkdocstrings +mkdocstrings[python] isort pre-commit pytest-asyncio From 29086d599c548d59bb8452fa77f06cf64b072400 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 4 Nov 2022 12:21:57 -0500 Subject: [PATCH 25/43] make job tasks sync --- prefect_kubernetes/jobs.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 2393008..c2fadcc 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -38,7 +38,7 @@ async def create_namespaced_job( @task -async def delete_namespaced_job( +def delete_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", @@ -54,7 +54,7 @@ async def delete_namespaced_job( holding authentication needed to generate the required API client. kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). - - delete_option_kwargs: Optional keyword arguments to pass to + delete_option_kwargs: Optional keyword arguments to pass to the V1DeleteOptions object (e.g. {"propagation_policy": "...", "grace_period_seconds": "..."}. @@ -75,7 +75,7 @@ async def delete_namespaced_job( @task -async def list_namespaced_job( +def list_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, @@ -103,7 +103,7 @@ async def list_namespaced_job( @task -async def patch_namespaced_job( +def patch_namespaced_job( job_name: str, body: Dict, kubernetes_credentials: KubernetesCredentials = None, @@ -139,7 +139,7 @@ async def patch_namespaced_job( @task -async def read_namespaced_job( +def read_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", @@ -171,7 +171,7 @@ async def read_namespaced_job( @task -async def replace_namespaced_job( +def replace_namespaced_job( body: dict, job_name: str, kubernetes_credentials: KubernetesCredentials, From 2091ceb5d84cc7cbcf846d62a1b49109b55fc2cd Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 4 Nov 2022 13:10:53 -0500 Subject: [PATCH 26/43] use run_sync_in_worker_thread --- prefect_kubernetes/jobs.py | 48 +++++++++++++++++++++++++------------- tests/conftest.py | 1 - 2 files changed, 32 insertions(+), 17 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index c2fadcc..cf77eaa 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -5,6 +5,7 @@ from kubernetes import client from kubernetes.client.models import V1Job, V1JobList, V1Status from prefect import task +from prefect.utilities.asyncutils import run_sync_in_worker_thread from prefect_kubernetes.credentials import KubernetesCredentials @@ -32,13 +33,13 @@ async def create_namespaced_job( """ api_client = kubernetes_credentials.get_batch_client() - return api_client.create_namespaced_job( - namespace=namespace, body=body, **kube_kwargs + return await run_sync_in_worker_thread( + api_client.create_namespaced_job, namespace=namespace, body=body, **kube_kwargs ) @task -def delete_namespaced_job( +async def delete_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", @@ -69,13 +70,16 @@ def delete_namespaced_job( if delete_option_kwargs: kube_kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) - return api_client.delete_namespaced_job( - name=job_name, namespace=namespace, **kube_kwargs + return await run_sync_in_worker_thread( + api_client.delete_namespaced_job, + name=job_name, + namespace=namespace, + **kube_kwargs, ) @task -def list_namespaced_job( +async def list_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", kube_kwargs: Optional[Dict] = None, @@ -96,14 +100,15 @@ def list_namespaced_job( kube_kwargs = kube_kwargs or {} - return api_client.list_namespaced_job( + return await run_sync_in_worker_thread( + api_client.list_namespaced_job, namespace=namespace, **kube_kwargs, ) @task -def patch_namespaced_job( +async def patch_namespaced_job( job_name: str, body: Dict, kubernetes_credentials: KubernetesCredentials = None, @@ -133,13 +138,17 @@ def patch_namespaced_job( kube_kwargs = kube_kwargs or {} - return api_client.patch_namespaced_job( - name=job_name, namespace=namespace, body=body, **kube_kwargs + return await run_sync_in_worker_thread( + api_client.patch_namespaced_job, + name=job_name, + namespace=namespace, + body=body, + **kube_kwargs, ) @task -def read_namespaced_job( +async def read_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", @@ -165,13 +174,16 @@ def read_namespaced_job( kube_kwargs = kube_kwargs or {} - return api_client.read_namespaced_job( - name=job_name, namespace=namespace, **kube_kwargs + return await run_sync_in_worker_thread( + api_client.read_namespaced_job, + name=job_name, + namespace=namespace, + **kube_kwargs, ) @task -def replace_namespaced_job( +async def replace_namespaced_job( body: dict, job_name: str, kubernetes_credentials: KubernetesCredentials, @@ -197,6 +209,10 @@ def replace_namespaced_job( kube_kwargs = kube_kwargs or {} - return api_client.replace_namespaced_job( - name=job_name, body=body, namespace=namespace, **kube_kwargs + return await run_sync_in_worker_thread( + api_client.replace_namespaced_job, + name=job_name, + body=body, + namespace=namespace, + **kube_kwargs, ) diff --git a/tests/conftest.py b/tests/conftest.py index 28b0e83..e047680 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -27,7 +27,6 @@ def successful_job_status(): @pytest.fixture def kubernetes_credentials(kube_config_dict): - return KubernetesCredentials( cluster_config=KubernetesClusterConfig( context_name="test", config=kube_config_dict From 5f15135290589a71ba25efd4158a21cb35afba12 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 4 Nov 2022 13:29:26 -0500 Subject: [PATCH 27/43] add examples to docstrings --- prefect_kubernetes/jobs.py | 86 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 86 insertions(+) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index cf77eaa..af828d9 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -30,6 +30,20 @@ async def create_namespaced_job( Returns: A Kubernetes `V1Job` object. + + Example: + Create a job in the default namespace: + ```python + from prefect import flow + from prefect_kubernetes import create_namespaced_job + + @flow + def kubernetes_orchestrator(): + v1_job_metadata = create_namespaced_job( + body={"metadata": {"name": "test-job"}}, + kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + ) + ``` """ api_client = kubernetes_credentials.get_batch_client() @@ -61,6 +75,20 @@ async def delete_namespaced_job( Returns: A Kubernetes `V1Status` object. + + Example: + Delete "my-job" in the default namespace: + ```python + from prefect import flow + from prefect_kubernetes import delete_namespaced_job + + @flow + def kubernetes_orchestrator(): + v1_job_status = delete_namespaced_job( + job_name="my-job", + kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + ) + ``` """ api_client = kubernetes_credentials.get_batch_client() @@ -95,6 +123,20 @@ async def list_namespaced_job( Returns: A Kubernetes `V1JobList` object. + + Example: + List jobs in "my-namespace": + ```python + from prefect import flow + from prefect_kubernetes import list_namespaced_job + + @flow + def kubernetes_orchestrator(): + namespaced_job_list = list_namespaced_job( + namespace="my-namespace", + kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + ) + ``` """ api_client = kubernetes_credentials.get_batch_client() @@ -132,6 +174,21 @@ async def patch_namespaced_job( Returns: A Kubernetes `V1Job` object. + + Example: + Patch "my-job" in the default namespace: + ```python + from prefect import flow + from prefect_kubernetes import patch_namespaced_job + + @flow + def kubernetes_orchestrator(): + v1_job_metadata = patch_namespaced_job( + job_name="my-job", + body={"metadata": {"labels": {"foo": "bar"}}}, + kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + ) + ``` """ api_client = kubernetes_credentials.get_batch_client() @@ -169,6 +226,20 @@ async def read_namespaced_job( Returns: A Kubernetes `V1Job` object. + + Example: + Read "my-job" in the default namespace: + ```python + from prefect import flow + from prefect_kubernetes import read_namespaced_job + + @flow + def kubernetes_orchestrator(): + v1_job_metadata = read_namespaced_job( + job_name="my-job", + kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + ) + ``` """ api_client = kubernetes_credentials.get_batch_client() @@ -204,6 +275,21 @@ async def replace_namespaced_job( Returns: A Kubernetes `V1Job` object. + + Example: + Replace "my-job" in the default namespace: + ```python + from prefect import flow + from prefect_kubernetes import replace_namespaced_job + + @flow + def kubernetes_orchestrator(): + v1_job_metadata = replace_namespaced_job( + body={"metadata": {"labels": {"foo": "bar"}}}, + job_name="my-job", + kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + ) + ``` """ api_client = kubernetes_credentials.get_batch_client() From f921cb937d0ddf6ad1238a4af57bb3e2cc2c41dc Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 4 Nov 2022 13:57:22 -0500 Subject: [PATCH 28/43] fix import and add example to readme --- README.md | 17 +++++++---------- prefect_kubernetes/jobs.py | 12 ++++++------ 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/README.md b/README.md index 04c5a65..a5c17c0 100644 --- a/README.md +++ b/README.md @@ -23,21 +23,18 @@ pip install prefect-kubernetes ``` ### Write and run a flow +#### List jobs in a specific namespace ```python from prefect import flow -from prefect_kubernetes.tasks import ( - goodbye_prefect_kubernetes, - hello_prefect_kubernetes, -) - +from prefect_kubernetes.jobs import list_namespaced_job @flow -def example_flow(): - hello_prefect_kubernetes - goodbye_prefect_kubernetes - -example_flow() +def kubernetes_orchestrator(): + namespaced_job_list = list_namespaced_job( + namespace="my-namespace", + kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + ) ``` ## Resources diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index af828d9..55829e2 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -35,7 +35,7 @@ async def create_namespaced_job( Create a job in the default namespace: ```python from prefect import flow - from prefect_kubernetes import create_namespaced_job + from prefect_kubernetes.jobs import create_namespaced_job @flow def kubernetes_orchestrator(): @@ -80,7 +80,7 @@ async def delete_namespaced_job( Delete "my-job" in the default namespace: ```python from prefect import flow - from prefect_kubernetes import delete_namespaced_job + from prefect_kubernetes.jobs import delete_namespaced_job @flow def kubernetes_orchestrator(): @@ -128,7 +128,7 @@ async def list_namespaced_job( List jobs in "my-namespace": ```python from prefect import flow - from prefect_kubernetes import list_namespaced_job + from prefect_kubernetes.jobs import list_namespaced_job @flow def kubernetes_orchestrator(): @@ -179,7 +179,7 @@ async def patch_namespaced_job( Patch "my-job" in the default namespace: ```python from prefect import flow - from prefect_kubernetes import patch_namespaced_job + from prefect_kubernetes.jobs import patch_namespaced_job @flow def kubernetes_orchestrator(): @@ -231,7 +231,7 @@ async def read_namespaced_job( Read "my-job" in the default namespace: ```python from prefect import flow - from prefect_kubernetes import read_namespaced_job + from prefect_kubernetes.jobs import read_namespaced_job @flow def kubernetes_orchestrator(): @@ -280,7 +280,7 @@ async def replace_namespaced_job( Replace "my-job" in the default namespace: ```python from prefect import flow - from prefect_kubernetes import replace_namespaced_job + from prefect_kubernetes.jobs import replace_namespaced_job @flow def kubernetes_orchestrator(): From 4a8d7166dc34a6c4ba45b802e8b460cd446ab379 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 4 Nov 2022 14:18:10 -0500 Subject: [PATCH 29/43] typing for kubekwargs --- prefect_kubernetes/jobs.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 55829e2..3e9fe5d 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -1,6 +1,6 @@ """Module to define tasks for interacting with Kubernetes jobs.""" -from typing import Dict, Optional +from typing import Any, Dict, Optional from kubernetes import client from kubernetes.client.models import V1Job, V1JobList, V1Status @@ -57,7 +57,7 @@ async def delete_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, + kube_kwargs: Optional[Dict[str, Any]] = None, delete_option_kwargs: Optional[Dict] = None, ) -> V1Status: """Task for deleting a namespaced Kubernetes job. @@ -110,7 +110,7 @@ def kubernetes_orchestrator(): async def list_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, + kube_kwargs: Optional[Dict[str, Any]] = None, ) -> V1JobList: """Task for listing namespaced Kubernetes jobs. @@ -155,7 +155,7 @@ async def patch_namespaced_job( body: Dict, kubernetes_credentials: KubernetesCredentials = None, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, + kube_kwargs: Optional[Dict[str, Any]] = None, ) -> V1Job: """Task for deleting a namespaced Kubernetes job. @@ -209,7 +209,7 @@ async def read_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, + kube_kwargs: Optional[Dict[str, Any]] = None, ) -> V1Job: """Task for reading a namespaced Kubernetes job. @@ -259,7 +259,7 @@ async def replace_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, + kube_kwargs: Optional[Dict[str, Any]] = None, ) -> V1Job: """Task for replacing a namespaced Kubernetes job. From c27981df051d719e7f6907e828a468e4542324ee Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Fri, 4 Nov 2022 14:21:07 -0500 Subject: [PATCH 30/43] unpacking syntax in docstrings --- prefect_kubernetes/jobs.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 3e9fe5d..f994290 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -24,7 +24,7 @@ async def create_namespaced_job( namespace: The Kubernetes namespace to create this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - kube_kwargs: Optional extra keyword arguments to pass to the + **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -67,9 +67,9 @@ async def delete_namespaced_job( namespace: The Kubernetes namespace to delete this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - kube_kwargs: Optional extra keyword arguments to pass to the + **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). - delete_option_kwargs: Optional keyword arguments to pass to + **delete_option_kwargs: Optional keyword arguments to pass to the V1DeleteOptions object (e.g. {"propagation_policy": "...", "grace_period_seconds": "..."}. @@ -118,7 +118,7 @@ async def list_namespaced_job( kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. namespace: The Kubernetes namespace to list jobs from. - kube_kwargs: Optional extra keyword arguments to pass to the + **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Returns: @@ -166,7 +166,7 @@ async def patch_namespaced_job( namespace: The Kubernetes namespace to patch this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - kube_kwargs: Optional extra keyword arguments to pass to the + **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Raises: @@ -218,7 +218,7 @@ async def read_namespaced_job( namespace: The Kubernetes namespace to read this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - kube_kwargs: Optional extra keyword arguments to pass to the + **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Raises: @@ -270,7 +270,7 @@ async def replace_namespaced_job( namespace: The Kubernetes namespace to replace this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - kube_kwargs: Optional extra keyword arguments to pass to the + **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Returns: From e5027ee10ea532055e794b22edbdd8de3f67bbfd Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 11:50:44 -0600 Subject: [PATCH 31/43] formattinga and unpack kube_kwargs --- docs/credentials.md | 2 +- docs/exceptions.md | 2 +- docs/jobs.md | 2 +- prefect_kubernetes/credentials.py | 2 +- prefect_kubernetes/jobs.py | 29 ++++++++++------------------- 5 files changed, 14 insertions(+), 23 deletions(-) diff --git a/docs/credentials.md b/docs/credentials.md index 7078caa..cc6bc7c 100644 --- a/docs/credentials.md +++ b/docs/credentials.md @@ -1 +1 @@ -:::prefect_kubernetes.credentials \ No newline at end of file +::: prefect_kubernetes.credentials \ No newline at end of file diff --git a/docs/exceptions.md b/docs/exceptions.md index bc23728..d8bc24f 100644 --- a/docs/exceptions.md +++ b/docs/exceptions.md @@ -1 +1 @@ -:::prefect_kubernetes.exceptions \ No newline at end of file +::: prefect_kubernetes.exceptions \ No newline at end of file diff --git a/docs/jobs.md b/docs/jobs.md index a8728be..bce587b 100644 --- a/docs/jobs.md +++ b/docs/jobs.md @@ -1 +1 @@ -:::prefect_kubernetes.jobs \ No newline at end of file +::: prefect_kubernetes.jobs \ No newline at end of file diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 85431ac..7ab7872 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -22,7 +22,7 @@ class KubernetesCredentials(Block): """Credentials block for generating configured Kubernetes API clients. - Args: + Attributes: cluster_config: a `KubernetesClusterConfig` block holding a JSON kube config for a specific kubernetes context diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index f994290..b8b4077 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -15,7 +15,7 @@ async def create_namespaced_job( body: Dict, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict] = None, + **kube_kwargs: Optional[Dict], ) -> V1Job: """Task for creating a namespaced Kubernetes job. @@ -57,8 +57,8 @@ async def delete_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict[str, Any]] = None, delete_option_kwargs: Optional[Dict] = None, + **kube_kwargs: Optional[Dict[str, Any]], ) -> V1Status: """Task for deleting a namespaced Kubernetes job. @@ -67,11 +67,12 @@ async def delete_namespaced_job( namespace: The Kubernetes namespace to delete this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - **kube_kwargs: Optional extra keyword arguments to pass to the - Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). - **delete_option_kwargs: Optional keyword arguments to pass to + delete_option_kwargs: Optional keyword arguments to pass to the V1DeleteOptions object (e.g. {"propagation_policy": "...", "grace_period_seconds": "..."}. + **kube_kwargs: Optional extra keyword arguments to pass to the + Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). + Returns: A Kubernetes `V1Status` object. @@ -93,8 +94,6 @@ def kubernetes_orchestrator(): api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = kube_kwargs or {} - if delete_option_kwargs: kube_kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) @@ -110,7 +109,7 @@ def kubernetes_orchestrator(): async def list_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict[str, Any]] = None, + **kube_kwargs: Optional[Dict[str, Any]], ) -> V1JobList: """Task for listing namespaced Kubernetes jobs. @@ -140,8 +139,6 @@ def kubernetes_orchestrator(): """ api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = kube_kwargs or {} - return await run_sync_in_worker_thread( api_client.list_namespaced_job, namespace=namespace, @@ -155,7 +152,7 @@ async def patch_namespaced_job( body: Dict, kubernetes_credentials: KubernetesCredentials = None, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict[str, Any]] = None, + **kube_kwargs: Optional[Dict[str, Any]], ) -> V1Job: """Task for deleting a namespaced Kubernetes job. @@ -193,8 +190,6 @@ def kubernetes_orchestrator(): api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = kube_kwargs or {} - return await run_sync_in_worker_thread( api_client.patch_namespaced_job, name=job_name, @@ -209,7 +204,7 @@ async def read_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict[str, Any]] = None, + **kube_kwargs: Optional[Dict[str, Any]], ) -> V1Job: """Task for reading a namespaced Kubernetes job. @@ -243,8 +238,6 @@ def kubernetes_orchestrator(): """ api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = kube_kwargs or {} - return await run_sync_in_worker_thread( api_client.read_namespaced_job, name=job_name, @@ -259,7 +252,7 @@ async def replace_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - kube_kwargs: Optional[Dict[str, Any]] = None, + **kube_kwargs: Optional[Dict[str, Any]], ) -> V1Job: """Task for replacing a namespaced Kubernetes job. @@ -293,8 +286,6 @@ def kubernetes_orchestrator(): """ api_client = kubernetes_credentials.get_batch_client() - kube_kwargs = kube_kwargs or {} - return await run_sync_in_worker_thread( api_client.replace_namespaced_job, name=job_name, From 6dd305aa34afb75ed4a060c1ccb389bbd7929364 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 12:00:44 -0600 Subject: [PATCH 32/43] fix tests for kube_kwargs, rm delete option kwargs --- prefect_kubernetes/jobs.py | 12 +++--------- tests/test_jobs.py | 14 +++++++------- 2 files changed, 10 insertions(+), 16 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index b8b4077..29c5ff4 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -2,7 +2,6 @@ from typing import Any, Dict, Optional -from kubernetes import client from kubernetes.client.models import V1Job, V1JobList, V1Status from prefect import task from prefect.utilities.asyncutils import run_sync_in_worker_thread @@ -57,7 +56,6 @@ async def delete_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - delete_option_kwargs: Optional[Dict] = None, **kube_kwargs: Optional[Dict[str, Any]], ) -> V1Status: """Task for deleting a namespaced Kubernetes job. @@ -67,9 +65,6 @@ async def delete_namespaced_job( namespace: The Kubernetes namespace to delete this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. - delete_option_kwargs: Optional keyword arguments to pass to - the V1DeleteOptions object (e.g. {"propagation_policy": "...", - "grace_period_seconds": "..."}. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -80,6 +75,7 @@ async def delete_namespaced_job( Example: Delete "my-job" in the default namespace: ```python + from kubernetes.client.models import V1DeleteOptions from prefect import flow from prefect_kubernetes.jobs import delete_namespaced_job @@ -88,15 +84,13 @@ def kubernetes_orchestrator(): v1_job_status = delete_namespaced_job( job_name="my-job", kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), + body=V1DeleteOptions(propagation_policy="Foreground"), ) ``` """ api_client = kubernetes_credentials.get_batch_client() - if delete_option_kwargs: - kube_kwargs.update(body=client.V1DeleteOptions(**delete_option_kwargs)) - return await run_sync_in_worker_thread( api_client.delete_namespaced_job, name=job_name, @@ -258,7 +252,7 @@ async def replace_namespaced_job( Args: body: A dictionary representation of a Kubernetes V1Job - specification + specification. job_name: The name of a job to replace. namespace: The Kubernetes namespace to replace this job in. kubernetes_credentials: KubernetesCredentials block diff --git a/tests/test_jobs.py b/tests/test_jobs.py index 6425d53..71515e6 100644 --- a/tests/test_jobs.py +++ b/tests/test_jobs.py @@ -12,7 +12,7 @@ async def test_invalid_body_raises_error(kubernetes_credentials): - with pytest.raises(TypeError): + with pytest.raises(ApiValueError): await create_namespaced_job.fn( body=None, kubernetes_credentials=kubernetes_credentials ) @@ -25,7 +25,7 @@ async def test_invalid_body_raises_error(kubernetes_credentials): async def test_create_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await create_namespaced_job.fn( body={"test": "a"}, - kube_kwargs={"a": "test"}, + a="test", kubernetes_credentials=kubernetes_credentials, ) @@ -38,7 +38,7 @@ async def test_create_namespaced_job(kubernetes_credentials, _mock_api_batch_cli async def test_delete_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await delete_namespaced_job.fn( job_name="test_job", - kube_kwargs={"a": "test"}, + a="test", kubernetes_credentials=kubernetes_credentials, ) assert ( @@ -50,7 +50,7 @@ async def test_delete_namespaced_job(kubernetes_credentials, _mock_api_batch_cli async def test_list_namespaced_job(kubernetes_credentials, _mock_api_batch_client): await list_namespaced_job.fn( namespace="ns", - kube_kwargs={"a": "test"}, + a="test", kubernetes_credentials=kubernetes_credentials, ) assert _mock_api_batch_client.list_namespaced_job.call_args[1]["namespace"] == "ns" @@ -61,7 +61,7 @@ async def test_patch_namespaced_job(kubernetes_credentials, _mock_api_batch_clie await patch_namespaced_job.fn( body={"test": "a"}, job_name="test_job", - kube_kwargs={"a": "test"}, + a="test", kubernetes_credentials=kubernetes_credentials, ) assert _mock_api_batch_client.patch_namespaced_job.call_args[1]["body"] == { @@ -77,7 +77,7 @@ async def test_read_namespaced_job(kubernetes_credentials, _mock_api_batch_clien await read_namespaced_job.fn( job_name="test_job", namespace="ns", - kube_kwargs={"a": "test"}, + a="test", kubernetes_credentials=kubernetes_credentials, ) assert _mock_api_batch_client.read_namespaced_job.call_args[1]["name"] == "test_job" @@ -90,7 +90,7 @@ async def test_replace_namespaced_job(kubernetes_credentials, _mock_api_batch_cl job_name="test_job", body={"test": "a"}, namespace="ns", - kube_kwargs={"a": "test"}, + a="test", kubernetes_credentials=kubernetes_credentials, ) assert ( From 217a152dc77a8f76a7b21dc7e34c2491c953cdc4 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 12:28:20 -0600 Subject: [PATCH 33/43] explicitly list tasks and body models --- CHANGELOG.md | 4 ++-- prefect_kubernetes/jobs.py | 12 +++++++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 531fcf3..7d454b7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,8 +8,8 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## Unreleased ### Added -- Credentials block for generating authenticated Kubernetes clients - [#19](https://github.com/PrefectHQ/prefect-kubernetes/pull/19) -- Tasks for interacting with `job` resources - [#19](https://github.com/PrefectHQ/prefect-kubernetes/pull/19) +- `KubernetesCredentials` block for generating authenticated Kubernetes clients - [#19](https://github.com/PrefectHQ/prefect-kubernetes/pull/19) +- Tasks for interacting with `job` resources: `create_namespaced_job`, `delete_namespaced_job`, `list_namespaced_job`, `patch_namespaced_job`, `read_namespaced_job`, `replace_namespaced_job` - [#19](https://github.com/PrefectHQ/prefect-kubernetes/pull/19) ### Changed diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 29c5ff4..bc5ee95 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -11,7 +11,7 @@ @task async def create_namespaced_job( - body: Dict, + body: V1Job, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", **kube_kwargs: Optional[Dict], @@ -34,6 +34,7 @@ async def create_namespaced_job( Create a job in the default namespace: ```python from prefect import flow + from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import create_namespaced_job @flow @@ -77,6 +78,7 @@ async def delete_namespaced_job( ```python from kubernetes.client.models import V1DeleteOptions from prefect import flow + from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import delete_namespaced_job @flow @@ -121,6 +123,7 @@ async def list_namespaced_job( List jobs in "my-namespace": ```python from prefect import flow + from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import list_namespaced_job @flow @@ -143,7 +146,7 @@ def kubernetes_orchestrator(): @task async def patch_namespaced_job( job_name: str, - body: Dict, + body: V1Job, kubernetes_credentials: KubernetesCredentials = None, namespace: Optional[str] = "default", **kube_kwargs: Optional[Dict[str, Any]], @@ -170,6 +173,7 @@ async def patch_namespaced_job( Patch "my-job" in the default namespace: ```python from prefect import flow + from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import patch_namespaced_job @flow @@ -220,6 +224,7 @@ async def read_namespaced_job( Read "my-job" in the default namespace: ```python from prefect import flow + from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import read_namespaced_job @flow @@ -242,7 +247,7 @@ def kubernetes_orchestrator(): @task async def replace_namespaced_job( - body: dict, + body: V1Job, job_name: str, kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", @@ -267,6 +272,7 @@ async def replace_namespaced_job( Replace "my-job" in the default namespace: ```python from prefect import flow + from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import replace_namespaced_job @flow From d5a8ac88f4cc1ab5c4ff14b926275489fd60885f Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 12:41:21 -0600 Subject: [PATCH 34/43] use contentful logo img --- prefect_kubernetes/credentials.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 7ab7872..d57ff27 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -62,7 +62,7 @@ def kubernetes_orchestrator(): """ _block_type_name = "Kubernetes Credentials" - _logo_url = "https://kubernetes-security.info/assets/img/logo.png?h=250" # noqa + _logo_url = "https://images.ctfassets.net/zscdif0zqppk/oYuHjIbc26oilfQSEMjRv/a61f5f6ef406eead2df5231835b4c4c2/logo.png?h=250" # noqa cluster_config: Optional[KubernetesClusterConfig] = None From fb974b7bf9ef43d99b7600313c6c46814ca65761 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 12:44:59 -0600 Subject: [PATCH 35/43] add creds import to readme example --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index a5c17c0..314b476 100644 --- a/README.md +++ b/README.md @@ -27,6 +27,7 @@ pip install prefect-kubernetes ```python from prefect import flow +from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import list_namespaced_job @flow From 8c5539dd1405ebacb1a0d6f5e1d238ff32a22f31 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 13:14:50 -0600 Subject: [PATCH 36/43] formatting and update examples for models --- prefect_kubernetes/credentials.py | 17 +++++++++------- prefect_kubernetes/exceptions.py | 6 +++--- prefect_kubernetes/jobs.py | 32 +++++++++++++++++-------------- 3 files changed, 31 insertions(+), 24 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index d57ff27..ea1a7f0 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -23,8 +23,8 @@ class KubernetesCredentials(Block): """Credentials block for generating configured Kubernetes API clients. Attributes: - cluster_config: a `KubernetesClusterConfig` block holding a JSON kube - config for a specific kubernetes context + cluster_config: A `KubernetesClusterConfig` block holding a JSON kube + config for a specific kubernetes context. Examples: Load stored Kubernetes credentials: @@ -51,12 +51,15 @@ class KubernetesCredentials(Block): from prefect_kubernetes import KubernetesCredentials from prefect_kubernetes.job import create_namespaced_job + from kubernetes.client.models import V1Job + kubernetes_credentials = KubernetesCredentials.load("my-k8s-credentials") @flow def kubernetes_orchestrator(): create_namespaced_job( - body={"Marvin": "42"}, kubernetes_credentials=kubernetes_credentials + body=V1Job(**{"Marvin": "42"}), + kubernetes_credentials=kubernetes_credentials ) ``` """ @@ -67,19 +70,19 @@ def kubernetes_orchestrator(): cluster_config: Optional[KubernetesClusterConfig] = None def get_core_client(self) -> client.CoreV1Api: - """Convenience method for retrieving a kubernetes api client for core resources + """Convenience method for retrieving a kubernetes api client for core resources. Returns: client.CoreV1Api: Kubernetes api client to interact with "pod", "service" - and "secret" resources + and "secret" resources. """ return self.get_kubernetes_client(resource="core") def get_batch_client(self) -> client.BatchV1Api: - """Convenience method for retrieving a kubernetes api client for job resources + """Convenience method for retrieving a kubernetes api client for job resources. Returns: - client.BatchV1Api: Kubernetes api client to interact with "job" resources + client.BatchV1Api: Kubernetes api client to interact with "job" resources. """ return self.get_kubernetes_client(resource="job") diff --git a/prefect_kubernetes/exceptions.py b/prefect_kubernetes/exceptions.py index 27ad2af..3a54def 100644 --- a/prefect_kubernetes/exceptions.py +++ b/prefect_kubernetes/exceptions.py @@ -4,12 +4,12 @@ class KubernetesJobDefinitionError(OpenApiException): - """An exception for when a Kubernetes job definition is invalid""" + """An exception for when a Kubernetes job definition is invalid.""" class KubernetesJobFailedError(OpenApiException): - """An exception for when a Kubernetes job fails""" + """An exception for when a Kubernetes job fails.""" class KubernetesResourceNotFoundError(ApiException): - """An exception for when a Kubernetes resource cannot be found by a client""" + """An exception for when a Kubernetes resource cannot be found by a client.""" diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index bc5ee95..26daa36 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -2,7 +2,7 @@ from typing import Any, Dict, Optional -from kubernetes.client.models import V1Job, V1JobList, V1Status +from kubernetes.client.models import V1DeleteOptions, V1Job, V1JobList, V1Status from prefect import task from prefect.utilities.asyncutils import run_sync_in_worker_thread @@ -19,9 +19,9 @@ async def create_namespaced_job( """Task for creating a namespaced Kubernetes job. Args: - body: A dictionary representing a Kubernetes V1Job specification. + body: A Kubernetes `V1Job` specification. namespace: The Kubernetes namespace to create this job in. - kubernetes_credentials: KubernetesCredentials block + kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -36,11 +36,12 @@ async def create_namespaced_job( from prefect import flow from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import create_namespaced_job + from kubernetes.client.models import V1Job @flow def kubernetes_orchestrator(): v1_job_metadata = create_namespaced_job( - body={"metadata": {"name": "test-job"}}, + body=V1Job(**{"metadata": {"name": "test-job"}}), kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), ) ``` @@ -56,6 +57,7 @@ def kubernetes_orchestrator(): async def delete_namespaced_job( job_name: str, kubernetes_credentials: KubernetesCredentials, + body: Optional[V1DeleteOptions] = None, namespace: Optional[str] = "default", **kube_kwargs: Optional[Dict[str, Any]], ) -> V1Status: @@ -63,9 +65,10 @@ async def delete_namespaced_job( Args: job_name: The name of a job to delete. - namespace: The Kubernetes namespace to delete this job in. - kubernetes_credentials: KubernetesCredentials block + kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. + body: A Kubernetes `V1DeleteOptions` object. + namespace: The Kubernetes namespace to delete this job in. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -95,6 +98,7 @@ def kubernetes_orchestrator(): return await run_sync_in_worker_thread( api_client.delete_namespaced_job, + body=V1DeleteOptions(**body if body else None), name=job_name, namespace=namespace, **kube_kwargs, @@ -110,7 +114,7 @@ async def list_namespaced_job( """Task for listing namespaced Kubernetes jobs. Args: - kubernetes_credentials: KubernetesCredentials block + kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. namespace: The Kubernetes namespace to list jobs from. **kube_kwargs: Optional extra keyword arguments to pass to the @@ -155,8 +159,7 @@ async def patch_namespaced_job( Args: job_name: The name of a job to patch. - body: A dictionary representation of a Kubernetes V1Job - specification. + body: A Kubernetes `V1Job` specification. namespace: The Kubernetes namespace to patch this job in. kubernetes_credentials: KubernetesCredentials block holding authentication needed to generate the required API client. @@ -176,11 +179,13 @@ async def patch_namespaced_job( from prefect_kubernetes.credentials import KubernetesCredentials from prefect_kubernetes.jobs import patch_namespaced_job + from kubernetes.client.models import V1Job + @flow def kubernetes_orchestrator(): v1_job_metadata = patch_namespaced_job( job_name="my-job", - body={"metadata": {"labels": {"foo": "bar"}}}, + body=V1Job(**{"metadata": {"labels": {"foo": "bar"}}}), kubernetes_credentials=KubernetesCredentials.load("k8s-creds"), ) ``` @@ -209,7 +214,7 @@ async def read_namespaced_job( Args: job_name: The name of a job to read. namespace: The Kubernetes namespace to read this job in. - kubernetes_credentials: KubernetesCredentials block + kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -256,11 +261,10 @@ async def replace_namespaced_job( """Task for replacing a namespaced Kubernetes job. Args: - body: A dictionary representation of a Kubernetes V1Job - specification. + body: A Kubernetes `V1Job` specification. job_name: The name of a job to replace. namespace: The Kubernetes namespace to replace this job in. - kubernetes_credentials: KubernetesCredentials block + kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). From b29dbd8da87a86ff654eea696dd9e2ff320a41bf Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 13:41:36 -0600 Subject: [PATCH 37/43] handle delete options --- prefect_kubernetes/jobs.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 26daa36..46bf932 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -96,9 +96,11 @@ def kubernetes_orchestrator(): api_client = kubernetes_credentials.get_batch_client() + if body: + kube_kwargs.update(body=V1DeleteOptions(**body)) + return await run_sync_in_worker_thread( api_client.delete_namespaced_job, - body=V1DeleteOptions(**body if body else None), name=job_name, namespace=namespace, **kube_kwargs, From 9085acc800f4e6601796ca13e546cdd604eccad6 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Mon, 7 Nov 2022 14:02:24 -0600 Subject: [PATCH 38/43] pass body kwarg directly --- prefect_kubernetes/jobs.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 46bf932..faf0889 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -96,12 +96,10 @@ def kubernetes_orchestrator(): api_client = kubernetes_credentials.get_batch_client() - if body: - kube_kwargs.update(body=V1DeleteOptions(**body)) - return await run_sync_in_worker_thread( api_client.delete_namespaced_job, name=job_name, + body=body, namespace=namespace, **kube_kwargs, ) From f3cdefa53c591b1e21925744fb1be9bbe67af087 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 9 Nov 2022 12:48:36 -0600 Subject: [PATCH 39/43] define client creation methods as context managers --- prefect_kubernetes/credentials.py | 97 +++++++++++++------------- prefect_kubernetes/jobs.py | 111 +++++++++++++++--------------- tests/conftest.py | 30 +++++--- tests/test_credentials.py | 6 +- 4 files changed, 131 insertions(+), 113 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index ea1a7f0..72aab4d 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -1,23 +1,14 @@ """Module for defining Kubernetes credential handling and client generation.""" -from typing import Optional, Union +from contextlib import contextmanager +from typing import Generator, Optional -from kubernetes import client from kubernetes import config as kube_config +from kubernetes.client import ApiClient, AppsV1Api, BatchV1Api, CoreV1Api from kubernetes.config.config_exception import ConfigException from prefect.blocks.core import Block from prefect.blocks.kubernetes import KubernetesClusterConfig -KubernetesClient = Union[ - client.BatchV1Api, client.CoreV1Api, client.AppsV1Api, client.ApiClient -] - -K8S_CLIENTS = { - "job": client.BatchV1Api, - "core": client.CoreV1Api, - "deployment": client.AppsV1Api, -} - class KubernetesCredentials(Block): """Credentials block for generating configured Kubernetes API clients. @@ -69,66 +60,74 @@ def kubernetes_orchestrator(): cluster_config: Optional[KubernetesClusterConfig] = None - def get_core_client(self) -> client.CoreV1Api: - """Convenience method for retrieving a kubernetes api client for core resources. + @contextmanager + def get_app_client(self) -> Generator[AppsV1Api, None, None]: + """Convenience method for retrieving a kubernetes api client for deployment resources Returns: - client.CoreV1Api: Kubernetes api client to interact with "pod", "service" - and "secret" resources. + Kubernetes api client generator to interact with "deployment" resources. """ - return self.get_kubernetes_client(resource="core") - - def get_batch_client(self) -> client.BatchV1Api: + generic_client = self.get_kubernetes_client() + try: + yield AppsV1Api(api_client=generic_client) + finally: + generic_client.rest_client.pool_manager.clear() + + @contextmanager + def get_batch_client(self) -> Generator[BatchV1Api, None, None]: """Convenience method for retrieving a kubernetes api client for job resources. Returns: - client.BatchV1Api: Kubernetes api client to interact with "job" resources. + Kubernetes api client generator to interact with "job" resources. """ - return self.get_kubernetes_client(resource="job") - - def get_app_client(self) -> client.AppsV1Api: - """Convenience method for retrieving a kubernetes api client for deployment resources + generic_client = self.get_kubernetes_client() + try: + yield BatchV1Api(api_client=generic_client) + finally: + generic_client.rest_client.pool_manager.clear() + + @contextmanager + def get_core_client(self) -> Generator[CoreV1Api, None, None]: + """Convenience method for retrieving a kubernetes api client for core resources. Returns: - client.AppsV1Api: Kubernetes api client to interact with deployments + Kubernetes api client generator to interact with "pod", "service" + and "secret" resources. """ - return self.get_kubernetes_client(resource="deployment") + generic_client = self.get_kubernetes_client() + try: + yield CoreV1Api(api_client=generic_client) + finally: + generic_client.rest_client.pool_manager.clear() - def get_kubernetes_client(self, resource: str) -> KubernetesClient: + def get_kubernetes_client(self) -> ApiClient: """ - Utility function for loading kubernetes client object for a given resource. + Utility function for configuring a generic Kubernetes client. It will attempt to connect to a Kubernetes cluster in three steps with the first successful connection attempt becoming the mode of communication with a cluster: 1. It will first attempt to use a `KubernetesCredentials` block's `cluster_config` to configure a client using - `KubernetesClusterConfig.configure_client` and then return the - resource specific client. + `KubernetesClusterConfig.configure_client`. 2. Attempt in-cluster connection (will only work when running on a pod). + 3. Attempt out-of-cluster connection using the default location for a kube config file. - Args: - resource: the name of the resource to retrieve a client for. - Currently you can use one of these values: `job`, `core`, - or `deployment`. - Returns: - An authenticated and configured Kubernetes Client. + An authenticated, generic Kubernetes Client. """ - resource_specific_client = K8S_CLIENTS[resource] - - if self.cluster_config: - self.cluster_config.configure_client() - return resource_specific_client() - - else: - try: - kube_config.load_incluster_config() - except ConfigException: - kube_config.load_kube_config() - - return resource_specific_client() + with ApiClient() as client: + if self.cluster_config: + self.cluster_config.configure_client() + return client + else: + try: + kube_config.load_incluster_config() + except ConfigException: + kube_config.load_kube_config() + + return client diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index faf0889..52cc6b7 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -11,10 +11,10 @@ @task async def create_namespaced_job( - body: V1Job, kubernetes_credentials: KubernetesCredentials, + body: V1Job, namespace: Optional[str] = "default", - **kube_kwargs: Optional[Dict], + **kube_kwargs: Dict[str, Any], ) -> V1Job: """Task for creating a namespaced Kubernetes job. @@ -46,20 +46,23 @@ def kubernetes_orchestrator(): ) ``` """ - api_client = kubernetes_credentials.get_batch_client() + with kubernetes_credentials.get_batch_client() as api_client: - return await run_sync_in_worker_thread( - api_client.create_namespaced_job, namespace=namespace, body=body, **kube_kwargs - ) + return await run_sync_in_worker_thread( + api_client.create_namespaced_job, + namespace=namespace, + body=body, + **kube_kwargs, + ) @task async def delete_namespaced_job( - job_name: str, kubernetes_credentials: KubernetesCredentials, + job_name: str, body: Optional[V1DeleteOptions] = None, namespace: Optional[str] = "default", - **kube_kwargs: Optional[Dict[str, Any]], + **kube_kwargs: Dict[str, Any], ) -> V1Status: """Task for deleting a namespaced Kubernetes job. @@ -94,22 +97,22 @@ def kubernetes_orchestrator(): ``` """ - api_client = kubernetes_credentials.get_batch_client() + with kubernetes_credentials.get_batch_client() as api_client: - return await run_sync_in_worker_thread( - api_client.delete_namespaced_job, - name=job_name, - body=body, - namespace=namespace, - **kube_kwargs, - ) + return await run_sync_in_worker_thread( + api_client.delete_namespaced_job, + name=job_name, + body=body, + namespace=namespace, + **kube_kwargs, + ) @task async def list_namespaced_job( kubernetes_credentials: KubernetesCredentials, namespace: Optional[str] = "default", - **kube_kwargs: Optional[Dict[str, Any]], + **kube_kwargs: Dict[str, Any], ) -> V1JobList: """Task for listing namespaced Kubernetes jobs. @@ -138,22 +141,22 @@ def kubernetes_orchestrator(): ) ``` """ - api_client = kubernetes_credentials.get_batch_client() + with kubernetes_credentials.get_batch_client() as api_client: - return await run_sync_in_worker_thread( - api_client.list_namespaced_job, - namespace=namespace, - **kube_kwargs, - ) + return await run_sync_in_worker_thread( + api_client.list_namespaced_job, + namespace=namespace, + **kube_kwargs, + ) @task async def patch_namespaced_job( + kubernetes_credentials: KubernetesCredentials, job_name: str, body: V1Job, - kubernetes_credentials: KubernetesCredentials = None, namespace: Optional[str] = "default", - **kube_kwargs: Optional[Dict[str, Any]], + **kube_kwargs: Dict[str, Any], ) -> V1Job: """Task for deleting a namespaced Kubernetes job. @@ -191,23 +194,23 @@ def kubernetes_orchestrator(): ``` """ - api_client = kubernetes_credentials.get_batch_client() + with kubernetes_credentials.get_batch_client() as api_client: - return await run_sync_in_worker_thread( - api_client.patch_namespaced_job, - name=job_name, - namespace=namespace, - body=body, - **kube_kwargs, - ) + return await run_sync_in_worker_thread( + api_client.patch_namespaced_job, + name=job_name, + namespace=namespace, + body=body, + **kube_kwargs, + ) @task async def read_namespaced_job( - job_name: str, kubernetes_credentials: KubernetesCredentials, + job_name: str, namespace: Optional[str] = "default", - **kube_kwargs: Optional[Dict[str, Any]], + **kube_kwargs: Dict[str, Any], ) -> V1Job: """Task for reading a namespaced Kubernetes job. @@ -240,23 +243,23 @@ def kubernetes_orchestrator(): ) ``` """ - api_client = kubernetes_credentials.get_batch_client() + with kubernetes_credentials.get_batch_client() as api_client: - return await run_sync_in_worker_thread( - api_client.read_namespaced_job, - name=job_name, - namespace=namespace, - **kube_kwargs, - ) + return await run_sync_in_worker_thread( + api_client.read_namespaced_job, + name=job_name, + namespace=namespace, + **kube_kwargs, + ) @task async def replace_namespaced_job( - body: V1Job, - job_name: str, kubernetes_credentials: KubernetesCredentials, + job_name: str, + body: V1Job, namespace: Optional[str] = "default", - **kube_kwargs: Optional[Dict[str, Any]], + **kube_kwargs: Dict[str, Any], ) -> V1Job: """Task for replacing a namespaced Kubernetes job. @@ -288,12 +291,12 @@ def kubernetes_orchestrator(): ) ``` """ - api_client = kubernetes_credentials.get_batch_client() - - return await run_sync_in_worker_thread( - api_client.replace_namespaced_job, - name=job_name, - body=body, - namespace=namespace, - **kube_kwargs, - ) + with kubernetes_credentials.get_batch_client() as api_client: + + return await run_sync_in_worker_thread( + api_client.replace_namespaced_job, + name=job_name, + body=body, + namespace=namespace, + **kube_kwargs, + ) diff --git a/tests/conftest.py b/tests/conftest.py index e047680..7fcad96 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -1,8 +1,10 @@ +from contextlib import contextmanager from pathlib import Path -from unittest import mock +from unittest.mock import MagicMock import pytest import yaml +from kubernetes.client import AppsV1Api, BatchV1Api, CoreV1Api from prefect.blocks.kubernetes import KubernetesClusterConfig from prefect_kubernetes.credentials import KubernetesCredentials @@ -18,7 +20,7 @@ def kube_config_dict(): @pytest.fixture def successful_job_status(): - job_status = mock.MagicMock() + job_status = MagicMock() job_status.status.active = None job_status.status.failed = None job_status.status.succeeded = 1 @@ -36,11 +38,15 @@ def kubernetes_credentials(kube_config_dict): @pytest.fixture def _mock_api_app_client(monkeypatch): - app_client = mock.MagicMock() + app_client = MagicMock(spec=AppsV1Api) + + @contextmanager + def get_app_client(_): + yield app_client monkeypatch.setattr( "prefect_kubernetes.credentials.KubernetesCredentials.get_app_client", - mock.MagicMock(return_value=app_client), + get_app_client, ) return app_client @@ -48,11 +54,15 @@ def _mock_api_app_client(monkeypatch): @pytest.fixture def _mock_api_batch_client(monkeypatch): - batch_client = mock.MagicMock() + batch_client = MagicMock(spec=BatchV1Api) + + @contextmanager + def get_batch_client(_): + yield batch_client monkeypatch.setattr( "prefect_kubernetes.credentials.KubernetesCredentials.get_batch_client", - mock.MagicMock(return_value=batch_client), + get_batch_client, ) return batch_client @@ -60,11 +70,15 @@ def _mock_api_batch_client(monkeypatch): @pytest.fixture def _mock_api_core_client(monkeypatch): - core_client = mock.MagicMock() + core_client = MagicMock(spec=CoreV1Api) + + @contextmanager + def get_core_client(_): + yield core_client monkeypatch.setattr( "prefect_kubernetes.credentials.KubernetesCredentials.get_core_client", - mock.MagicMock(return_value=core_client), + get_core_client, ) return core_client diff --git a/tests/test_credentials.py b/tests/test_credentials.py index 8040ba6..17fdcb6 100644 --- a/tests/test_credentials.py +++ b/tests/test_credentials.py @@ -11,5 +11,7 @@ ], ) def test_client_return_type(kubernetes_credentials, resource_type_method, client_type): - resource_specific_client = getattr(kubernetes_credentials, resource_type_method)() - assert isinstance(resource_specific_client, client_type) + resource_specific_client = getattr(kubernetes_credentials, resource_type_method) + + with resource_specific_client() as client: + assert isinstance(client, client_type) From 6740fd34aacfd62150be0a6e23d81cec0343a10e Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 9 Nov 2022 13:03:35 -0600 Subject: [PATCH 40/43] more appropriate name for generic client method --- prefect_kubernetes/credentials.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 72aab4d..92a7610 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -67,7 +67,7 @@ def get_app_client(self) -> Generator[AppsV1Api, None, None]: Returns: Kubernetes api client generator to interact with "deployment" resources. """ - generic_client = self.get_kubernetes_client() + generic_client = self.get_generic_client() try: yield AppsV1Api(api_client=generic_client) finally: @@ -80,7 +80,7 @@ def get_batch_client(self) -> Generator[BatchV1Api, None, None]: Returns: Kubernetes api client generator to interact with "job" resources. """ - generic_client = self.get_kubernetes_client() + generic_client = self.get_generic_client() try: yield BatchV1Api(api_client=generic_client) finally: @@ -94,13 +94,13 @@ def get_core_client(self) -> Generator[CoreV1Api, None, None]: Kubernetes api client generator to interact with "pod", "service" and "secret" resources. """ - generic_client = self.get_kubernetes_client() + generic_client = self.get_generic_client() try: yield CoreV1Api(api_client=generic_client) finally: generic_client.rest_client.pool_manager.clear() - def get_kubernetes_client(self) -> ApiClient: + def get_generic_client(self) -> ApiClient: """ Utility function for configuring a generic Kubernetes client. It will attempt to connect to a Kubernetes cluster in three steps with From da6565843a78caefa57abbceb6ff672ca50b8ee5 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 9 Nov 2022 14:30:21 -0600 Subject: [PATCH 41/43] reorder docstring args --- prefect_kubernetes/credentials.py | 69 +++++++++++++++---------------- prefect_kubernetes/jobs.py | 20 ++++----- 2 files changed, 44 insertions(+), 45 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 92a7610..41abbe4 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -62,43 +62,43 @@ def kubernetes_orchestrator(): @contextmanager def get_app_client(self) -> Generator[AppsV1Api, None, None]: - """Convenience method for retrieving a kubernetes api client for deployment resources + """Convenience method for retrieving a Kubernetes API client for deployment resources. - Returns: - Kubernetes api client generator to interact with "deployment" resources. + Yields: + Kubernetes API client generator to interact with "deployment" resources. """ - generic_client = self.get_generic_client() - try: - yield AppsV1Api(api_client=generic_client) - finally: - generic_client.rest_client.pool_manager.clear() + with self.get_generic_client() as generic_client: + try: + yield AppsV1Api(api_client=generic_client) + finally: + generic_client.rest_client.pool_manager.clear() @contextmanager def get_batch_client(self) -> Generator[BatchV1Api, None, None]: - """Convenience method for retrieving a kubernetes api client for job resources. + """Convenience method for retrieving a Kubernetes API client for job resources. - Returns: - Kubernetes api client generator to interact with "job" resources. + Yields: + Kubernetes API client generator to interact with "job" resources. """ - generic_client = self.get_generic_client() - try: - yield BatchV1Api(api_client=generic_client) - finally: - generic_client.rest_client.pool_manager.clear() + with self.get_generic_client() as generic_client: + try: + yield BatchV1Api(api_client=generic_client) + finally: + generic_client.rest_client.pool_manager.clear() @contextmanager def get_core_client(self) -> Generator[CoreV1Api, None, None]: - """Convenience method for retrieving a kubernetes api client for core resources. + """Convenience method for retrieving a Kubernetes API client for core resources. - Returns: - Kubernetes api client generator to interact with "pod", "service" + Yields: + Kubernetes API client to interact with "pod", "service" and "secret" resources. """ - generic_client = self.get_generic_client() - try: - yield CoreV1Api(api_client=generic_client) - finally: - generic_client.rest_client.pool_manager.clear() + with self.get_generic_client() as generic_client: + try: + yield CoreV1Api(api_client=generic_client) + finally: + generic_client.rest_client.pool_manager.clear() def get_generic_client(self) -> ApiClient: """ @@ -120,14 +120,13 @@ def get_generic_client(self) -> ApiClient: An authenticated, generic Kubernetes Client. """ - with ApiClient() as client: - if self.cluster_config: - self.cluster_config.configure_client() - return client - else: - try: - kube_config.load_incluster_config() - except ConfigException: - kube_config.load_kube_config() - - return client + if self.cluster_config: + self.cluster_config.configure_client() + else: + try: + kube_config.load_incluster_config() + except ConfigException: + kube_config.load_kube_config() + + client = ApiClient() + return client diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 52cc6b7..7b62865 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -19,10 +19,10 @@ async def create_namespaced_job( """Task for creating a namespaced Kubernetes job. Args: - body: A Kubernetes `V1Job` specification. - namespace: The Kubernetes namespace to create this job in. kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. + body: A Kubernetes `V1Job` specification. + namespace: The Kubernetes namespace to create this job in. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -67,9 +67,9 @@ async def delete_namespaced_job( """Task for deleting a namespaced Kubernetes job. Args: - job_name: The name of a job to delete. kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. + job_name: The name of a job to delete. body: A Kubernetes `V1DeleteOptions` object. namespace: The Kubernetes namespace to delete this job in. **kube_kwargs: Optional extra keyword arguments to pass to the @@ -161,11 +161,11 @@ async def patch_namespaced_job( """Task for deleting a namespaced Kubernetes job. Args: + kubernetes_credentials: KubernetesCredentials block + holding authentication needed to generate the required API client. job_name: The name of a job to patch. body: A Kubernetes `V1Job` specification. namespace: The Kubernetes namespace to patch this job in. - kubernetes_credentials: KubernetesCredentials block - holding authentication needed to generate the required API client. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -215,10 +215,10 @@ async def read_namespaced_job( """Task for reading a namespaced Kubernetes job. Args: - job_name: The name of a job to read. - namespace: The Kubernetes namespace to read this job in. kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. + job_name: The name of a job to read. + namespace: The Kubernetes namespace to read this job in. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). @@ -264,11 +264,11 @@ async def replace_namespaced_job( """Task for replacing a namespaced Kubernetes job. Args: - body: A Kubernetes `V1Job` specification. - job_name: The name of a job to replace. - namespace: The Kubernetes namespace to replace this job in. kubernetes_credentials: `KubernetesCredentials` block holding authentication needed to generate the required API client. + job_name: The name of a job to replace. + body: A Kubernetes `V1Job` specification. + namespace: The Kubernetes namespace to replace this job in. **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). From 4a67a43d5e635c31262dd04b2c4a44aa6f34cde9 Mon Sep 17 00:00:00 2001 From: Nathan Nowack Date: Wed, 9 Nov 2022 14:33:31 -0600 Subject: [PATCH 42/43] rm "generator" --- prefect_kubernetes/credentials.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/prefect_kubernetes/credentials.py b/prefect_kubernetes/credentials.py index 41abbe4..96f287b 100644 --- a/prefect_kubernetes/credentials.py +++ b/prefect_kubernetes/credentials.py @@ -65,7 +65,7 @@ def get_app_client(self) -> Generator[AppsV1Api, None, None]: """Convenience method for retrieving a Kubernetes API client for deployment resources. Yields: - Kubernetes API client generator to interact with "deployment" resources. + Kubernetes API client to interact with "deployment" resources. """ with self.get_generic_client() as generic_client: try: @@ -78,7 +78,7 @@ def get_batch_client(self) -> Generator[BatchV1Api, None, None]: """Convenience method for retrieving a Kubernetes API client for job resources. Yields: - Kubernetes API client generator to interact with "job" resources. + Kubernetes API client to interact with "job" resources. """ with self.get_generic_client() as generic_client: try: From 368ff4c6c06273257dcba44b89fa9f79301b122a Mon Sep 17 00:00:00 2001 From: Andrew <15331990+ahuang11@users.noreply.github.com> Date: Wed, 9 Nov 2022 12:39:05 -0800 Subject: [PATCH 43/43] Apply suggestions from code review --- prefect_kubernetes/jobs.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/prefect_kubernetes/jobs.py b/prefect_kubernetes/jobs.py index 7b62865..7287eaf 100644 --- a/prefect_kubernetes/jobs.py +++ b/prefect_kubernetes/jobs.py @@ -26,7 +26,6 @@ async def create_namespaced_job( **kube_kwargs: Optional extra keyword arguments to pass to the Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). - Returns: A Kubernetes `V1Job` object. @@ -170,7 +169,7 @@ async def patch_namespaced_job( Kubernetes API (e.g. `{"pretty": "...", "dry_run": "..."}`). Raises: - ValueError: if `job_name` is `None` + ValueError: if `job_name` is `None`. Returns: A Kubernetes `V1Job` object.