From 47dce70ead38e17598a88743910687cdabdcc82a Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 3 Oct 2025 09:16:03 -0500 Subject: [PATCH 01/58] app code working but needs cleanup and testing --- .../overlord/KubernetesOverlordModule.java | 2 + .../k8s/overlord/KubernetesTaskRunner.java | 3 +- .../common/DruidKubernetesClient.java | 122 +++++++++ .../overlord/common/KubernetesClientApi.java | 11 + .../common/KubernetesInformerExecutor.java | 9 + .../overlord/common/KubernetesPeonClient.java | 258 +++++++++++++----- .../overlord/KubernetesTaskRunnerTest.java | 6 +- .../common/KubernetesPeonClientTest.java | 22 +- .../overlord/common/TestKubernetesClient.java | 27 ++ .../DruidPeonClientIntegrationTest.java | 4 +- 10 files changed, 376 insertions(+), 88 deletions(-) create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index 55fb08627b2c..4e8a061e4cd6 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -133,6 +133,8 @@ public DruidKubernetesClient makeKubernetesClient( config.setHttpProxy(null); } + config.setNamespace(kubernetesTaskRunnerConfig.getNamespace()); + client = new DruidKubernetesClient(httpClientConfig, config); lifecycle.addHandler( diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 51913d13a71e..dc8ccffd6380 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -347,7 +347,8 @@ public void start() log.info("Starting K8sTaskRunner..."); // Load tasks from previously running jobs and wait for their statuses to start running. final List> taskStatusActiveList = new ArrayList<>(); - final List peonJobs = client.getPeonJobs(); + // Get all existing peon jobs. + final List peonJobs = client.getPeonJobs(true); log.info("Locating [%,d] active tasks.", peonJobs.size()); for (Job job : peonJobs) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 9c1940bb1f44..8adf8efdb758 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -19,13 +19,30 @@ package org.apache.druid.k8s.overlord.common; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.Config; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientBuilder; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; public class DruidKubernetesClient implements KubernetesClientApi { + private static final EmittingLogger log = new EmittingLogger(DruidKubernetesClient.class); + + private static final long INFORMER_RESYNC_PERIOD_MS = 30 * 1000L; // todo make this configurable by operator + private final KubernetesClient kubernetesClient; + private final SharedIndexInformer podInformer; + private final SharedIndexInformer jobInformer; public DruidKubernetesClient(DruidKubernetesHttpClientConfig httpClientConfig, Config kubernetesClientConfig) { @@ -33,6 +50,8 @@ public DruidKubernetesClient(DruidKubernetesHttpClientConfig httpClientConfig, C .withHttpClientFactory(new DruidKubernetesHttpClientFactory(httpClientConfig)) .withConfig(kubernetesClientConfig) .build(); + this.podInformer = setupPodInformer(kubernetesClient.getNamespace()); + this.jobInformer = setupJobInformer(kubernetesClient.getNamespace()); } @Override @@ -41,6 +60,19 @@ public T executeRequest(KubernetesExecutor executor) throws KubernetesRes return executor.executeRequest(kubernetesClient); } + @Override + public T executePodCacheRequest(KubernetesInformerExecutor executor) + { + return executor.executeRequest(podInformer); + } + + @Override + public T executeJobCacheRequest(KubernetesInformerExecutor executor) + { + return executor.executeRequest(jobInformer); + } + + /** * This client automatically gets closed by the druid lifecycle, it should not be closed when used as it is * meant to be reused. @@ -52,4 +84,94 @@ public KubernetesClient getClient() { return this.kubernetesClient; } + + @Override + public SharedIndexInformer getPodInformer() + { + return podInformer; + } + + @Override + public SharedIndexInformer getJobInformer() + { + return jobInformer; + } + + private SharedIndexInformer setupPodInformer(String namespace) + { + SharedIndexInformer podInformer = + kubernetesClient.pods() + .inNamespace(namespace) + .inform(new ResourceEventHandler<>() { + @Override + public void onAdd(Pod pod) { + log.info("Pod " + pod.getMetadata().getName() + " got added"); + } + @Override + public void onUpdate(Pod oldPod, Pod newPod) { + log.info("Pod " + oldPod.getMetadata().getName() + " got updated"); + } + @Override + public void onDelete(Pod pod, boolean deletedFinalStateUnknown) { + log.info("Pod " + pod.getMetadata().getName() + " got deleted"); + } + }, INFORMER_RESYNC_PERIOD_MS); + + Function> jobNameIndexer = pod -> { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + return Collections.singletonList(jobName); + } + } + return Collections.emptyList(); + }; + + Map>> customPodIndexers = new HashMap<>(); + customPodIndexers.put("byJobName", jobNameIndexer); + + podInformer.addIndexers(customPodIndexers); + return podInformer; + } + + private SharedIndexInformer setupJobInformer(String namespace) + { + SharedIndexInformer jobInformer = + kubernetesClient.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .inform(new ResourceEventHandler<>() { + @Override + public void onAdd(Job job) { + log.info("Job " + job.getMetadata().getName() + " got added"); + } + @Override + public void onUpdate(Job oldJob, Job newJob) { + log.info("Job " + oldJob.getMetadata().getName() + " got updated"); + } + @Override + public void onDelete(Job job, boolean deletedFinalStateUnknown) { + log.info("Job " + job.getMetadata().getName() + " got deleted"); + } + }, INFORMER_RESYNC_PERIOD_MS); + + Function> overlordNamespaceIndexer = job -> { + if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { + String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); + if (overlordNamespace != null) { + return Collections.singletonList(overlordNamespace); + } + } + return Collections.emptyList(); + }; + + Map>> customJobIndexers = new HashMap<>(); + customJobIndexers.put("byOverlordNamespace", overlordNamespaceIndexer); + + jobInformer.addIndexers(customJobIndexers); + + return jobInformer; + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java index 655e4435f205..1569430cf1a7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java @@ -19,15 +19,26 @@ package org.apache.druid.k8s.overlord.common; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; // Wraps all kubernetes api calls, to ensure you open and close connections properly public interface KubernetesClientApi { T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException; + T executePodCacheRequest(KubernetesInformerExecutor executor); + + T executeJobCacheRequest(KubernetesInformerExecutor executor); + // use only when handling streams of data, example if you want to pass around an input stream from a pod, // then you would call this instead of executeRequest as you would want to keep the connection open until you // are done with the stream. Callers responsibility to clean up when using this method KubernetesClient getClient(); + + SharedIndexInformer getPodInformer(); + + SharedIndexInformer getJobInformer(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java new file mode 100644 index 000000000000..0364caac368b --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java @@ -0,0 +1,9 @@ +package org.apache.druid.k8s.overlord.common; + +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; + +@FunctionalInterface +public interface KubernetesInformerExecutor +{ + T executeRequest(SharedIndexInformer informer); +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index b06e8efb824b..0b8b37179139 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -29,6 +29,7 @@ import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientException; import io.fabric8.kubernetes.client.dsl.LogWatch; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; import io.vertx.core.http.HttpClosedException; import org.apache.druid.error.DruidException; import org.apache.druid.indexing.common.task.IndexTaskUtils; @@ -55,6 +56,7 @@ public class KubernetesPeonClient private final String overlordNamespace; private final boolean debugJobs; private final ServiceEmitter emitter; + private final boolean useEventsAnalysisOnPodNotFound = false; public KubernetesPeonClient( KubernetesClientApi clientApi, @@ -111,28 +113,75 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) { - return clientApi.executeRequest(client -> { - Job job = client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .waitUntilCondition( - x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null - && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), - howLong, - unit - ); + long timeoutMs = unit.toMillis(howLong); + long startTime = System.currentTimeMillis(); + long pollInterval = 5000; + long jobAppearanceGracePeriodMs = 90000; // 90 seconds grace for job to appear in cache + + boolean jobSeenInCache = false; + + do { + if (!clientApi.getJobInformer().hasSynced()) { + // Checking before the informer has synced will likely result in a false negative. + try { + Thread.sleep(pollInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + continue; + } + + Job job = clientApi.executeJobCacheRequest((informer) -> + informer.getIndexer() + .byIndex("byOverlordNamespace", overlordNamespace).stream() + .filter(j -> taskId.getK8sJobName().equals(j.getMetadata().getName())) + .findFirst() + .orElse(null)); + if (job == null) { + long elapsed = System.currentTimeMillis() - startTime; + + // Give grace period for job to appear in cache after creation + if (!jobSeenInCache && elapsed < jobAppearanceGracePeriodMs) { + log.debug("Job [%s] not yet in cache, waiting... (elapsed: %d ms)", taskId, elapsed); + try { + Thread.sleep(pollInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + continue; + } + + // After grace period or if we've seen it before, job is truly missing log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); return new JobResponse(null, PeonPhase.FAILED); } - if (job.getStatus().getSucceeded() != null) { - return new JobResponse(job, PeonPhase.SUCCEEDED); + + // Job found! Mark that we've seen it + jobSeenInCache = true; + + // Check if job is complete + if (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) { + if (job.getStatus().getSucceeded() != null) { + return new JobResponse(job, PeonPhase.SUCCEEDED); + } + log.warn("Task %s failed with status %s", taskId, job.getStatus()); + return new JobResponse(job, PeonPhase.FAILED); } - log.warn("Task %s failed with status %s", taskId, job.getStatus()); - return new JobResponse(job, PeonPhase.FAILED); - }); + + // Job still running, wait and check again + try { + Thread.sleep(pollInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } while (System.currentTimeMillis() - startTime < timeoutMs); + + log.warn("Timed out waiting for job [%s] to complete", taskId); + return new JobResponse(null, PeonPhase.FAILED); } public boolean deletePeonJob(K8sTaskId taskId) @@ -200,41 +249,60 @@ public Optional getPeonLogs(K8sTaskId taskId) } } - public List getPeonJobs() + public List getPeonJobs(boolean useCache) { + if (useCache) { + if (!clientApi.getJobInformer().hasSynced()) { + log.warn("K8s job informer cache not synced, getting jobs directly from k8s"); + useCache = false; + } + } + return this.overlordNamespace.isEmpty() - ? getPeonJobsWithoutOverlordNamespaceKeyLabels() - : getPeonJobsWithOverlordNamespaceKeyLabels(); + ? getPeonJobsWithoutOverlordNamespaceKeyLabels(useCache) + : getPeonJobsWithOverlordNamespaceKeyLabels(useCache); } - private List getPeonJobsWithoutOverlordNamespaceKeyLabels() + private List getPeonJobsWithoutOverlordNamespaceKeyLabels(boolean useCache) { - return clientApi.executeRequest(client -> client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .list() - .getItems()); + if (useCache) { + return clientApi.executeJobCacheRequest(informer -> informer.getIndexer().list()); + } else { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .list() + .getItems()); + } } - private List getPeonJobsWithOverlordNamespaceKeyLabels() + private List getPeonJobsWithOverlordNamespaceKeyLabels(boolean useCache) { - return clientApi.executeRequest(client -> client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .withLabel(DruidK8sConstants.OVERLORD_NAMESPACE_KEY, overlordNamespace) - .list() - .getItems()); + if (useCache) { + return clientApi.executeJobCacheRequest(informer -> informer.getIndexer() + .byIndex("byOverlordNamespace", overlordNamespace)); + } else { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .withLabel( + DruidK8sConstants.OVERLORD_NAMESPACE_KEY, + overlordNamespace + ) + .list() + .getItems()); + } } public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit) { AtomicInteger numDeleted = new AtomicInteger(); return clientApi.executeRequest(client -> { - List jobs = getJobsToCleanup(getPeonJobs(), howFarBack, timeUnit); + List jobs = getJobsToCleanup(getPeonJobs(true), howFarBack, timeUnit); jobs.forEach(x -> { if (!client.batch() .v1() @@ -269,27 +337,23 @@ private List getJobsToCleanup(List candidates, long howFarBack, TimeUn public Optional getPeonPod(String jobName) { - return clientApi.executeRequest(client -> getPeonPod(client, jobName)); + return clientApi.executePodCacheRequest(informer -> getPeonPod(informer, jobName)); } - private Optional getPeonPod(KubernetesClient client, String jobName) + private Optional getPeonPod(SharedIndexInformer informer, String jobName) { - List pods = client.pods() - .inNamespace(namespace) - .withLabel("job-name", jobName) - .list() - .getItems(); + List pods = informer.getIndexer().byIndex("byJobName", jobName); return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); } public Pod waitForPodResultWithRetries(final Pod pod, long howLong, TimeUnit timeUnit) { - return clientApi.executeRequest(client -> waitForPodResultWithRetries(client, pod, howLong, timeUnit, 5, RetryUtils.DEFAULT_MAX_TRIES)); + return clientApi.executePodCacheRequest(informer -> waitForPodResultWithRetries(informer, pod, howLong, timeUnit, 5, RetryUtils.DEFAULT_MAX_TRIES)); } public Pod getPeonPodWithRetries(String jobName) { - return clientApi.executeRequest(client -> getPeonPodWithRetries(client, jobName, 5, RetryUtils.DEFAULT_MAX_TRIES)); + return clientApi.executePodCacheRequest(informer -> getPeonPodWithRetries(informer, jobName, 5, RetryUtils.DEFAULT_MAX_TRIES)); } public void createK8sJobWithRetries(Job job) @@ -359,33 +423,25 @@ void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, i * The method will wait up to the specified timeout for the pod to become ready, and retry the entire wait operation * if transient connection issues are encountered. * - * @param client the Kubernetes client to use for pod operations - * @param pod the pod to wait for - * @param howLong the maximum time to wait for the pod to become ready - * @param timeUnit the time unit for the wait timeout + * @param pod the pod to wait for + * @param howLong the maximum time to wait for the pod to become ready + * @param timeUnit the time unit for the wait timeout * @param quietTries number of initial retry attempts without logging warnings - * @param maxTries maximum total number of retry attempts + * @param maxTries maximum total number of retry attempts * @return the pod in its ready state, or null if the pod disappeared or wait operation failed * @throws DruidException if waiting fails after all retry attempts or encounters non-retryable errors */ @VisibleForTesting - Pod waitForPodResultWithRetries(KubernetesClient client, Pod pod, long howLong, TimeUnit timeUnit, int quietTries, int maxTries) + Pod waitForPodResultWithRetries(SharedIndexInformer informer, Pod pod, long howLong, TimeUnit timeUnit, int quietTries, int maxTries) { try { return RetryUtils.retry( - () -> client.pods() - .inNamespace(namespace) - .withName(pod.getMetadata().getName()) - .waitUntilCondition( - p -> { - if (p == null) { - return true; - } - return p.getStatus() != null && p.getStatus().getPodIP() != null; - }, howLong, timeUnit), - this::isRetryableTransientConnectionPoolException, quietTries, maxTries); - } - catch (Exception e) { + () -> waitForPodResultUsingCache(informer, pod, howLong, timeUnit), + this::isRetryableTransientConnectionPoolException, + quietTries, + maxTries + ); + } catch (Exception e) { throw DruidException.defensive(e, "Error when waiting for pod[%s] to start", pod.getMetadata().getName()); } } @@ -404,7 +460,7 @@ Pod waitForPodResultWithRetries(KubernetesClient client, Pod pod, long howLong, *
  • Pod not found scenarios, except when blacklisted error messages from {@link DruidK8sConstants#BLACKLISTED_PEON_POD_ERROR_MESSAGES} are encountered
  • * * - * @param client the Kubernetes client to use for pod and event operations + * @param informer the Kubernetes informer to use for pod and event operations * @param jobName the name of the job whose pod should be retrieved * @param quietTries number of initial retry attempts without logging warnings * @param maxTries maximum total number of retry attempts @@ -413,18 +469,24 @@ Pod waitForPodResultWithRetries(KubernetesClient client, Pod pod, long howLong, * @throws DruidException if retrieval fails due to other errors */ @VisibleForTesting - Pod getPeonPodWithRetries(KubernetesClient client, String jobName, int quietTries, int maxTries) + Pod getPeonPodWithRetries(SharedIndexInformer informer, String jobName, int quietTries, int maxTries) { try { return RetryUtils.retry( () -> { - Optional maybePod = getPeonPod(client, jobName); + Optional maybePod = getPeonPod(informer, jobName); if (maybePod.isPresent()) { return maybePod.get(); } - // If the pod is missing, we can take a look at job events to discover potential problems with pod creation. - List events = getPeonEvents(client, jobName); + List events; + if (useEventsAnalysisOnPodNotFound) { + // If the pod is missing, we can take a look at job events to discover potential problems with pod creation. + // This is an optional analysis step as it requires an additional API call and may not be desirable in all environments. + events = clientApi.executeRequest((client) -> getPeonEvents(client, jobName)); + } else { + events = List.of(); + } if (events.isEmpty()) { throw new KubernetesResourceNotFoundException("K8s pod with label[job-name=%s] not found", jobName); @@ -514,6 +576,60 @@ private List getPeonEvents(KubernetesClient client, String jobName) } } + private Pod waitForPodResultUsingCache(SharedIndexInformer informer, Pod pod, long howLong, TimeUnit timeUnit) + { + log.info("Waiting for pod[%s] to be in running state using pod cache", pod.getMetadata().getName()); + long timeoutMs = timeUnit.toMillis(howLong); + long startTime = System.currentTimeMillis(); + long pollInterval = 2000; // Poll every 2 seconds + + String podName = pod.getMetadata().getName(); + + do { + if (informer.hasSynced()) { + // Wait for informer to sync + try { + Thread.sleep(pollInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + continue; + } + + Pod currentPod = informer.getIndexer().list().stream() + .filter(p -> podName.equals(p.getMetadata().getName())) + .findFirst() + .orElse(null); + if (currentPod == null) { + // Pod disappeared + return null; + } + + // Check if pod is ready (has IP) + if (currentPod.getStatus() != null && currentPod.getStatus().getPodIP() != null) { + return currentPod; + } + + // Wait before polling again + long remainingTime = timeoutMs - (System.currentTimeMillis() - startTime); + if (remainingTime <= 0) { + break; + } + + try { + Thread.sleep(Math.min(pollInterval, remainingTime)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } while (System.currentTimeMillis() - startTime < timeoutMs); + + // Timeout - return null + return null; + } + + private void emitK8sPodMetrics(Task task, String metric, long durationMs) { ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 083bf2db0e6c..697d0566b86f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -136,7 +136,7 @@ protected KubernetesWorkItem joinAsync(Task task) .endMetadata() .build(); - EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job)); + EasyMock.expect(peonClient.getPeonJobs(false)).andReturn(ImmutableList.of(job)); EasyMock.expect(taskAdapter.toTask(job)).andReturn(task); EasyMock.expect(kubernetesPeonLifecycle.getTaskStartedSuccessfullyFuture()).andReturn( settableFuture @@ -192,7 +192,7 @@ protected KubernetesWorkItem joinAsync(Task task) .endMetadata() .build(); - EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job, job2)); + EasyMock.expect(peonClient.getPeonJobs(false)).andReturn(ImmutableList.of(job, job2)); EasyMock.expect(taskAdapter.toTask(job)).andReturn(task); EasyMock.expect(taskAdapter.toTask(job2)).andThrow(new IOException("deserialization exception")); @@ -235,7 +235,7 @@ protected KubernetesWorkItem joinAsync(Task task) .endMetadata() .build(); - EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job)); + EasyMock.expect(peonClient.getPeonJobs(false)).andReturn(ImmutableList.of(job)); EasyMock.expect(taskAdapter.toTask(job)).andThrow(new IOException()); replayAll(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java index 6bc7b4d283a9..2be1d8e5fc84 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java @@ -355,7 +355,7 @@ void test_getPeonJobs_withJob_returnsPodList() client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(); + List jobs = instance.getPeonJobs(false); Assertions.assertEquals(1, jobs.size()); } @@ -375,7 +375,7 @@ void test_getPeonJobs_withJobInDifferentNamespace_returnsPodList() client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(); + List jobs = instance.getPeonJobs(false); Assertions.assertEquals(1, jobs.size()); } @@ -395,7 +395,7 @@ void test_getPeonJobs_withJobInDifferentNamespaceButOverlordNamespaceNotSpecifie client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(); + List jobs = instance.getPeonJobs(false); Assertions.assertEquals(0, jobs.size()); } @@ -414,7 +414,7 @@ void test_getPeonJobs_withJobInSameNamespaceWithoutLabels_doesNotReturnPodList() client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(); + List jobs = instance.getPeonJobs(false); Assertions.assertEquals(0, jobs.size()); } @@ -422,7 +422,7 @@ void test_getPeonJobs_withJobInSameNamespaceWithoutLabels_doesNotReturnPodList() @Test void test_getPeonJobs_withoutJob_returnsEmptyList() { - List jobs = instance.getPeonJobs(); + List jobs = instance.getPeonJobs(false); Assertions.assertEquals(0, jobs.size()); } @@ -594,7 +594,7 @@ void test_getPeonPodWithRetries_withoutPod_raisesKubernetesResourceNotFoundExcep String k8sJobName = new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName(); KubernetesResourceNotFoundException e = Assertions.assertThrows( KubernetesResourceNotFoundException.class, - () -> instance.getPeonPodWithRetries(clientApi.getClient(), k8sJobName, 1, 1) + () -> instance.getPeonPodWithRetries(clientApi.getPodInformer(), k8sJobName, 1, 1) ); Assertions.assertEquals(e.getMessage(), @@ -632,7 +632,7 @@ void test_getPeonPodWithRetries_withoutPod_noRestartForBlacklistedEvent_raisesKu // Task declared to retry for 3 times should only try once when a blacklisted event message is found. KubernetesResourceNotFoundException e = Assertions.assertThrows( KubernetesResourceNotFoundException.class, - () -> instance.getPeonPodWithRetries(clientApi.getClient(), k8sJobName, 0, 3) + () -> instance.getPeonPodWithRetries(clientApi.getPodInformer(), k8sJobName, 0, 3) ); // Ensure event message is propagated to the users. @@ -798,8 +798,8 @@ void test_waitForPodResultWithRetries_withSuccessfulPodReady_returnsPod() // Should return the pod successfully Pod result = instance.waitForPodResultWithRetries( - clientApi.getClient(), - pod, + clientApi.getPodInformer(), + pod, 1, TimeUnit.SECONDS, 0, @@ -835,8 +835,8 @@ void test_waitForPodResultWithRetries_withNonRetryableFailure_throwsDruidExcepti DruidException e = Assertions.assertThrows( DruidException.class, () -> instance.waitForPodResultWithRetries( - clientApi.getClient(), - pod, + clientApi.getPodInformer(), + pod, 1, TimeUnit.MILLISECONDS, // Very short timeout to force failure 0, diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java index 57be98251a9a..e353847b61c1 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -19,7 +19,10 @@ package org.apache.druid.k8s.overlord.common; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; public class TestKubernetesClient implements KubernetesClientApi { @@ -37,9 +40,33 @@ public T executeRequest(KubernetesExecutor executor) throws KubernetesRes return executor.executeRequest(client); } + @Override + public T executePodCacheRequest(KubernetesInformerExecutor executor) + { + return null; + } + + @Override + public T executeJobCacheRequest(KubernetesInformerExecutor executor) + { + return null; + } + @Override public KubernetesClient getClient() { return client; } + + @Override + public SharedIndexInformer getPodInformer() + { + return null; + } + + @Override + public SharedIndexInformer getJobInformer() + { + return null; + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 446c2502fa11..b16a6e22ffcd 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -135,7 +135,7 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception peonClient.launchPeonJobAndWaitForStart(job, task, 1, TimeUnit.MINUTES); // there should be one job that is a k8s peon job that exists - List jobs = peonClient.getPeonJobs(); + List jobs = peonClient.getPeonJobs(false); assertEquals(1, jobs.size()); K8sTaskId taskId = new K8sTaskId(null, task.getId()); @@ -185,7 +185,7 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception assertTrue(peonClient.deletePeonJob(taskId)); // we cleaned up the job, none should exist - List existingJobs = peonClient.getPeonJobs(); + List existingJobs = peonClient.getPeonJobs(false); assertEquals(0, existingJobs.size()); } } From e4fbdf56e0de176bc98c59c192285a31b06ef376 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 3 Oct 2025 13:09:57 -0500 Subject: [PATCH 02/58] caching side cleaner. need to add back direct client --- .../common/DruidKubernetesClient.java | 8 + .../overlord/common/KubernetesPeonClient.java | 279 +++++------------- .../common/KubernetesPeonClientTest.java | 143 --------- .../DruidPeonClientIntegrationTest.java | 87 ------ 4 files changed, 86 insertions(+), 431 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 8adf8efdb758..5e0a2a100593 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -167,8 +167,16 @@ public void onDelete(Job job, boolean deletedFinalStateUnknown) { return Collections.emptyList(); }; + Function> jobNameIndexer = job -> { + if (job.getMetadata() != null && job.getMetadata().getName() != null) { + return Collections.singletonList(job.getMetadata().getName()); + } + return Collections.emptyList(); + }; + Map>> customJobIndexers = new HashMap<>(); customJobIndexers.put("byOverlordNamespace", overlordNamespaceIndexer); + customJobIndexers.put("byJobName", jobNameIndexer); jobInformer.addIndexers(customJobIndexers); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 0b8b37179139..1bf575dcb226 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -25,6 +25,7 @@ import io.fabric8.kubernetes.api.model.ObjectReference; import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder; import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodStatus; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientException; @@ -56,7 +57,6 @@ public class KubernetesPeonClient private final String overlordNamespace; private final boolean debugJobs; private final ServiceEmitter emitter; - private final boolean useEventsAnalysisOnPodNotFound = false; public KubernetesPeonClient( KubernetesClientApi clientApi, @@ -86,29 +86,24 @@ public KubernetesPeonClient( public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) throws IllegalStateException { long start = System.currentTimeMillis(); - // launch job - return clientApi.executeRequest(client -> { - String jobName = job.getMetadata().getName(); - - log.info("Submitting job[%s] for task[%s].", jobName, task.getId()); - createK8sJobWithRetries(job); - log.info("Submitted job[%s] for task[%s]. Waiting for POD to launch.", jobName, task.getId()); - // Wait for the pod to be available - Pod mainPod = getPeonPodWithRetries(jobName); - log.info("Pod for job[%s] launched for task[%s]. Waiting for pod to be in running state.", jobName, task.getId()); + // launch job + String jobName = job.getMetadata().getName(); + log.info("Submitting job[%s] for task[%s].", jobName, task.getId()); + createK8sJobWithRetries(job); + log.info("Submitted job[%s] for task[%s]. Waiting for POD to launch and be ready.", jobName, task.getId()); - // Wait for the pod to be in state running, completed, or failed. - Pod result = waitForPodResultWithRetries(mainPod, howLong, timeUnit); + // Wait for the Pod to be created and then reach ready state + Pod result = clientApi.executePodCacheRequest(informer -> waitForPeonPodCreateAndReady(informer, jobName, howLong, timeUnit)); - if (result == null) { - throw new ISE("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", task.getId()); - } - log.info("Pod for job[%s] is in state [%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); - long duration = System.currentTimeMillis() - start; - emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); - return result; - }); + // Evaluate result of job launch + if (result == null) { + throw new ISE("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", task.getId()); + } + log.info("Pod for job[%s] is in state [%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); + long duration = System.currentTimeMillis() - start; + emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); + return result; } public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) @@ -132,19 +127,28 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time continue; } - Job job = clientApi.executeJobCacheRequest((informer) -> - informer.getIndexer() - .byIndex("byOverlordNamespace", overlordNamespace).stream() - .filter(j -> taskId.getK8sJobName().equals(j.getMetadata().getName())) - .findFirst() - .orElse(null)); + Job job; + if (!overlordNamespace.isEmpty()) { + log.info("byOverlordNamespace lookup"); + job = clientApi.executeJobCacheRequest((informer) -> + informer.getIndexer() + .byIndex("byOverlordNamespace", overlordNamespace).stream() + .filter(j -> taskId.getK8sJobName() + .equals(j.getMetadata().getName())) + .findFirst() + .orElse(null)); + } else { + log.info("byJobName lookup"); + job = clientApi.executeJobCacheRequest(informer -> + informer.getIndexer().byIndex("byJobName", taskId.getK8sJobName()).stream().findFirst().orElse(null)); + } if (job == null) { long elapsed = System.currentTimeMillis() - startTime; // Give grace period for job to appear in cache after creation if (!jobSeenInCache && elapsed < jobAppearanceGracePeriodMs) { - log.debug("Job [%s] not yet in cache, waiting... (elapsed: %d ms)", taskId, elapsed); + log.info("Job [%s] not yet in cache, waiting... (elapsed: %d ms)", taskId, elapsed); try { Thread.sleep(pollInterval); } catch (InterruptedException e) { @@ -155,7 +159,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time } // After grace period or if we've seen it before, job is truly missing - log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); + log.warn("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); return new JobResponse(null, PeonPhase.FAILED); } @@ -165,6 +169,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time // Check if job is complete if (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) { if (job.getStatus().getSucceeded() != null) { + log.info("Task %s completed successfully", taskId); return new JobResponse(job, PeonPhase.SUCCEEDED); } log.warn("Task %s failed with status %s", taskId, job.getStatus()); @@ -340,22 +345,12 @@ public Optional getPeonPod(String jobName) return clientApi.executePodCacheRequest(informer -> getPeonPod(informer, jobName)); } - private Optional getPeonPod(SharedIndexInformer informer, String jobName) + private Optional getPeonPod(SharedIndexInformer informer, String jobName) { List pods = informer.getIndexer().byIndex("byJobName", jobName); return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); } - public Pod waitForPodResultWithRetries(final Pod pod, long howLong, TimeUnit timeUnit) - { - return clientApi.executePodCacheRequest(informer -> waitForPodResultWithRetries(informer, pod, howLong, timeUnit, 5, RetryUtils.DEFAULT_MAX_TRIES)); - } - - public Pod getPeonPodWithRetries(String jobName) - { - return clientApi.executePodCacheRequest(informer -> getPeonPodWithRetries(informer, jobName, 5, RetryUtils.DEFAULT_MAX_TRIES)); - } - public void createK8sJobWithRetries(Job job) { clientApi.executeRequest(client -> { @@ -413,127 +408,6 @@ void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, i } } - /** - * Waits for a Kubernetes pod to reach a ready state with retry logic for transient connection pool exceptions. - *

    - * This method waits for the specified pod to have a valid status with a pod IP assigned, indicating - * it has been scheduled and is in a ready state. The method includes retry logic to handle transient - * connection pool exceptions that may occur during the wait operation. - *

    - * The method will wait up to the specified timeout for the pod to become ready, and retry the entire wait operation - * if transient connection issues are encountered. - * - * @param pod the pod to wait for - * @param howLong the maximum time to wait for the pod to become ready - * @param timeUnit the time unit for the wait timeout - * @param quietTries number of initial retry attempts without logging warnings - * @param maxTries maximum total number of retry attempts - * @return the pod in its ready state, or null if the pod disappeared or wait operation failed - * @throws DruidException if waiting fails after all retry attempts or encounters non-retryable errors - */ - @VisibleForTesting - Pod waitForPodResultWithRetries(SharedIndexInformer informer, Pod pod, long howLong, TimeUnit timeUnit, int quietTries, int maxTries) - { - try { - return RetryUtils.retry( - () -> waitForPodResultUsingCache(informer, pod, howLong, timeUnit), - this::isRetryableTransientConnectionPoolException, - quietTries, - maxTries - ); - } catch (Exception e) { - throw DruidException.defensive(e, "Error when waiting for pod[%s] to start", pod.getMetadata().getName()); - } - } - - /** - * Retrieves the pod associated with a Kubernetes job with retry logic for transient failures. - *

    - * This method searches for a pod with the specified job name label and includes retry logic - * to handle both transient connection pool exceptions and cases where the pod may not be - * immediately available after job creation. If no pod is found, the method examines job - * events to provide detailed error information about pod creation failures. - *

    - * The retry logic applies to: - *

      - *
    • Transient connection pool exceptions
    • - *
    • Pod not found scenarios, except when blacklisted error messages from {@link DruidK8sConstants#BLACKLISTED_PEON_POD_ERROR_MESSAGES} are encountered
    • - *
    - * - * @param informer the Kubernetes informer to use for pod and event operations - * @param jobName the name of the job whose pod should be retrieved - * @param quietTries number of initial retry attempts without logging warnings - * @param maxTries maximum total number of retry attempts - * @return the pod associated with the job - * @throws KubernetesResourceNotFoundException if the pod cannot be found after all retry attempts - * @throws DruidException if retrieval fails due to other errors - */ - @VisibleForTesting - Pod getPeonPodWithRetries(SharedIndexInformer informer, String jobName, int quietTries, int maxTries) - { - try { - return RetryUtils.retry( - () -> { - Optional maybePod = getPeonPod(informer, jobName); - if (maybePod.isPresent()) { - return maybePod.get(); - } - - List events; - if (useEventsAnalysisOnPodNotFound) { - // If the pod is missing, we can take a look at job events to discover potential problems with pod creation. - // This is an optional analysis step as it requires an additional API call and may not be desirable in all environments. - events = clientApi.executeRequest((client) -> getPeonEvents(client, jobName)); - } else { - events = List.of(); - } - - if (events.isEmpty()) { - throw new KubernetesResourceNotFoundException("K8s pod with label[job-name=%s] not found", jobName); - } else { - Event latestEvent = events.get(events.size() - 1); - throw new KubernetesResourceNotFoundException( - "Job[%s] failed to create pods. Message[%s]", jobName, latestEvent.getMessage()); - } - }, - this::shouldRetryWaitForStartingPeonPod, quietTries, maxTries - ); - } - catch (KubernetesResourceNotFoundException e) { - throw e; - } - catch (Exception e) { - throw DruidException.defensive(e, "Error when looking for K8s pod with label[job-name=%s]", jobName); - } - } - - /** - * Determines if this exception, specifically when containing Kubernetes job event messages, permits a retry attempt. - *

    - * The method checks the exception message against a predefined list of Kubernetes event messages. - * These substrings, found in {@link DruidK8sConstants#BLACKLISTED_PEON_POD_ERROR_MESSAGES}, - * represent Kubernetes event that indicate a retry for starting the Peon Pod would likely be futile. - */ - private boolean shouldRetryWaitForStartingPeonPod(Throwable e) - { - if (isRetryableTransientConnectionPoolException(e)) { - return true; - } - - if (!(e instanceof KubernetesResourceNotFoundException)) { - return false; - } - - String errorMessage = e.getMessage(); - for (String blacklistedMessage : DruidK8sConstants.BLACKLISTED_PEON_POD_ERROR_MESSAGES) { - if (errorMessage.contains(blacklistedMessage)) { - return false; - } - } - - return true; - } - /** * Checks if the exception is a potentially transient connection pool exception. *

    @@ -553,40 +427,29 @@ private boolean isRetryableTransientConnectionPoolException(Throwable e) return false; } - private List getPeonEvents(KubernetesClient client, String jobName) - { - ObjectReference objectReference = new ObjectReferenceBuilder() - .withApiVersion("batch/v1") - .withKind("Job") - .withName(jobName) - .withNamespace(this.namespace) - .build(); - - try { - return client.v1() - .events() - .inNamespace(this.namespace) - .withInvolvedObject(objectReference) - .list() - .getItems(); - } - catch (KubernetesClientException e) { - log.warn("Failed to get events for job[%s]; %s", jobName, e.getMessage()); - return List.of(); - } - } - - private Pod waitForPodResultUsingCache(SharedIndexInformer informer, Pod pod, long howLong, TimeUnit timeUnit) + /** + * Waits for a pod associated with a job to be created and reach ready state using the pod cache. + * This method polls the informer cache until the pod appears and has a pod IP assigned. + * + * @param informer the pod informer to query + * @param jobName the name of the job whose pod we're waiting for + * @param howLong the maximum time to wait + * @param timeUnit the time unit for the timeout + * @return the pod in ready state, or null if the pod disappeared after being seen + * @throws DruidException if the pod never appears within the timeout period + */ + private Pod waitForPeonPodCreateAndReady(SharedIndexInformer informer, String jobName, long howLong, TimeUnit timeUnit) { - log.info("Waiting for pod[%s] to be in running state using pod cache", pod.getMetadata().getName()); + log.info("Waiting for pod for job[%s] to be created and ready using pod cache", jobName); long timeoutMs = timeUnit.toMillis(howLong); long startTime = System.currentTimeMillis(); long pollInterval = 2000; // Poll every 2 seconds - String podName = pod.getMetadata().getName(); + boolean podSeenInCache = false; + String podName = null; do { - if (informer.hasSynced()) { + if (!informer.hasSynced()) { // Wait for informer to sync try { Thread.sleep(pollInterval); @@ -597,18 +460,27 @@ private Pod waitForPodResultUsingCache(SharedIndexInformer informer, Pod po continue; } - Pod currentPod = informer.getIndexer().list().stream() - .filter(p -> podName.equals(p.getMetadata().getName())) - .findFirst() - .orElse(null); - if (currentPod == null) { - // Pod disappeared - return null; - } + List pods = informer.getIndexer().byIndex("byJobName", jobName); + + if (pods.isEmpty()) { + // If we've seen the pod before and now it's gone, it was deleted + if (podSeenInCache) { + log.info("Pod for job[%s] disappeared after being seen in cache", jobName); + return null; + } + // Otherwise keep waiting for it to appear + } else { + Pod currentPod = pods.get(0); + podSeenInCache = true; + podName = currentPod.getMetadata().getName(); + + // Check if pod is ready (has IP) + if (currentPod.getStatus() != null && currentPod.getStatus().getPodIP() != null) { + log.info("Pod[%s] for job[%s] is ready with IP: %s", podName, jobName, currentPod.getStatus().getPodIP()); + return currentPod; + } - // Check if pod is ready (has IP) - if (currentPod.getStatus() != null && currentPod.getStatus().getPodIP() != null) { - return currentPod; + log.info("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); } // Wait before polling again @@ -625,8 +497,13 @@ private Pod waitForPodResultUsingCache(SharedIndexInformer informer, Pod po } } while (System.currentTimeMillis() - startTime < timeoutMs); - // Timeout - return null - return null; + // Timeout + if (podSeenInCache) { + log.warn("Timeout waiting for pod[%s] for job[%s] to become ready", podName, jobName); + return null; + } else { + throw DruidException.defensive("Timeout waiting for pod for job[%s] to be created", jobName); + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java index 2be1d8e5fc84..d08fe5b15bed 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java @@ -563,82 +563,6 @@ void test_getPeonPod_withoutPod_returnsEmptyOptional() Assertions.assertFalse(maybePod.isPresent()); } - @Test - void test_getPeonPodWithRetries_withPod_returnsPod() - { - server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder().build()) - .once(); - - server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addToLabels("job-name", KUBERNETES_JOB_NAME) - .endMetadata() - .endItem() - .build() - ).once(); - - Pod pod = instance.getPeonPodWithRetries(new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName()); - - Assertions.assertNotNull(pod); - } - - @Test - void test_getPeonPodWithRetries_withoutPod_raisesKubernetesResourceNotFoundException() - { - String k8sJobName = new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName(); - KubernetesResourceNotFoundException e = Assertions.assertThrows( - KubernetesResourceNotFoundException.class, - () -> instance.getPeonPodWithRetries(clientApi.getPodInformer(), k8sJobName, 1, 1) - ); - - Assertions.assertEquals(e.getMessage(), - StringUtils.format("K8s pod with label[job-name=%s] not found", k8sJobName)); - } - - @Test - void test_getPeonPodWithRetries_withoutPod_noRestartForBlacklistedEvent_raisesKubernetesResourceNotFoundException() - { - String k8sJobName = new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName(); - String blacklistedMessage = DruidK8sConstants.BLACKLISTED_PEON_POD_ERROR_MESSAGES.get(0); - - final String eventsPath = "/api/v1/namespaces/namespace/events?fieldSelector=" + - "involvedObject.name%3D" + k8sJobName + - "%2CinvolvedObject.namespace%3D" + NAMESPACE + - "%2CinvolvedObject.kind%3DJob" + - "%2CinvolvedObject.apiVersion%3Dbatch%2Fv1"; - - server.expect().get() - .withPath(eventsPath) - .andReturn(HttpURLConnection.HTTP_OK, new EventListBuilder() - .addNewItem() - .withMessage(blacklistedMessage) - .withNewInvolvedObject() - .withApiVersion("batch/v1") - .withKind("Job") - .withName(k8sJobName) - .withNamespace(NAMESPACE) - .endInvolvedObject() - .endItem() - .build()) - // Test will fail if task is retried more than once. - .once(); - - // Task declared to retry for 3 times should only try once when a blacklisted event message is found. - KubernetesResourceNotFoundException e = Assertions.assertThrows( - KubernetesResourceNotFoundException.class, - () -> instance.getPeonPodWithRetries(clientApi.getPodInformer(), k8sJobName, 0, 3) - ); - - // Ensure event message is propagated to the users. - Assertions.assertTrue(e.getMessage().contains(blacklistedMessage)); - } - @Test void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() { @@ -780,71 +704,4 @@ void test_createK8sJobWithRetries_withJobAlreadyExists_succeedsGracefully() () -> instance.createK8sJobWithRetries(clientApi.getClient(), job, 0, 5) ); } - - @Test - void test_waitForPodResultWithRetries_withSuccessfulPodReady_returnsPod() - { - Pod pod = new PodBuilder() - .withNewMetadata() - .withName(POD_NAME) - .endMetadata() - .withNewStatus() - .withPodIP("192.168.1.100") - .endStatus() - .build(); - - // Create the pod in the mock client - client.pods().inNamespace(NAMESPACE).resource(pod).create(); - - // Should return the pod successfully - Pod result = instance.waitForPodResultWithRetries( - clientApi.getPodInformer(), - pod, - 1, - TimeUnit.SECONDS, - 0, - 3 - ); - - Assertions.assertNotNull(result); - Assertions.assertEquals(POD_NAME, result.getMetadata().getName()); - Assertions.assertEquals("192.168.1.100", result.getStatus().getPodIP()); - } - - @Test - void test_waitForPodResultWithRetries_withNonRetryableFailure_throwsDruidException() - { - Pod pod = new PodBuilder() - .withNewMetadata() - .withName(POD_NAME) - .endMetadata() - .withNewStatus() - .withPodIP(null) // Pod without IP, will timeout - .endStatus() - .build(); - - String podPath = "/api/v1/namespaces/" + NAMESPACE + "/pods/" + POD_NAME; - - // Mock server to return the pod without IP, causing timeout - server.expect().get() - .withPath(podPath + "?watch=true") - .andReturn(HttpURLConnection.HTTP_INTERNAL_ERROR, "Internal server error") - .once(); - - // Should throw DruidException after failure - DruidException e = Assertions.assertThrows( - DruidException.class, - () -> instance.waitForPodResultWithRetries( - clientApi.getPodInformer(), - pod, - 1, - TimeUnit.MILLISECONDS, // Very short timeout to force failure - 0, - 1 - ) - ); - - // Verify the error message contains our pod name - Assertions.assertTrue(e.getMessage().contains(POD_NAME)); - } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index b16a6e22ffcd..601692ec56c7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -101,91 +101,4 @@ public void setup() startupLoggingConfig = new StartupLoggingConfig(); taskConfig = new TaskConfigBuilder().setBaseDir("src/test/resources").build(); } - - @Disabled - @Test - public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception - { - PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); - - Task task = K8sTestUtils.getTask(); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("default") - .build(); - K8sTaskAdapter adapter = new SingleContainerTaskAdapter( - k8sClient, - config, - taskConfig, - startupLoggingConfig, - druidNode, - jsonMapper, - null - ); - String taskBasePath = "/home/taskDir"; - PeonCommandContext context = new PeonCommandContext( - Collections.singletonList("sleep 10; for i in `seq 1 1000`; do echo $i; done; exit 0"), - new ArrayList<>(), - new File(taskBasePath), - config.getCpuCoreInMicro() - ); - - Job job = adapter.createJobFromPodSpec(podSpec, task, context); - - // launch the job and wait to start... - peonClient.launchPeonJobAndWaitForStart(job, task, 1, TimeUnit.MINUTES); - - // there should be one job that is a k8s peon job that exists - List jobs = peonClient.getPeonJobs(false); - assertEquals(1, jobs.size()); - - K8sTaskId taskId = new K8sTaskId(null, task.getId()); - InputStream peonLogs = peonClient.getPeonLogs(taskId).get(); - List expectedLogs = IntStream.range(1, 1001).boxed().collect(Collectors.toList()); - List actualLogs = new ArrayList<>(); - Thread thread = new Thread(() -> { - try { - actualLogs.addAll(IOUtils.readLines(peonLogs, "UTF-8") - .stream() - .map(Integer::parseInt) - .collect(Collectors.toList())); - } - catch (UncheckedIOException e) { - throw new RuntimeException(e); - } - }); - thread.start(); - - // assert that the env variable is corret - Task taskFromEnvVar = adapter.toTask(job); - assertEquals(task, taskFromEnvVar); - - // now copy the task.json file from the pod and make sure its the same as our task.json we expected - Path downloadPath = Paths.get(tempDir.toAbsolutePath().toString(), "task.json"); - Pod mainJobPod = peonClient.getPeonPodWithRetries(taskId.getK8sJobName()); - k8sClient.executeRequest(client -> { - client.pods() - .inNamespace("default") - .withName(mainJobPod.getMetadata().getName()) - .file(Paths.get(taskBasePath, "task.json").toString()) - .copy(downloadPath); - return null; - }); - - String taskJsonFromPod = FileUtils.readFileToString(new File(downloadPath.toString()), StandardCharsets.UTF_8); - Task taskFromPod = jsonMapper.readValue(taskJsonFromPod, Task.class); - assertEquals(task, taskFromPod); - - - JobResponse jobStatusResult = peonClient.waitForPeonJobCompletion(taskId, 2, TimeUnit.MINUTES); - thread.join(); - assertEquals(PeonPhase.SUCCEEDED, jobStatusResult.getPhase()); - // as long as there were no exceptions we are good! - assertEquals(expectedLogs, actualLogs); - // cleanup my job - assertTrue(peonClient.deletePeonJob(taskId)); - - // we cleaned up the job, none should exist - List existingJobs = peonClient.getPeonJobs(false); - assertEquals(0, existingJobs.size()); - } } From 8911e1f0b45a6758ed88ebc5858a11de93f16573 Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 5 Oct 2025 18:44:59 -0500 Subject: [PATCH 03/58] Implementation ready for deeper UT and ET writing --- .../overlord/KubernetesOverlordModule.java | 2 +- .../k8s/overlord/KubernetesPeonLifecycle.java | 6 +- .../KubernetesPeonLifecycleFactory.java | 6 +- .../k8s/overlord/KubernetesTaskRunner.java | 8 +- .../overlord/KubernetesTaskRunnerConfig.java | 22 ++ .../overlord/KubernetesTaskRunnerFactory.java | 56 ++- ...java => AbstractKubernetesPeonClient.java} | 294 +++----------- .../common/CachingKubernetesPeonClient.java | 220 +++++++++++ .../common/DirectKubernetesPeonClient.java | 257 +++++++++++++ .../common/DruidKubernetesClient.java | 20 +- .../overlord/KubernetesPeonLifecycleTest.java | 5 +- .../KubernetesTaskRunnerFactoryTest.java | 2 +- .../overlord/KubernetesTaskRunnerTest.java | 10 +- .../CachingKubernetesPeonClientTest.java | 234 +++++++++++ ...va => DirectKubernetesPeonClientTest.java} | 363 ++++++++++++------ .../overlord/common/TestKubernetesClient.java | 17 +- .../DruidPeonClientIntegrationTest.java | 36 +- .../taskadapter/K8sTaskAdapterTest.java | 22 +- .../MultiContainerTaskAdapterTest.java | 6 +- .../SingleContainerTaskAdapterTest.java | 2 +- 20 files changed, 1142 insertions(+), 446 deletions(-) rename extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/{KubernetesPeonClient.java => AbstractKubernetesPeonClient.java} (51%) create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java create mode 100644 extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java rename extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/{KubernetesPeonClientTest.java => DirectKubernetesPeonClientTest.java} (65%) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index 4e8a061e4cd6..5f32005f0c0f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -135,7 +135,7 @@ public DruidKubernetesClient makeKubernetesClient( config.setNamespace(kubernetesTaskRunnerConfig.getNamespace()); - client = new DruidKubernetesClient(httpClientConfig, config); + client = new DruidKubernetesClient(httpClientConfig, config, kubernetesTaskRunnerConfig.isEnablePeonClientCache()); lifecycle.addHandler( new Lifecycle.Handler() diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index dae8890a089b..ca84c84de7f9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -39,7 +39,7 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; -import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @@ -93,7 +93,7 @@ protected enum State private final K8sTaskId taskId; private final TaskLogs taskLogs; private final Task task; - private final KubernetesPeonClient kubernetesClient; + private final AbstractKubernetesPeonClient kubernetesClient; private final ObjectMapper mapper; private final TaskStateListener stateListener; private final SettableFuture taskStartedSuccessfullyFuture; @@ -107,7 +107,7 @@ protected enum State protected KubernetesPeonLifecycle( Task task, K8sTaskId taskId, - KubernetesPeonClient kubernetesClient, + AbstractKubernetesPeonClient kubernetesClient, TaskLogs taskLogs, ObjectMapper mapper, TaskStateListener stateListener, diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java index 8bd7db2ebf3e..fd23dddc8b48 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java @@ -22,18 +22,18 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.k8s.overlord.common.K8sTaskId; -import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory { - private final KubernetesPeonClient client; + private final AbstractKubernetesPeonClient client; private final TaskLogs taskLogs; private final ObjectMapper mapper; private final long logSaveTimeoutMs; public KubernetesPeonLifecycleFactory( - KubernetesPeonClient client, + AbstractKubernetesPeonClient client, TaskLogs taskLogs, ObjectMapper mapper, long logSaveTimeoutMs diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index dc8ccffd6380..3653bb6ffeab 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -54,7 +54,7 @@ import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.k8s.overlord.common.K8sTaskId; -import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.TaskLogStreamer; @@ -108,7 +108,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner protected final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); protected final TaskAdapter adapter; - private final KubernetesPeonClient client; + private final AbstractKubernetesPeonClient client; private final KubernetesTaskRunnerConfig config; private final ListeningExecutorService exec; private final HttpClient httpClient; @@ -120,7 +120,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner public KubernetesTaskRunner( TaskAdapter adapter, KubernetesTaskRunnerConfig config, - KubernetesPeonClient client, + AbstractKubernetesPeonClient client, HttpClient httpClient, PeonLifecycleFactory peonLifecycleFactory, ServiceEmitter emitter @@ -348,7 +348,7 @@ public void start() // Load tasks from previously running jobs and wait for their statuses to start running. final List> taskStatusActiveList = new ArrayList<>(); // Get all existing peon jobs. - final List peonJobs = client.getPeonJobs(true); + final List peonJobs = client.getPeonJobs(); log.info("Locating [%,d] active tasks.", peonJobs.size()); for (Job job : peonJobs) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index d810c9ee23a2..26f5e51aa6eb 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -80,6 +80,10 @@ public class KubernetesTaskRunnerConfig // disable using http / https proxy environment variables private boolean disableClientProxy; + @JsonProperty + // enable using kubernetes informer cache for peon client operations + private boolean enableKubernetesClientCaching = false; + @JsonProperty @NotNull private Period maxTaskDuration = new Period("PT4H"); @@ -153,6 +157,7 @@ private KubernetesTaskRunnerConfig( String kubexitImage, Long graceTerminationPeriodSeconds, boolean disableClientProxy, + boolean enableKubernetesClientCaching, Period maxTaskDuration, Period taskCleanupDelay, Period taskCleanupInterval, @@ -194,6 +199,10 @@ private KubernetesTaskRunnerConfig( this.graceTerminationPeriodSeconds ); this.disableClientProxy = disableClientProxy; + this.enableKubernetesClientCaching = ObjectUtils.defaultIfNull( + enableKubernetesClientCaching, + this.enableKubernetesClientCaching + ); this.maxTaskDuration = ObjectUtils.defaultIfNull( maxTaskDuration, this.maxTaskDuration @@ -290,6 +299,11 @@ public boolean isDisableClientProxy() return disableClientProxy; } + public boolean isEnablePeonClientCache() + { + return enableKubernetesClientCaching; + } + public Period getTaskTimeout() { return maxTaskDuration; @@ -367,6 +381,7 @@ public static class Builder private String kubexitImage; private Long graceTerminationPeriodSeconds; private boolean disableClientProxy; + private boolean enableKubernetesClientCaching; private Period maxTaskDuration; private Period taskCleanupDelay; private Period taskCleanupInterval; @@ -438,6 +453,12 @@ public Builder withDisableClientProxy(boolean disableClientProxy) return this; } + public Builder withEnablePeonClientCache(boolean enableKubernetesClientCaching) + { + this.enableKubernetesClientCaching = enableKubernetesClientCaching; + return this; + } + public Builder withTaskTimeout(Period taskTimeout) { this.maxTaskDuration = taskTimeout; @@ -523,6 +544,7 @@ public KubernetesTaskRunnerConfig build() this.kubexitImage, this.graceTerminationPeriodSeconds, this.disableClientProxy, + this.enableKubernetesClientCaching, this.maxTaskDuration, this.taskCleanupDelay, this.taskCleanupInterval, diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index dd8111ed49ed..16e19499ee7f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -27,7 +27,9 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; -import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.CachingKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.DirectKubernetesPeonClient; import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.TaskLogs; @@ -70,22 +72,44 @@ public KubernetesTaskRunnerFactory( @Override public KubernetesTaskRunner build() { - KubernetesPeonClient peonClient; - if (adapterTypeAllowingTasksInDifferentNamespaces.contains(taskAdapter.getAdapterType())) { - peonClient = new KubernetesPeonClient( - druidKubernetesClient, - kubernetesTaskRunnerConfig.getNamespace(), - kubernetesTaskRunnerConfig.getOverlordNamespace(), - kubernetesTaskRunnerConfig.isDebugJobs(), - emitter - ); + AbstractKubernetesPeonClient peonClient; + boolean enableCache = kubernetesTaskRunnerConfig.isEnablePeonClientCache(); + boolean useOverlordNamespace = adapterTypeAllowingTasksInDifferentNamespaces.contains(taskAdapter.getAdapterType()); + + if (enableCache) { + if (useOverlordNamespace) { + peonClient = new CachingKubernetesPeonClient( + druidKubernetesClient, + kubernetesTaskRunnerConfig.getNamespace(), + kubernetesTaskRunnerConfig.getOverlordNamespace(), + kubernetesTaskRunnerConfig.isDebugJobs(), + emitter + ); + } else { + peonClient = new CachingKubernetesPeonClient( + druidKubernetesClient, + kubernetesTaskRunnerConfig.getNamespace(), + kubernetesTaskRunnerConfig.isDebugJobs(), + emitter + ); + } } else { - peonClient = new KubernetesPeonClient( - druidKubernetesClient, - kubernetesTaskRunnerConfig.getNamespace(), - kubernetesTaskRunnerConfig.isDebugJobs(), - emitter - ); + if (useOverlordNamespace) { + peonClient = new DirectKubernetesPeonClient( + druidKubernetesClient, + kubernetesTaskRunnerConfig.getNamespace(), + kubernetesTaskRunnerConfig.getOverlordNamespace(), + kubernetesTaskRunnerConfig.isDebugJobs(), + emitter + ); + } else { + peonClient = new DirectKubernetesPeonClient( + druidKubernetesClient, + kubernetesTaskRunnerConfig.getNamespace(), + kubernetesTaskRunnerConfig.isDebugJobs(), + emitter + ); + } } runner = new KubernetesTaskRunner( diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java similarity index 51% rename from extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java rename to extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index 1bf575dcb226..23b3804cf326 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -21,16 +21,11 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; -import io.fabric8.kubernetes.api.model.Event; -import io.fabric8.kubernetes.api.model.ObjectReference; -import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder; import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.PodStatus; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientException; import io.fabric8.kubernetes.client.dsl.LogWatch; -import io.fabric8.kubernetes.client.informers.SharedIndexInformer; import io.vertx.core.http.HttpClosedException; import org.apache.druid.error.DruidException; import org.apache.druid.indexing.common.task.IndexTaskUtils; @@ -48,17 +43,20 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -public class KubernetesPeonClient +/** + * Abstract base class for Kubernetes peon clients, providing common functionality for mutable and log related operations + */ +public abstract class AbstractKubernetesPeonClient { - private static final EmittingLogger log = new EmittingLogger(KubernetesPeonClient.class); + protected static final EmittingLogger log = new EmittingLogger(AbstractKubernetesPeonClient.class); - private final KubernetesClientApi clientApi; - private final String namespace; - private final String overlordNamespace; + protected final KubernetesClientApi clientApi; + protected final String namespace; + protected final String overlordNamespace; private final boolean debugJobs; private final ServiceEmitter emitter; - public KubernetesPeonClient( + AbstractKubernetesPeonClient( KubernetesClientApi clientApi, String namespace, String overlordNamespace, @@ -73,17 +71,42 @@ public KubernetesPeonClient( this.emitter = emitter; } - public KubernetesPeonClient( - KubernetesClientApi clientApi, - String namespace, - boolean debugJobs, - ServiceEmitter emitter - ) - { - this(clientApi, namespace, "", debugJobs, emitter); - } + /** + * Wait for the K8s job associated with the given taskId to complete, or until the given timeout is reached + * + * @return JobResponse indicating the result of the job + */ + public abstract JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit); + + /** + * Get the list of all peon jobs in the namespace associated with this client + */ + public abstract List getPeonJobs(); - public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) throws IllegalStateException + /** + * Get the Pod associated with the given job name, if it exists + * + * @return an Optional containing the Pod if it exists, or absent if not found + */ + public abstract Optional getPeonPod(String jobName); + + /** + * Waits for a pod associated with a job to be created and reach ready state using the pod cache. + * This method polls the informer cache until the pod appears and has a pod IP assigned. + * + * @param jobName the name of the job whose pod we're waiting for + * @param howLong the maximum time to wait + * @param timeUnit the time unit for the timeout + * @return the pod in ready state, or null if the pod disappeared after being seen + * @throws DruidException if the pod never appears within the timeout period + */ + protected abstract Pod waitUntilPeonPodCreatedAndReady( String jobName, long howLong, TimeUnit timeUnit); + + /** + * Launches the given Job. Waits for the associated pod and job to be created and start running. + * + */ + public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) { long start = System.currentTimeMillis(); @@ -94,7 +117,7 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn log.info("Submitted job[%s] for task[%s]. Waiting for POD to launch and be ready.", jobName, task.getId()); // Wait for the Pod to be created and then reach ready state - Pod result = clientApi.executePodCacheRequest(informer -> waitForPeonPodCreateAndReady(informer, jobName, howLong, timeUnit)); + Pod result = waitUntilPeonPodCreatedAndReady(jobName, howLong, timeUnit); // Evaluate result of job launch if (result == null) { @@ -106,89 +129,6 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn return result; } - public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) - { - long timeoutMs = unit.toMillis(howLong); - long startTime = System.currentTimeMillis(); - long pollInterval = 5000; - long jobAppearanceGracePeriodMs = 90000; // 90 seconds grace for job to appear in cache - - boolean jobSeenInCache = false; - - do { - if (!clientApi.getJobInformer().hasSynced()) { - // Checking before the informer has synced will likely result in a false negative. - try { - Thread.sleep(pollInterval); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - continue; - } - - Job job; - if (!overlordNamespace.isEmpty()) { - log.info("byOverlordNamespace lookup"); - job = clientApi.executeJobCacheRequest((informer) -> - informer.getIndexer() - .byIndex("byOverlordNamespace", overlordNamespace).stream() - .filter(j -> taskId.getK8sJobName() - .equals(j.getMetadata().getName())) - .findFirst() - .orElse(null)); - } else { - log.info("byJobName lookup"); - job = clientApi.executeJobCacheRequest(informer -> - informer.getIndexer().byIndex("byJobName", taskId.getK8sJobName()).stream().findFirst().orElse(null)); - } - - if (job == null) { - long elapsed = System.currentTimeMillis() - startTime; - - // Give grace period for job to appear in cache after creation - if (!jobSeenInCache && elapsed < jobAppearanceGracePeriodMs) { - log.info("Job [%s] not yet in cache, waiting... (elapsed: %d ms)", taskId, elapsed); - try { - Thread.sleep(pollInterval); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - continue; - } - - // After grace period or if we've seen it before, job is truly missing - log.warn("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); - return new JobResponse(null, PeonPhase.FAILED); - } - - // Job found! Mark that we've seen it - jobSeenInCache = true; - - // Check if job is complete - if (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) { - if (job.getStatus().getSucceeded() != null) { - log.info("Task %s completed successfully", taskId); - return new JobResponse(job, PeonPhase.SUCCEEDED); - } - log.warn("Task %s failed with status %s", taskId, job.getStatus()); - return new JobResponse(job, PeonPhase.FAILED); - } - - // Job still running, wait and check again - try { - Thread.sleep(pollInterval); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } while (System.currentTimeMillis() - startTime < timeoutMs); - - log.warn("Timed out waiting for job [%s] to complete", taskId); - return new JobResponse(null, PeonPhase.FAILED); - } - public boolean deletePeonJob(K8sTaskId taskId) { if (!debugJobs) { @@ -254,60 +194,11 @@ public Optional getPeonLogs(K8sTaskId taskId) } } - public List getPeonJobs(boolean useCache) - { - if (useCache) { - if (!clientApi.getJobInformer().hasSynced()) { - log.warn("K8s job informer cache not synced, getting jobs directly from k8s"); - useCache = false; - } - } - - return this.overlordNamespace.isEmpty() - ? getPeonJobsWithoutOverlordNamespaceKeyLabels(useCache) - : getPeonJobsWithOverlordNamespaceKeyLabels(useCache); - } - - private List getPeonJobsWithoutOverlordNamespaceKeyLabels(boolean useCache) - { - if (useCache) { - return clientApi.executeJobCacheRequest(informer -> informer.getIndexer().list()); - } else { - return clientApi.executeRequest(client -> client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .list() - .getItems()); - } - } - - private List getPeonJobsWithOverlordNamespaceKeyLabels(boolean useCache) - { - if (useCache) { - return clientApi.executeJobCacheRequest(informer -> informer.getIndexer() - .byIndex("byOverlordNamespace", overlordNamespace)); - } else { - return clientApi.executeRequest(client -> client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .withLabel( - DruidK8sConstants.OVERLORD_NAMESPACE_KEY, - overlordNamespace - ) - .list() - .getItems()); - } - } - public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit) { AtomicInteger numDeleted = new AtomicInteger(); return clientApi.executeRequest(client -> { - List jobs = getJobsToCleanup(getPeonJobs(true), howFarBack, timeUnit); + List jobs = getJobsToCleanup(getPeonJobs(), howFarBack, timeUnit); jobs.forEach(x -> { if (!client.batch() .v1() @@ -340,17 +231,6 @@ private List getJobsToCleanup(List candidates, long howFarBack, TimeUn return toDelete; } - public Optional getPeonPod(String jobName) - { - return clientApi.executePodCacheRequest(informer -> getPeonPod(informer, jobName)); - } - - private Optional getPeonPod(SharedIndexInformer informer, String jobName) - { - List pods = informer.getIndexer().byIndex("byJobName", jobName); - return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); - } - public void createK8sJobWithRetries(Job job) { clientApi.executeRequest(client -> { @@ -417,7 +297,7 @@ void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, i * We have experienced connections in the pool being closed by the server-side but remaining in the pool. These issues * should be safe to retry in many cases. */ - private boolean isRetryableTransientConnectionPoolException(Throwable e) + protected boolean isRetryableTransientConnectionPoolException(Throwable e) { if (e instanceof KubernetesClientException) { return e.getMessage() != null && e.getMessage().contains("Connection was closed"); @@ -427,86 +307,6 @@ private boolean isRetryableTransientConnectionPoolException(Throwable e) return false; } - /** - * Waits for a pod associated with a job to be created and reach ready state using the pod cache. - * This method polls the informer cache until the pod appears and has a pod IP assigned. - * - * @param informer the pod informer to query - * @param jobName the name of the job whose pod we're waiting for - * @param howLong the maximum time to wait - * @param timeUnit the time unit for the timeout - * @return the pod in ready state, or null if the pod disappeared after being seen - * @throws DruidException if the pod never appears within the timeout period - */ - private Pod waitForPeonPodCreateAndReady(SharedIndexInformer informer, String jobName, long howLong, TimeUnit timeUnit) - { - log.info("Waiting for pod for job[%s] to be created and ready using pod cache", jobName); - long timeoutMs = timeUnit.toMillis(howLong); - long startTime = System.currentTimeMillis(); - long pollInterval = 2000; // Poll every 2 seconds - - boolean podSeenInCache = false; - String podName = null; - - do { - if (!informer.hasSynced()) { - // Wait for informer to sync - try { - Thread.sleep(pollInterval); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - continue; - } - - List pods = informer.getIndexer().byIndex("byJobName", jobName); - - if (pods.isEmpty()) { - // If we've seen the pod before and now it's gone, it was deleted - if (podSeenInCache) { - log.info("Pod for job[%s] disappeared after being seen in cache", jobName); - return null; - } - // Otherwise keep waiting for it to appear - } else { - Pod currentPod = pods.get(0); - podSeenInCache = true; - podName = currentPod.getMetadata().getName(); - - // Check if pod is ready (has IP) - if (currentPod.getStatus() != null && currentPod.getStatus().getPodIP() != null) { - log.info("Pod[%s] for job[%s] is ready with IP: %s", podName, jobName, currentPod.getStatus().getPodIP()); - return currentPod; - } - - log.info("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); - } - - // Wait before polling again - long remainingTime = timeoutMs - (System.currentTimeMillis() - startTime); - if (remainingTime <= 0) { - break; - } - - try { - Thread.sleep(Math.min(pollInterval, remainingTime)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } while (System.currentTimeMillis() - startTime < timeoutMs); - - // Timeout - if (podSeenInCache) { - log.warn("Timeout waiting for pod[%s] for job[%s] to become ready", podName, jobName); - return null; - } else { - throw DruidException.defensive("Timeout waiting for pod for job[%s] to be created", jobName); - } - } - - private void emitK8sPodMetrics(Task task, String metric, long durationMs) { ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java new file mode 100644 index 000000000000..41e683e0a9ac --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -0,0 +1,220 @@ +package org.apache.druid.k8s.overlord.common; + +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * A KubernetesPeonClient implementation that uses cached informers to read Job and Pod state. + *

    + * This reduces load on the Kubernetes API server by centralizing watches allowing tasks to query cached K8s resource + * information. + *

    + */ +public class CachingKubernetesPeonClient extends AbstractKubernetesPeonClient +{ + protected static final EmittingLogger log = new EmittingLogger(CachingKubernetesPeonClient.class); + + public CachingKubernetesPeonClient( + KubernetesClientApi clientApi, + String namespace, + String overlordNamespace, + boolean debugJobs, + ServiceEmitter emitter + ) + { + super(clientApi, namespace, overlordNamespace, debugJobs, emitter); + } + + public CachingKubernetesPeonClient( + KubernetesClientApi clientApi, + String namespace, + boolean debugJobs, + ServiceEmitter emitter + ) + { + super(clientApi, namespace, "", debugJobs, emitter); + } + + @Override + public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) + { + long timeoutMs = unit.toMillis(howLong); + long startTime = System.currentTimeMillis(); + long pollInterval = 5000; + long jobAppearanceGracePeriodMs = 90000; // 90 seconds grace for job to appear in cache + + boolean jobSeenInCache = false; + + do { + if (!clientApi.getJobInformer().hasSynced()) { + // Checking before the informer has synced will likely result in a false negative. + try { + Thread.sleep(pollInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + continue; + } + + Job job; + if (!overlordNamespace.isEmpty()) { + job = clientApi.executeJobCacheRequest((informer) -> + informer.getIndexer() + .byIndex("byOverlordNamespace", overlordNamespace).stream() + .filter(j -> taskId.getK8sJobName() + .equals(j.getMetadata().getName())) + .findFirst() + .orElse(null)); + } else { + job = clientApi.executeJobCacheRequest(informer -> + informer.getIndexer().byIndex("byJobName", taskId.getK8sJobName()).stream().findFirst().orElse(null)); + } + + if (job == null) { + long elapsed = System.currentTimeMillis() - startTime; + + // Give grace period for job to appear in cache after creation + if (!jobSeenInCache && elapsed < jobAppearanceGracePeriodMs) { + log.debug("Job [%s] not yet in cache, waiting... (elapsed: %d ms)", taskId, elapsed); + try { + Thread.sleep(pollInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + continue; + } + + // After grace period or if we've seen it before, job is truly missing + log.warn("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); + return new JobResponse(null, PeonPhase.FAILED); + } + + // Job found! Mark that we've seen it + jobSeenInCache = true; + + // Check if job is complete + if (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) { + if (job.getStatus().getSucceeded() > 0) { + log.info("K8s job [%s] completed successfully", taskId); + return new JobResponse(job, PeonPhase.SUCCEEDED); + } + log.warn("K8s job [%s] failed with status %s", taskId, job.getStatus()); + return new JobResponse(job, PeonPhase.FAILED); + } + + // Job still running, wait and check again + try { + Thread.sleep(pollInterval); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } while (System.currentTimeMillis() - startTime < timeoutMs); + + log.warn("Timed out waiting for K8s job [%s] to complete", taskId); + return new JobResponse(null, PeonPhase.FAILED); + } + + @Override + public List getPeonJobs() + { + if (overlordNamespace.isEmpty()) { + return clientApi.executeJobCacheRequest(informer -> informer.getIndexer().list()); + } else { + return clientApi.executeJobCacheRequest(informer -> + informer.getIndexer() + .byIndex("byOverlordNamespace", overlordNamespace)); + } + } + + @Override + public Optional getPeonPod(String jobName) + { + return clientApi.executePodCacheRequest(informer -> { + List pods = informer.getIndexer().byIndex("byJobName", jobName); + return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); + }); + } + + @Override + protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) + { + return clientApi.executePodCacheRequest(informer -> { + long timeoutMs = timeUnit.toMillis(howLong); + long startTime = System.currentTimeMillis(); + long pollInterval = 2000; // Poll every 2 seconds + + boolean podSeenInCache = false; + String podName = null; + + do { + if (!informer.hasSynced()) { + // Wait for informer to sync + try { + Thread.sleep(pollInterval); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + continue; + } + + List pods = informer.getIndexer().byIndex("byJobName", jobName); + + if (pods.isEmpty()) { + // If we've seen the pod before, and now it's gone, it was deleted + if (podSeenInCache) { + log.warn("Pod for job[%s] disappeared after being seen in cache", jobName); + return null; + } + // Otherwise keep waiting for it to appear + } else { + Pod currentPod = pods.get(0); + podSeenInCache = true; + podName = currentPod.getMetadata().getName(); + + // Check if pod is ready (has IP) + if (currentPod.getStatus() != null && currentPod.getStatus().getPodIP() != null) { + log.info("Pod[%s] for job[%s] is ready with IP: %s", podName, jobName, currentPod.getStatus().getPodIP()); + return currentPod; + } + + log.debug("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); + } + + // Wait before polling again + long remainingTime = timeoutMs - (System.currentTimeMillis() - startTime); + if (remainingTime <= 0) { + break; + } + + try { + Thread.sleep(Math.min(pollInterval, remainingTime)); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } while (System.currentTimeMillis() - startTime < timeoutMs); + + // Timeout + if (podSeenInCache) { + log.warn("Timeout waiting for pod[%s] for job[%s] to become ready", podName, jobName); + return null; + } else { + throw DruidException.defensive("Timeout waiting for pod for job[%s] to be created", jobName); + } + }); + } + +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java new file mode 100644 index 000000000000..81d9b449176b --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java @@ -0,0 +1,257 @@ +package org.apache.druid.k8s.overlord.common; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.Event; +import io.fabric8.kubernetes.api.model.ObjectReference; +import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.KubernetesClientException; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * A KubernetesPeonClient implementation that directly queries the Kubernetes API server for Job and Pod state. + *

    + * This implementation does not use caching and may put more load on the Kubernetes API server when many tasks + * are running. + *

    + */ +public class DirectKubernetesPeonClient extends AbstractKubernetesPeonClient +{ + protected static final EmittingLogger log = new EmittingLogger(CachingKubernetesPeonClient.class); + + public DirectKubernetesPeonClient( + KubernetesClientApi clientApi, + String namespace, + String overlordNamespace, + boolean debugJobs, + ServiceEmitter emitter + ) + { + super(clientApi, namespace, overlordNamespace, debugJobs, emitter); + } + + public DirectKubernetesPeonClient( + KubernetesClientApi clientApi, + String namespace, + boolean debugJobs, + ServiceEmitter emitter + ) + { + super(clientApi, namespace, "", debugJobs, emitter); + } + + @Override + public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) + { + return clientApi.executeRequest(client -> { + Job job = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .waitUntilCondition( + x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null + && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), + howLong, + unit + ); + if (job == null) { + log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); + return new JobResponse(null, PeonPhase.FAILED); + } + if (job.getStatus().getSucceeded() != null) { + return new JobResponse(job, PeonPhase.SUCCEEDED); + } + log.warn("Task %s failed with status %s", taskId, job.getStatus()); + return new JobResponse(job, PeonPhase.FAILED); + }); + } + + @Override + public List getPeonJobs() + { + if (overlordNamespace.isEmpty()) { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .list() + .getItems()); + } else { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .withLabel( + DruidK8sConstants.OVERLORD_NAMESPACE_KEY, + overlordNamespace + ) + .list() + .getItems()); + } + } + + @Override + public Optional getPeonPod(String jobName) + { + return clientApi.executeRequest(client -> getPeonPod(client, jobName)); + } + + @Nullable + @Override + protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) + { + Pod pod = clientApi.executeRequest(client -> getPeonPodWithRetries(client, jobName, 5, RetryUtils.DEFAULT_MAX_TRIES)); + if (pod == null) { + return null; + } + return clientApi.executeRequest(client -> waitForPodResultWithRetries(client, pod, howLong, timeUnit, 5, RetryUtils.DEFAULT_MAX_TRIES)); + + } + + /** + * Waits for a Kubernetes pod to reach a ready state with retry logic for transient connection pool exceptions. + *

    + * This method waits for the specified pod to have a valid status with a pod IP assigned, indicating + * it has been scheduled and is in a ready state. The method includes retry logic to handle transient + * connection pool exceptions that may occur during the wait operation. + *

    + * The method will wait up to the specified timeout for the pod to become ready, and retry the entire wait operation + * if transient connection issues are encountered. + * + * @param client the Kubernetes client to use for pod operations + * @param pod the pod to wait for + * @param howLong the maximum time to wait for the pod to become ready + * @param timeUnit the time unit for the wait timeout + * @param quietTries number of initial retry attempts without logging warnings + * @param maxTries maximum total number of retry attempts + * @return the pod in its ready state, or null if the pod disappeared or wait operation failed + * @throws DruidException if waiting fails after all retry attempts or encounters non-retryable errors + */ + @VisibleForTesting + Pod waitForPodResultWithRetries(KubernetesClient client, Pod pod, long howLong, TimeUnit timeUnit, int quietTries, int maxTries) + { + try { + return RetryUtils.retry( + () -> client.pods() + .inNamespace(namespace) + .withName(pod.getMetadata().getName()) + .waitUntilCondition( + p -> { + if (p == null) { + return true; + } + return p.getStatus() != null && p.getStatus().getPodIP() != null; + }, howLong, timeUnit), + this::isRetryableTransientConnectionPoolException, quietTries, maxTries); + } + catch (Exception e) { + throw DruidException.defensive(e, "Error when waiting for pod[%s] to start", pod.getMetadata().getName()); + } + } + + @VisibleForTesting + Pod getPeonPodWithRetries(KubernetesClient client, String jobName, int quietTries, int maxTries) + { + try { + return RetryUtils.retry( + () -> { + Optional maybePod = getPeonPod(client, jobName); + if (maybePod.isPresent()) { + return maybePod.get(); + } + + // If the pod is missing, we can take a look at job events to discover potential problems with pod creation. + List events = getPeonEvents(client, jobName); + + if (events.isEmpty()) { + throw new KubernetesResourceNotFoundException("K8s pod with label[job-name=%s] not found", jobName); + } else { + Event latestEvent = events.get(events.size() - 1); + throw new KubernetesResourceNotFoundException( + "Job[%s] failed to create pods. Message[%s]", jobName, latestEvent.getMessage()); + } + }, + this::shouldRetryWaitForStartingPeonPod, quietTries, maxTries + ); + } + catch (KubernetesResourceNotFoundException e) { + throw e; + } + catch (Exception e) { + throw DruidException.defensive(e, "Error when looking for K8s pod with label[job-name=%s]", jobName); + } + } + + private Optional getPeonPod(KubernetesClient client, String jobName) + { + List pods = client.pods() + .inNamespace(namespace) + .withLabel("job-name", jobName) + .list() + .getItems(); + return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); + } + + /** + * Determines if this exception, specifically when containing Kubernetes job event messages, permits a retry attempt. + *

    + * The method checks the exception message against a predefined list of Kubernetes event messages. + * These substrings, found in {@link DruidK8sConstants#BLACKLISTED_PEON_POD_ERROR_MESSAGES}, + * represent Kubernetes event that indicate a retry for starting the Peon Pod would likely be futile. + */ + private boolean shouldRetryWaitForStartingPeonPod(Throwable e) + { + if (isRetryableTransientConnectionPoolException(e)) { + return true; + } + + if (!(e instanceof KubernetesResourceNotFoundException)) { + return false; + } + + String errorMessage = e.getMessage(); + for (String blacklistedMessage : DruidK8sConstants.BLACKLISTED_PEON_POD_ERROR_MESSAGES) { + if (errorMessage.contains(blacklistedMessage)) { + return false; + } + } + + return true; + } + + private List getPeonEvents(KubernetesClient client, String jobName) + { + ObjectReference objectReference = new ObjectReferenceBuilder() + .withApiVersion("batch/v1") + .withKind("Job") + .withName(jobName) + .withNamespace(this.namespace) + .build(); + + try { + return client.v1() + .events() + .inNamespace(this.namespace) + .withInvolvedObject(objectReference) + .list() + .getItems(); + } + catch (KubernetesClientException e) { + log.warn("Failed to get events for job[%s]; %s", jobName, e.getMessage()); + return List.of(); + } + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 5e0a2a100593..cde7084a4573 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -26,6 +26,7 @@ import io.fabric8.kubernetes.client.KubernetesClientBuilder; import io.fabric8.kubernetes.client.informers.ResourceEventHandler; import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.emitter.EmittingLogger; import java.util.Collections; @@ -44,14 +45,21 @@ public class DruidKubernetesClient implements KubernetesClientApi private final SharedIndexInformer podInformer; private final SharedIndexInformer jobInformer; - public DruidKubernetesClient(DruidKubernetesHttpClientConfig httpClientConfig, Config kubernetesClientConfig) + public DruidKubernetesClient(DruidKubernetesHttpClientConfig httpClientConfig, Config kubernetesClientConfig, + boolean enableCache + ) { this.kubernetesClient = new KubernetesClientBuilder() .withHttpClientFactory(new DruidKubernetesHttpClientFactory(httpClientConfig)) .withConfig(kubernetesClientConfig) .build(); - this.podInformer = setupPodInformer(kubernetesClient.getNamespace()); - this.jobInformer = setupJobInformer(kubernetesClient.getNamespace()); + if (enableCache) { + this.podInformer = setupPodInformer(kubernetesClient.getNamespace()); + this.jobInformer = setupJobInformer(kubernetesClient.getNamespace()); + } else { + this.podInformer = null; + this.jobInformer = null; + } } @Override @@ -63,12 +71,18 @@ public T executeRequest(KubernetesExecutor executor) throws KubernetesRes @Override public T executePodCacheRequest(KubernetesInformerExecutor executor) { + if (podInformer == null) { + throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); + } return executor.executeRequest(podInformer); } @Override public T executeJobCacheRequest(KubernetesInformerExecutor executor) { + if (jobInformer == null) { + throw DruidException.defensive("Job informer is not initialized, caching is disabled"); + } return executor.executeRequest(jobInformer); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 4484a1076189..441850ae9345 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -34,7 +34,7 @@ import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; -import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; @@ -63,7 +63,8 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport private static final Period LOG_SAVE_TIMEOUT = new Period("PT300S"); private static final Period SHORT_LOG_SAVE_TIMEOUT = new Period("PT1S"); - @Mock KubernetesPeonClient kubernetesClient; + @Mock + AbstractKubernetesPeonClient kubernetesClient; @Mock TaskLogs taskLogs; @Mock LogWatch logWatch; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 49720642216f..c81cdb0a45d9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -57,7 +57,7 @@ public void setup() .build(); taskLogs = new NoopTaskLogs(); druidKubernetesClient = - new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build()); + new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build(), false); taskAdapter = new TestTaskAdapter(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 697d0566b86f..ea8889d3245b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -38,7 +38,7 @@ import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.k8s.overlord.common.K8sTestUtils; -import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -74,7 +74,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport @Mock private HttpClient httpClient; @Mock private TaskAdapter taskAdapter; - @Mock private KubernetesPeonClient peonClient; + @Mock private AbstractKubernetesPeonClient peonClient; @Mock private KubernetesPeonLifecycle kubernetesPeonLifecycle; @Mock private ServiceEmitter emitter; @Mock private ListenableFuture statusFuture; @@ -136,7 +136,7 @@ protected KubernetesWorkItem joinAsync(Task task) .endMetadata() .build(); - EasyMock.expect(peonClient.getPeonJobs(false)).andReturn(ImmutableList.of(job)); + EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job)); EasyMock.expect(taskAdapter.toTask(job)).andReturn(task); EasyMock.expect(kubernetesPeonLifecycle.getTaskStartedSuccessfullyFuture()).andReturn( settableFuture @@ -192,7 +192,7 @@ protected KubernetesWorkItem joinAsync(Task task) .endMetadata() .build(); - EasyMock.expect(peonClient.getPeonJobs(false)).andReturn(ImmutableList.of(job, job2)); + EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job, job2)); EasyMock.expect(taskAdapter.toTask(job)).andReturn(task); EasyMock.expect(taskAdapter.toTask(job2)).andThrow(new IOException("deserialization exception")); @@ -235,7 +235,7 @@ protected KubernetesWorkItem joinAsync(Task task) .endMetadata() .build(); - EasyMock.expect(peonClient.getPeonJobs(false)).andReturn(ImmutableList.of(job)); + EasyMock.expect(peonClient.getPeonJobs()).andReturn(ImmutableList.of(job)); EasyMock.expect(taskAdapter.toTask(job)).andThrow(new IOException()); replayAll(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java new file mode 100644 index 000000000000..589c886c7fc0 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import io.fabric8.kubernetes.client.informers.cache.Indexer; +import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.List; + +@EnableKubernetesMockClient(crud = true) +public class CachingKubernetesPeonClientTest +{ + private static final String ID = "id"; + private static final String TASK_NAME_PREFIX = ""; + private static final String JOB_NAME = ID; + private static final String KUBERNETES_JOB_NAME = KubernetesOverlordUtils.convertTaskIdToJobName(JOB_NAME); + private static final String POD_NAME = "name"; + private static final String NAMESPACE = "namespace"; + + private KubernetesClient client; + private KubernetesMockServer server; + private KubernetesClientApi clientApi; + private CachingKubernetesPeonClient instance; + private StubServiceEmitter serviceEmitter; + + @BeforeEach + public void setup() + { + serviceEmitter = new StubServiceEmitter("service", "host"); + } + + @Test + void test_getPeonPod_withPodInCache_returnsPresentOptional() + { + // Create mocks + SharedIndexInformer podInformer = EasyMock.createMock(SharedIndexInformer.class); + Indexer indexer = EasyMock.createMock(Indexer.class); + + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .endMetadata() + .build(); + + // Set up expectations + EasyMock.expect(podInformer.getIndexer()).andReturn(indexer); + EasyMock.expect(indexer.byIndex("byJobName", KUBERNETES_JOB_NAME)) + .andReturn(List.of(pod)); + + EasyMock.replay(podInformer, indexer); + + clientApi = new TestKubernetesClient(this.client, podInformer, null); + instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + + Optional result = instance.getPeonPod(KUBERNETES_JOB_NAME); + + Assertions.assertTrue(result.isPresent()); + Assertions.assertEquals(POD_NAME, result.get().getMetadata().getName()); + + EasyMock.verify(podInformer, indexer); + } + + @Test + void test_getPeonPod_withoutPodInCache_returnsAbsentOptional() + { + SharedIndexInformer podInformer = EasyMock.createMock(SharedIndexInformer.class); + Indexer indexer = EasyMock.createMock(Indexer.class); + + EasyMock.expect(podInformer.getIndexer()).andReturn(indexer); + EasyMock.expect(indexer.byIndex("byJobName", KUBERNETES_JOB_NAME)) + .andReturn(Collections.emptyList()); + + EasyMock.replay(podInformer, indexer); + + clientApi = new TestKubernetesClient(this.client, podInformer, null); + instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + + Optional result = instance.getPeonPod(KUBERNETES_JOB_NAME); + + Assertions.assertFalse(result.isPresent()); + + EasyMock.verify(podInformer, indexer); + } + + @Test + void test_getPeonPod_withMultiplePodsForSameJob_returnsFirstOne() + { + SharedIndexInformer podInformer = EasyMock.createMock(SharedIndexInformer.class); + Indexer indexer = EasyMock.createMock(Indexer.class); + + Pod pod1 = new PodBuilder() + .withNewMetadata() + .withName("pod-1") + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .endMetadata() + .build(); + + Pod pod2 = new PodBuilder() + .withNewMetadata() + .withName("pod-2") + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .endMetadata() + .build(); + + EasyMock.expect(podInformer.getIndexer()).andReturn(indexer); + EasyMock.expect(indexer.byIndex("byJobName", KUBERNETES_JOB_NAME)) + .andReturn(List.of(pod1, pod2)); + + EasyMock.replay(podInformer, indexer); + + clientApi = new TestKubernetesClient(this.client, podInformer, null); + instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + + Optional result = instance.getPeonPod(KUBERNETES_JOB_NAME); + + Assertions.assertTrue(result.isPresent()); + Assertions.assertEquals("pod-1", result.get().getMetadata().getName()); + + EasyMock.verify(podInformer, indexer); + } + + @Test + void test_getPeonJobs_withoutOverlordNamespace_returnsAllJobsFromCache() + { + SharedIndexInformer jobInformer = EasyMock.createMock(SharedIndexInformer.class); + Indexer indexer = EasyMock.createMock(Indexer.class); + + Job job = new JobBuilder() + .withNewMetadata() + .withName(KUBERNETES_JOB_NAME) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .build(); + + EasyMock.expect(jobInformer.getIndexer()).andReturn(indexer); + EasyMock.expect(indexer.list()).andReturn(List.of(job)); + + EasyMock.replay(jobInformer, indexer); + + clientApi = new TestKubernetesClient(this.client, null, jobInformer); + instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + + List jobs = instance.getPeonJobs(); + + Assertions.assertEquals(1, jobs.size()); + Assertions.assertEquals(KUBERNETES_JOB_NAME, jobs.get(0).getMetadata().getName()); + + EasyMock.verify(jobInformer, indexer); + } + + @Test + void test_getPeonJobs_withOverlordNamespace_returnsFilteredJobs() + { + SharedIndexInformer jobInformer = EasyMock.createMock(SharedIndexInformer.class); + Indexer indexer = EasyMock.createMock(Indexer.class); + + Job job = new JobBuilder() + .withNewMetadata() + .withName(KUBERNETES_JOB_NAME) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .addToLabels(DruidK8sConstants.OVERLORD_NAMESPACE_KEY, "overlord-ns") + .endMetadata() + .build(); + + EasyMock.expect(jobInformer.getIndexer()).andReturn(indexer); + EasyMock.expect(indexer.byIndex("byOverlordNamespace", "overlord-ns")) + .andReturn(List.of(job)); + + EasyMock.replay(jobInformer, indexer); + + clientApi = new TestKubernetesClient(this.client, null, jobInformer); + instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, "overlord-ns", false, serviceEmitter); + + List jobs = instance.getPeonJobs(); + + Assertions.assertEquals(1, jobs.size()); + Assertions.assertEquals(KUBERNETES_JOB_NAME, jobs.get(0).getMetadata().getName()); + + EasyMock.verify(jobInformer, indexer); + } + + @Test + void test_getPeonJobs_whenCacheEmpty_returnsEmptyList() + { + SharedIndexInformer jobInformer = EasyMock.createMock(SharedIndexInformer.class); + Indexer indexer = EasyMock.createMock(Indexer.class); + + EasyMock.expect(jobInformer.getIndexer()).andReturn(indexer); + EasyMock.expect(indexer.list()).andReturn(Collections.emptyList()); + + EasyMock.replay(jobInformer, indexer); + + clientApi = new TestKubernetesClient(this.client, null, jobInformer); + instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + + List jobs = instance.getPeonJobs(); + + Assertions.assertEquals(0, jobs.size()); + + EasyMock.verify(jobInformer, indexer); + } +} \ No newline at end of file diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java similarity index 65% rename from extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java rename to extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index d08fe5b15bed..06c9f1b86921 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -45,7 +45,7 @@ import java.util.concurrent.TimeUnit; @EnableKubernetesMockClient(crud = true) -public class KubernetesPeonClientTest +public class DirectKubernetesPeonClientTest { private static final String ID = "id"; private static final String TASK_NAME_PREFIX = ""; @@ -57,15 +57,15 @@ public class KubernetesPeonClientTest private KubernetesClient client; private KubernetesMockServer server; private KubernetesClientApi clientApi; - private KubernetesPeonClient instance; + private DirectKubernetesPeonClient instance; private StubServiceEmitter serviceEmitter; @BeforeEach public void setup() { - clientApi = new TestKubernetesClient(this.client); + clientApi = new TestKubernetesClient(this.client, null, null); serviceEmitter = new StubServiceEmitter("service", "host"); - instance = new KubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); } @Test @@ -105,16 +105,16 @@ void test_launchPeonJobAndWaitForStart_withDisappearingPod_throwIllegalStateExce .build(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addToLabels("job-name", JOB_NAME) - .endMetadata() - .endItem() - .build() - ).once(); + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() + .addNewItem() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", JOB_NAME) + .endMetadata() + .endItem() + .build() + ).once(); Assertions.assertThrows( IllegalStateException.class, @@ -148,7 +148,7 @@ void test_launchPeonJobAndWaitForStart_withPendingPod_throwIllegalStateException () -> instance.launchPeonJobAndWaitForStart(job, NoopTask.create(), 1, TimeUnit.SECONDS) ); } - + @Test void test_waitForPeonJobCompletion_withSuccessfulJob_returnsJobResponseWithJobAndSucceededPeonPhase() { @@ -235,8 +235,8 @@ void test_deletePeonJob_withoutJob_returnsFalse() @Test void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() { - KubernetesPeonClient instance = new KubernetesPeonClient( - new TestKubernetesClient(this.client), + DirectKubernetesPeonClient instance = new DirectKubernetesPeonClient( + new TestKubernetesClient(this.client, null, null), NAMESPACE, true, serviceEmitter @@ -260,8 +260,8 @@ void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() @Test void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() { - KubernetesPeonClient instance = new KubernetesPeonClient( - new TestKubernetesClient(this.client), + DirectKubernetesPeonClient instance = new DirectKubernetesPeonClient( + new TestKubernetesClient(this.client, null, null), NAMESPACE, true, serviceEmitter @@ -274,48 +274,48 @@ void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() void test_getPeonLogs_withJob_returnsInputStreamInOptional() { server.expect().get() - .withPath("/apis/batch/v1/namespaces/namespace/jobs/" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder() - .withNewMetadata() - .withName(KUBERNETES_JOB_NAME) - .withUid("uid") - .endMetadata() - .withNewSpec() - .withNewTemplate() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endTemplate() - .endSpec() - .build() - ).once(); + .withPath("/apis/batch/v1/namespaces/namespace/jobs/" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder() + .withNewMetadata() + .withName(KUBERNETES_JOB_NAME) + .withUid("uid") + .endMetadata() + .withNewSpec() + .withNewTemplate() + .withNewSpec() + .addNewContainer() + .withName("main") + .endContainer() + .endSpec() + .endTemplate() + .endSpec() + .build() + ).once(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=controller-uid%3Duid") - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addNewOwnerReference() - .withUid("uid") - .withController(true) - .endOwnerReference() - .endMetadata() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endItem() - .build() - ).once(); + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=controller-uid%3Duid") + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() + .addNewItem() + .withNewMetadata() + .withName(POD_NAME) + .addNewOwnerReference() + .withUid("uid") + .withController(true) + .endOwnerReference() + .endMetadata() + .withNewSpec() + .addNewContainer() + .withName("main") + .endContainer() + .endSpec() + .endItem() + .build() + ).once(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") - .andReturn(HttpURLConnection.HTTP_OK, "data") - .once(); + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); Optional maybeInputStream = instance.getPeonLogs(new K8sTaskId(TASK_NAME_PREFIX, ID)); Assertions.assertTrue(maybeInputStream.isPresent()); @@ -355,7 +355,7 @@ void test_getPeonJobs_withJob_returnsPodList() client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(false); + List jobs = instance.getPeonJobs(); Assertions.assertEquals(1, jobs.size()); } @@ -363,7 +363,7 @@ void test_getPeonJobs_withJob_returnsPodList() @Test void test_getPeonJobs_withJobInDifferentNamespace_returnsPodList() { - instance = new KubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); + instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); Job job = new JobBuilder() .withNewMetadata() @@ -375,7 +375,7 @@ void test_getPeonJobs_withJobInDifferentNamespace_returnsPodList() client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(false); + List jobs = instance.getPeonJobs(); Assertions.assertEquals(1, jobs.size()); } @@ -383,7 +383,7 @@ void test_getPeonJobs_withJobInDifferentNamespace_returnsPodList() @Test void test_getPeonJobs_withJobInDifferentNamespaceButOverlordNamespaceNotSpecified_doesNotReturnPodList() { - instance = new KubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); + instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); Job job = new JobBuilder() .withNewMetadata() @@ -395,7 +395,7 @@ void test_getPeonJobs_withJobInDifferentNamespaceButOverlordNamespaceNotSpecifie client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(false); + List jobs = instance.getPeonJobs(); Assertions.assertEquals(0, jobs.size()); } @@ -403,7 +403,7 @@ void test_getPeonJobs_withJobInDifferentNamespaceButOverlordNamespaceNotSpecifie @Test void test_getPeonJobs_withJobInSameNamespaceWithoutLabels_doesNotReturnPodList() { - instance = new KubernetesPeonClient(clientApi, NAMESPACE, NAMESPACE, false, serviceEmitter); + instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, NAMESPACE, false, serviceEmitter); Job job = new JobBuilder() .withNewMetadata() @@ -414,7 +414,7 @@ void test_getPeonJobs_withJobInSameNamespaceWithoutLabels_doesNotReturnPodList() client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - List jobs = instance.getPeonJobs(false); + List jobs = instance.getPeonJobs(); Assertions.assertEquals(0, jobs.size()); } @@ -422,7 +422,7 @@ void test_getPeonJobs_withJobInSameNamespaceWithoutLabels_doesNotReturnPodList() @Test void test_getPeonJobs_withoutJob_returnsEmptyList() { - List jobs = instance.getPeonJobs(false); + List jobs = instance.getPeonJobs(); Assertions.assertEquals(0, jobs.size()); } @@ -563,52 +563,128 @@ void test_getPeonPod_withoutPod_returnsEmptyOptional() Assertions.assertFalse(maybePod.isPresent()); } + @Test + void test_getPeonPodWithRetries_withPod_returnsPod() + { + server.expect().get() + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder().build()) + .once(); + + server.expect().get() + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() + .addNewItem() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .endMetadata() + .endItem() + .build() + ).once(); + + Pod pod = instance.getPeonPodWithRetries(clientApi.getClient(), new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName(), 0, 2); + + Assertions.assertNotNull(pod); + } + + @Test + void test_getPeonPodWithRetries_withoutPod_raisesKubernetesResourceNotFoundException() + { + String k8sJobName = new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName(); + KubernetesResourceNotFoundException e = Assertions.assertThrows( + KubernetesResourceNotFoundException.class, + () -> instance.getPeonPodWithRetries(clientApi.getClient(), k8sJobName, 1, 1) + ); + + Assertions.assertEquals(e.getMessage(), + StringUtils.format("K8s pod with label[job-name=%s] not found", k8sJobName)); + } + + @Test + void test_getPeonPodWithRetries_withoutPod_noRestartForBlacklistedEvent_raisesKubernetesResourceNotFoundException() + { + String k8sJobName = new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName(); + String blacklistedMessage = DruidK8sConstants.BLACKLISTED_PEON_POD_ERROR_MESSAGES.get(0); + + final String eventsPath = "/api/v1/namespaces/namespace/events?fieldSelector=" + + "involvedObject.name%3D" + k8sJobName + + "%2CinvolvedObject.namespace%3D" + NAMESPACE + + "%2CinvolvedObject.kind%3DJob" + + "%2CinvolvedObject.apiVersion%3Dbatch%2Fv1"; + + server.expect().get() + .withPath(eventsPath) + .andReturn(HttpURLConnection.HTTP_OK, new EventListBuilder() + .addNewItem() + .withMessage(blacklistedMessage) + .withNewInvolvedObject() + .withApiVersion("batch/v1") + .withKind("Job") + .withName(k8sJobName) + .withNamespace(NAMESPACE) + .endInvolvedObject() + .endItem() + .build()) + // Test will fail if task is retried more than once. + .once(); + + // Task declared to retry for 3 times should only try once when a blacklisted event message is found. + KubernetesResourceNotFoundException e = Assertions.assertThrows( + KubernetesResourceNotFoundException.class, + () -> instance.getPeonPodWithRetries(clientApi.getClient(), k8sJobName, 0, 3) + ); + + // Ensure event message is propagated to the users. + Assertions.assertTrue(e.getMessage().contains(blacklistedMessage)); + } + @Test void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() { server.expect().get() - .withPath("/apis/batch/v1/namespaces/namespace/jobs/" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder() - .withNewMetadata() - .withName(KUBERNETES_JOB_NAME) - .withUid("uid") - .endMetadata() - .withNewSpec() - .withNewTemplate() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endTemplate() - .endSpec() - .build() - ).once(); + .withPath("/apis/batch/v1/namespaces/namespace/jobs/" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder() + .withNewMetadata() + .withName(KUBERNETES_JOB_NAME) + .withUid("uid") + .endMetadata() + .withNewSpec() + .withNewTemplate() + .withNewSpec() + .addNewContainer() + .withName("main") + .endContainer() + .endSpec() + .endTemplate() + .endSpec() + .build() + ).once(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=controller-uid%3Duid") - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addNewOwnerReference() - .withUid("uid") - .withController(true) - .endOwnerReference() - .endMetadata() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endItem() - .build() - ).once(); + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=controller-uid%3Duid") + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() + .addNewItem() + .withNewMetadata() + .withName(POD_NAME) + .addNewOwnerReference() + .withUid("uid") + .withController(true) + .endOwnerReference() + .endMetadata() + .withNewSpec() + .addNewContainer() + .withName("main") + .endContainer() + .endSpec() + .endItem() + .build() + ).once(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") - .andReturn(HttpURLConnection.HTTP_OK, "data") - .once(); + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); Optional maybeLogWatch = instance.getPeonLogWatcher(new K8sTaskId(TASK_NAME_PREFIX, ID)); Assertions.assertTrue(maybeLogWatch.isPresent()); @@ -668,9 +744,9 @@ void test_createK8sJobWithRetries_withNonRetryableException_failsImmediately() // Return 403 Forbidden - this is not a retryable exception server.expect().post() - .withPath(jobPath) - .andReturn(HttpURLConnection.HTTP_FORBIDDEN, "Forbidden: insufficient permissions") - .once(); + .withPath(jobPath) + .andReturn(HttpURLConnection.HTTP_FORBIDDEN, "Forbidden: insufficient permissions") + .once(); // Should fail immediately without retries DruidException e = Assertions.assertThrows( @@ -695,13 +771,80 @@ void test_createK8sJobWithRetries_withJobAlreadyExists_succeedsGracefully() // Return 409 Conflict - job already exists server.expect().post() - .withPath(jobPath) - .andReturn(HttpURLConnection.HTTP_CONFLICT, "Job already exists") - .once(); + .withPath(jobPath) + .andReturn(HttpURLConnection.HTTP_CONFLICT, "Job already exists") + .once(); // Should succeed gracefully without throwing exception Assertions.assertDoesNotThrow( () -> instance.createK8sJobWithRetries(clientApi.getClient(), job, 0, 5) ); } -} + + @Test + void test_waitForPodResultWithRetries_withSuccessfulPodReady_returnsPod() + { + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .endMetadata() + .withNewStatus() + .withPodIP("192.168.1.100") + .endStatus() + .build(); + + // Create the pod in the mock client + client.pods().inNamespace(NAMESPACE).resource(pod).create(); + + // Should return the pod successfully + Pod result = instance.waitForPodResultWithRetries( + clientApi.getClient(), + pod, + 1, + TimeUnit.SECONDS, + 0, + 3 + ); + + Assertions.assertNotNull(result); + Assertions.assertEquals(POD_NAME, result.getMetadata().getName()); + Assertions.assertEquals("192.168.1.100", result.getStatus().getPodIP()); + } + + @Test + void test_waitForPodResultWithRetries_withNonRetryableFailure_throwsDruidException() + { + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .endMetadata() + .withNewStatus() + .withPodIP(null) // Pod without IP, will timeout + .endStatus() + .build(); + + String podPath = "/api/v1/namespaces/" + NAMESPACE + "/pods/" + POD_NAME; + + // Mock server to return the pod without IP, causing timeout + server.expect().get() + .withPath(podPath + "?watch=true") + .andReturn(HttpURLConnection.HTTP_INTERNAL_ERROR, "Internal server error") + .once(); + + // Should throw DruidException after failure + DruidException e = Assertions.assertThrows( + DruidException.class, + () -> instance.waitForPodResultWithRetries( + clientApi.getClient(), + pod, + 1, + TimeUnit.MILLISECONDS, // Very short timeout to force failure + 0, + 1 + ) + ); + + // Verify the error message contains our pod name + Assertions.assertTrue(e.getMessage().contains(POD_NAME)); + } +} \ No newline at end of file diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java index e353847b61c1..4cb2a97432e9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -28,10 +28,17 @@ public class TestKubernetesClient implements KubernetesClientApi { private final KubernetesClient client; + private final SharedIndexInformer podInformer; + private final SharedIndexInformer jobInformer; - public TestKubernetesClient(KubernetesClient client) + public TestKubernetesClient(KubernetesClient client, + SharedIndexInformer podInformer, + SharedIndexInformer jobInformer + ) { this.client = client; + this.podInformer = podInformer; + this.jobInformer = jobInformer; } @Override @@ -43,13 +50,13 @@ public T executeRequest(KubernetesExecutor executor) throws KubernetesRes @Override public T executePodCacheRequest(KubernetesInformerExecutor executor) { - return null; + return executor.executeRequest(podInformer); } @Override public T executeJobCacheRequest(KubernetesInformerExecutor executor) { - return null; + return executor.executeRequest(jobInformer); } @Override @@ -61,12 +68,12 @@ public KubernetesClient getClient() @Override public SharedIndexInformer getPodInformer() { - return null; + return podInformer; } @Override public SharedIndexInformer getJobInformer() { - return null; + return jobInformer; } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 601692ec56c7..b2765ad0a732 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -21,48 +21,22 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.PodSpec; -import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.ConfigBuilder; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.IndexTask; -import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.apache.druid.k8s.overlord.common.DirectKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesHttpClientConfig; -import org.apache.druid.k8s.overlord.common.JobResponse; -import org.apache.druid.k8s.overlord.common.K8sTaskId; -import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.KubernetesClientApi; -import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; -import org.apache.druid.k8s.overlord.common.PeonCommandContext; -import org.apache.druid.k8s.overlord.common.PeonPhase; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.File; -import java.io.InputStream; -import java.io.UncheckedIOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import static org.junit.Assert.assertTrue; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -75,7 +49,7 @@ public class DruidPeonClientIntegrationTest private TaskConfig taskConfig; private DruidNode druidNode; private KubernetesClientApi k8sClient; - private KubernetesPeonClient peonClient; + private AbstractKubernetesPeonClient peonClient; private ObjectMapper jsonMapper; @BeforeEach @@ -87,8 +61,8 @@ public void setup() new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), new NamedType(IndexTask.IndexTuningConfig.class, "index") ); - k8sClient = new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build()); - peonClient = new KubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); + k8sClient = new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build(), false); + peonClient = new DirectKubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); druidNode = new DruidNode( "test", null, diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 16cccf04e280..3df5ac272ace 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -121,7 +121,7 @@ public K8sTaskAdapterTest() void testAddingLabelsAndAnnotations() throws IOException { final PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); - TestKubernetesClient testClient = new TestKubernetesClient(client) + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null) { @SuppressWarnings("unchecked") @Override @@ -174,7 +174,7 @@ public PodSpec getSpec() public void serializingAndDeserializingATask() throws IOException { // given a task create a k8s job - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); @@ -212,7 +212,7 @@ public void serializingAndDeserializingATask() throws IOException public void fromTask_dontSetTaskJSON() throws IOException { final PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); - TestKubernetesClient testClient = new TestKubernetesClient(client) + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null) { @SuppressWarnings("unchecked") @Override @@ -276,7 +276,7 @@ public PodSpec getSpec() @Test public void toTask_useTaskPayloadManager() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); @@ -308,7 +308,7 @@ public void toTask_useTaskPayloadManager() throws IOException @Test public void getTaskId() { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -330,7 +330,7 @@ public void getTaskId() @Test public void getTaskId_noAnnotations() { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -352,7 +352,7 @@ public void getTaskId_noAnnotations() @Test public void getTaskId_missingTaskIdAnnotation() { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -451,7 +451,7 @@ void testNoPrimaryFound() @Test void testAddingMonitors() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); PeonCommandContext context = new PeonCommandContext( new ArrayList<>(), new ArrayList<>(), @@ -530,7 +530,7 @@ void testAddingMonitors() throws IOException @Test void testEphemeralStorageIsRespected() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); Pod pod = K8sTestUtils.fileToResource("ephemeralPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") @@ -580,7 +580,7 @@ void testEphemeralStorageIsRespected() throws IOException @Test void testProbesRemoved() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); Pod pod = K8sTestUtils.fileToResource("probesPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") @@ -630,7 +630,7 @@ void testProbesRemoved() throws IOException @Test void testCPUResourceIsRespected() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); Pod pod = K8sTestUtils.fileToResource("ephemeralPodSpec.yaml", Pod.class); List javaOpts = new ArrayList<>(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java index 48330d625069..844053667d17 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java @@ -85,7 +85,7 @@ public void setup() @Test public void testMultiContainerSupport() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") @@ -136,7 +136,7 @@ public void testMultiContainerSupport() throws IOException @Test public void testMultiContainerSupportWithNamedContainer() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpecOrder.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") @@ -190,7 +190,7 @@ public void testMultiContainerSupportWithNamedContainer() throws IOException @Test public void testOverridingPeonMonitors() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); Pod pod = K8sTestUtils.fileToResource("podSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java index d7d74cf1812a..60fa5eb31ed8 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java @@ -84,7 +84,7 @@ public void setup() @Test public void testSingleContainerSupport() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client); + TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") From ce7493a036b2c4056161c11ea2e07cbe7cdb02ca Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 5 Oct 2025 18:59:43 -0500 Subject: [PATCH 04/58] checkstyle cleanup --- .../k8s/overlord/KubernetesPeonLifecycle.java | 2 +- .../KubernetesPeonLifecycleFactory.java | 2 +- .../k8s/overlord/KubernetesTaskRunner.java | 2 +- .../overlord/KubernetesTaskRunnerFactory.java | 2 +- .../common/AbstractKubernetesPeonClient.java | 43 ++++--- .../common/CachingKubernetesPeonClient.java | 38 +++++- .../common/DirectKubernetesPeonClient.java | 19 +++ .../common/DruidKubernetesClient.java | 78 +++++++----- .../common/KubernetesInformerExecutor.java | 19 +++ .../overlord/KubernetesPeonLifecycleTest.java | 2 +- .../overlord/KubernetesTaskRunnerTest.java | 2 +- .../CachingKubernetesPeonClientTest.java | 2 +- .../DirectKubernetesPeonClientTest.java | 2 +- .../DruidPeonClientIntegrationTest.java | 120 +++++++++++++++++- 14 files changed, 267 insertions(+), 66 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index ca84c84de7f9..ed3ee695488f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -36,10 +36,10 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java index fd23dddc8b48..ebb3e6737010 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.tasklogs.TaskLogs; public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 3653bb6ffeab..9155dc124e9d 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -53,8 +53,8 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; -import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.TaskLogStreamer; diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 16e19499ee7f..dbb58b325c4a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -26,10 +26,10 @@ import org.apache.druid.indexing.overlord.TaskRunnerFactory; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.CachingKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.DirectKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.TaskLogs; diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index 23b3804cf326..7c70252ccc9a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -56,7 +56,7 @@ public abstract class AbstractKubernetesPeonClient private final boolean debugJobs; private final ServiceEmitter emitter; - AbstractKubernetesPeonClient( + AbstractKubernetesPeonClient( KubernetesClientApi clientApi, String namespace, String overlordNamespace, @@ -94,13 +94,13 @@ public abstract class AbstractKubernetesPeonClient * Waits for a pod associated with a job to be created and reach ready state using the pod cache. * This method polls the informer cache until the pod appears and has a pod IP assigned. * - * @param jobName the name of the job whose pod we're waiting for - * @param howLong the maximum time to wait + * @param jobName the name of the job whose pod we're waiting for + * @param howLong the maximum time to wait * @param timeUnit the time unit for the timeout * @return the pod in ready state, or null if the pod disappeared after being seen * @throws DruidException if the pod never appears within the timeout period */ - protected abstract Pod waitUntilPeonPodCreatedAndReady( String jobName, long howLong, TimeUnit timeUnit); + protected abstract Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit); /** * Launches the given Job. Waits for the associated pod and job to be created and start running. @@ -121,7 +121,10 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn // Evaluate result of job launch if (result == null) { - throw new ISE("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", task.getId()); + throw new ISE( + "K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", + task.getId() + ); } log.info("Pod for job[%s] is in state [%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); long duration = System.currentTimeMillis() - start; @@ -155,12 +158,12 @@ public Optional getPeonLogWatcher(K8sTaskId taskId) KubernetesClient k8sClient = clientApi.getClient(); try { LogWatch logWatch = k8sClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .inContainer("main") - .watchLog(); + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .inContainer("main") + .watchLog(); if (logWatch == null) { return Optional.absent(); } @@ -177,12 +180,12 @@ public Optional getPeonLogs(K8sTaskId taskId) KubernetesClient k8sClient = clientApi.getClient(); try { InputStream logStream = k8sClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .inContainer("main") - .getLogInputStream(); + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .inContainer("main") + .getLogInputStream(); if (logStream == null) { return Optional.absent(); } @@ -250,10 +253,10 @@ public void createK8sJobWithRetries(Job job) * The retry logic only applies to transient connection pool exceptions. Other exceptions will cause the method to * fail immediately. * - * @param client the Kubernetes client to use for job creation - * @param job the Kubernetes job to create + * @param client the Kubernetes client to use for job creation + * @param job the Kubernetes job to create * @param quietTries number of initial retry attempts without logging warnings - * @param maxTries maximum total number of retry attempts + * @param maxTries maximum total number of retry attempts * @throws DruidException if job creation fails after all retry attempts or encounters non-retryable errors */ @VisibleForTesting diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 41e683e0a9ac..d61af3b56600 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.k8s.overlord.common; import com.google.common.base.Optional; @@ -57,7 +76,8 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time // Checking before the informer has synced will likely result in a false negative. try { Thread.sleep(pollInterval); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); } @@ -75,7 +95,11 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time .orElse(null)); } else { job = clientApi.executeJobCacheRequest(informer -> - informer.getIndexer().byIndex("byJobName", taskId.getK8sJobName()).stream().findFirst().orElse(null)); + informer.getIndexer() + .byIndex("byJobName", taskId.getK8sJobName()) + .stream() + .findFirst() + .orElse(null)); } if (job == null) { @@ -86,7 +110,8 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time log.debug("Job [%s] not yet in cache, waiting... (elapsed: %d ms)", taskId, elapsed); try { Thread.sleep(pollInterval); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); } @@ -114,7 +139,8 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time // Job still running, wait and check again try { Thread.sleep(pollInterval); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); } @@ -131,8 +157,8 @@ public List getPeonJobs() return clientApi.executeJobCacheRequest(informer -> informer.getIndexer().list()); } else { return clientApi.executeJobCacheRequest(informer -> - informer.getIndexer() - .byIndex("byOverlordNamespace", overlordNamespace)); + informer.getIndexer() + .byIndex("byOverlordNamespace", overlordNamespace)); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java index 81d9b449176b..052000498c5e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.k8s.overlord.common; import com.google.common.annotations.VisibleForTesting; diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index cde7084a4573..78a9ded617bc 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -45,8 +45,10 @@ public class DruidKubernetesClient implements KubernetesClientApi private final SharedIndexInformer podInformer; private final SharedIndexInformer jobInformer; - public DruidKubernetesClient(DruidKubernetesHttpClientConfig httpClientConfig, Config kubernetesClientConfig, - boolean enableCache + public DruidKubernetesClient( + DruidKubernetesHttpClientConfig httpClientConfig, + Config kubernetesClientConfig, + boolean enableCache ) { this.kubernetesClient = new KubernetesClientBuilder() @@ -116,20 +118,28 @@ private SharedIndexInformer setupPodInformer(String namespace) SharedIndexInformer podInformer = kubernetesClient.pods() .inNamespace(namespace) - .inform(new ResourceEventHandler<>() { - @Override - public void onAdd(Pod pod) { - log.info("Pod " + pod.getMetadata().getName() + " got added"); - } - @Override - public void onUpdate(Pod oldPod, Pod newPod) { - log.info("Pod " + oldPod.getMetadata().getName() + " got updated"); - } - @Override - public void onDelete(Pod pod, boolean deletedFinalStateUnknown) { - log.info("Pod " + pod.getMetadata().getName() + " got deleted"); - } - }, INFORMER_RESYNC_PERIOD_MS); + .inform( + new ResourceEventHandler<>() + { + @Override + public void onAdd(Pod pod) + { + log.info("Pod " + pod.getMetadata().getName() + " got added"); + } + + @Override + public void onUpdate(Pod oldPod, Pod newPod) + { + log.info("Pod " + oldPod.getMetadata().getName() + " got updated"); + } + + @Override + public void onDelete(Pod pod, boolean deletedFinalStateUnknown) + { + log.info("Pod " + pod.getMetadata().getName() + " got deleted"); + } + }, INFORMER_RESYNC_PERIOD_MS + ); Function> jobNameIndexer = pod -> { if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { @@ -156,20 +166,28 @@ private SharedIndexInformer setupJobInformer(String namespace) .jobs() .inNamespace(namespace) .withLabel(DruidK8sConstants.LABEL_KEY) - .inform(new ResourceEventHandler<>() { - @Override - public void onAdd(Job job) { - log.info("Job " + job.getMetadata().getName() + " got added"); - } - @Override - public void onUpdate(Job oldJob, Job newJob) { - log.info("Job " + oldJob.getMetadata().getName() + " got updated"); - } - @Override - public void onDelete(Job job, boolean deletedFinalStateUnknown) { - log.info("Job " + job.getMetadata().getName() + " got deleted"); - } - }, INFORMER_RESYNC_PERIOD_MS); + .inform( + new ResourceEventHandler<>() + { + @Override + public void onAdd(Job job) + { + log.info("Job " + job.getMetadata().getName() + " got added"); + } + + @Override + public void onUpdate(Job oldJob, Job newJob) + { + log.info("Job " + oldJob.getMetadata().getName() + " got updated"); + } + + @Override + public void onDelete(Job job, boolean deletedFinalStateUnknown) + { + log.info("Job " + job.getMetadata().getName() + " got deleted"); + } + }, INFORMER_RESYNC_PERIOD_MS + ); Function> overlordNamespaceIndexer = job -> { if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java index 0364caac368b..53cd6c13575f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.k8s.overlord.common; import io.fabric8.kubernetes.client.informers.SharedIndexInformer; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 441850ae9345..3559be031498 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -31,10 +31,10 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index ea8889d3245b..4882a31bf83c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -37,8 +37,8 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; -import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index 589c886c7fc0..db23c2b73a4c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -231,4 +231,4 @@ void test_getPeonJobs_whenCacheEmpty_returnsEmptyList() EasyMock.verify(jobInformer, indexer); } -} \ No newline at end of file +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index 06c9f1b86921..95a6af3b1c09 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -847,4 +847,4 @@ void test_waitForPodResultWithRetries_withNonRetryableFailure_throwsDruidExcepti // Verify the error message contains our pod name Assertions.assertTrue(e.getMessage().contains(POD_NAME)); } -} \ No newline at end of file +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index b2765ad0a732..5220eeadd13c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -21,22 +21,49 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodSpec; +import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.ConfigBuilder; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.DirectKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesHttpClientConfig; +import org.apache.druid.k8s.overlord.common.JobResponse; +import org.apache.druid.k8s.overlord.common.K8sTaskId; +import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.KubernetesClientApi; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.PeonCommandContext; +import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.junit.Assert.assertTrue; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -49,7 +76,7 @@ public class DruidPeonClientIntegrationTest private TaskConfig taskConfig; private DruidNode druidNode; private KubernetesClientApi k8sClient; - private AbstractKubernetesPeonClient peonClient; + private DirectKubernetesPeonClient peonClient; private ObjectMapper jsonMapper; @BeforeEach @@ -75,4 +102,93 @@ public void setup() startupLoggingConfig = new StartupLoggingConfig(); taskConfig = new TaskConfigBuilder().setBaseDir("src/test/resources").build(); } + + @Disabled + @Test + public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception + { + PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); + + Task task = K8sTestUtils.getTask(); + KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() + .withNamespace("default") + .build(); + K8sTaskAdapter adapter = new SingleContainerTaskAdapter( + k8sClient, + config, + taskConfig, + startupLoggingConfig, + druidNode, + jsonMapper, + null + ); + String taskBasePath = "/home/taskDir"; + PeonCommandContext context = new PeonCommandContext( + Collections.singletonList("sleep 10; for i in `seq 1 1000`; do echo $i; done; exit 0"), + new ArrayList<>(), + new File(taskBasePath), + config.getCpuCoreInMicro() + ); + + Job job = adapter.createJobFromPodSpec(podSpec, task, context); + + // launch the job and wait to start... + peonClient.launchPeonJobAndWaitForStart(job, task, 1, TimeUnit.MINUTES); + + // there should be one job that is a k8s peon job that exists + List jobs = peonClient.getPeonJobs(); + assertEquals(1, jobs.size()); + + K8sTaskId taskId = new K8sTaskId(null, task.getId()); + InputStream peonLogs = peonClient.getPeonLogs(taskId).get(); + List expectedLogs = IntStream.range(1, 1001).boxed().collect(Collectors.toList()); + List actualLogs = new ArrayList<>(); + Thread thread = new Thread(() -> { + try { + actualLogs.addAll(IOUtils.readLines(peonLogs, "UTF-8") + .stream() + .map(Integer::parseInt) + .collect(Collectors.toList())); + } + catch (UncheckedIOException e) { + throw new RuntimeException(e); + } + }); + thread.start(); + + // assert that the env variable is corret + Task taskFromEnvVar = adapter.toTask(job); + assertEquals(task, taskFromEnvVar); + + // now copy the task.json file from the pod and make sure its the same as our task.json we expected + Path downloadPath = Paths.get(tempDir.toAbsolutePath().toString(), "task.json"); + Optional maybeMainJobPod = peonClient.getPeonPod(taskId.getK8sJobName()); + assertTrue(maybeMainJobPod.isPresent()); + Pod mainJobPod = maybeMainJobPod.get(); + k8sClient.executeRequest(client -> { + client.pods() + .inNamespace("default") + .withName(mainJobPod.getMetadata().getName()) + .file(Paths.get(taskBasePath, "task.json").toString()) + .copy(downloadPath); + return null; + }); + + String taskJsonFromPod = FileUtils.readFileToString(new File(downloadPath.toString()), StandardCharsets.UTF_8); + Task taskFromPod = jsonMapper.readValue(taskJsonFromPod, Task.class); + assertEquals(task, taskFromPod); + + + JobResponse jobStatusResult = peonClient.waitForPeonJobCompletion(taskId, 2, TimeUnit.MINUTES); + thread.join(); + assertEquals(PeonPhase.SUCCEEDED, jobStatusResult.getPhase()); + // as long as there were no exceptions we are good! + assertEquals(expectedLogs, actualLogs); + // cleanup my job + assertTrue(peonClient.deletePeonJob(taskId)); + + // we cleaned up the job, none should exist + List existingJobs = peonClient.getPeonJobs(); + assertEquals(0, existingJobs.size()); + } } From 8f13cda5869f1c9f60e33d4f701cf1a9baec09c0 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 7 Oct 2025 19:08:22 -0500 Subject: [PATCH 05/58] Remove the busy waiting. Overhaul caching client testing --- .../common/AbstractKubernetesPeonClient.java | 9 + .../common/CachingKubernetesPeonClient.java | 225 +++--- .../common/DirectKubernetesPeonClient.java | 17 + .../common/DruidKubernetesClient.java | 38 +- .../overlord/common/KubernetesClientApi.java | 2 + .../KubernetesResourceEventNotifier.java | 116 ++++ .../CachingKubernetesPeonClientTest.java | 653 +++++++++++++++--- .../KubernetesResourceEventNotifierTest.java | 313 +++++++++ .../overlord/common/TestKubernetesClient.java | 6 + 9 files changed, 1148 insertions(+), 231 deletions(-) create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java create mode 100644 extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index 7c70252ccc9a..c28c73fd652f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -36,6 +36,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import javax.annotation.Nullable; import java.io.InputStream; import java.sql.Timestamp; import java.util.ArrayList; @@ -90,6 +91,13 @@ public abstract class AbstractKubernetesPeonClient */ public abstract Optional getPeonPod(String jobName); + /** + * Get the Job with the given name, if it exists + * + * @return an Optional containing the Job if it exists, or absent if not found + */ + public abstract Optional getPeonJob(String jobName); + /** * Waits for a pod associated with a job to be created and reach ready state using the pod cache. * This method polls the informer cache until the pod appears and has a pod IP assigned. @@ -100,6 +108,7 @@ public abstract class AbstractKubernetesPeonClient * @return the pod in ready state, or null if the pod disappeared after being seen * @throws DruidException if the pod never appears within the timeout period */ + @Nullable protected abstract Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit); /** diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index d61af3b56600..879266e7a0ad 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -26,7 +26,9 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import javax.annotation.Nullable; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; /** @@ -66,87 +68,38 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time { long timeoutMs = unit.toMillis(howLong); long startTime = System.currentTimeMillis(); - long pollInterval = 5000; - long jobAppearanceGracePeriodMs = 90000; // 90 seconds grace for job to appear in cache - - boolean jobSeenInCache = false; + CompletableFuture jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); do { - if (!clientApi.getJobInformer().hasSynced()) { - // Checking before the informer has synced will likely result in a false negative. - try { - Thread.sleep(pollInterval); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - continue; - } - - Job job; - if (!overlordNamespace.isEmpty()) { - job = clientApi.executeJobCacheRequest((informer) -> - informer.getIndexer() - .byIndex("byOverlordNamespace", overlordNamespace).stream() - .filter(j -> taskId.getK8sJobName() - .equals(j.getMetadata().getName())) - .findFirst() - .orElse(null)); - } else { - job = clientApi.executeJobCacheRequest(informer -> - informer.getIndexer() - .byIndex("byJobName", taskId.getK8sJobName()) - .stream() - .findFirst() - .orElse(null)); - } - - if (job == null) { - long elapsed = System.currentTimeMillis() - startTime; - - // Give grace period for job to appear in cache after creation - if (!jobSeenInCache && elapsed < jobAppearanceGracePeriodMs) { - log.debug("Job [%s] not yet in cache, waiting... (elapsed: %d ms)", taskId, elapsed); - try { - Thread.sleep(pollInterval); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); + try { + Optional maybeJob = getPeonJob(taskId.getK8sJobName()); + if (maybeJob.isPresent()) { + Job job = maybeJob.get(); + JobResponse currentResponse = determineJobResponse(job); + if (currentResponse.getPhase() != PeonPhase.RUNNING) { + return currentResponse; } - continue; } - - // After grace period or if we've seen it before, job is truly missing - log.warn("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); - return new JobResponse(null, PeonPhase.FAILED); - } - - // Job found! Mark that we've seen it - jobSeenInCache = true; - - // Check if job is complete - if (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) { - if (job.getStatus().getSucceeded() > 0) { - log.info("K8s job [%s] completed successfully", taskId); - return new JobResponse(job, PeonPhase.SUCCEEDED); + Job job = jobFuture.get(timeoutMs, TimeUnit.MILLISECONDS); + // Immediately set up to watch for the next change in case we need to wait again + jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); + log.debug("Received job[%s] change notification", taskId.getK8sJobName()); + if (job == null) { + log.warn("K8s job for the task[%s] was not found. It can happen if the task was canceled", taskId); + return new JobResponse(null, PeonPhase.FAILED); } - log.warn("K8s job [%s] failed with status %s", taskId, job.getStatus()); - return new JobResponse(job, PeonPhase.FAILED); - } - // Job still running, wait and check again - try { - Thread.sleep(pollInterval); + JobResponse currentResponse = determineJobResponse(job); + if (currentResponse.getPhase() != PeonPhase.RUNNING) { + return currentResponse; + } } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); + catch (Throwable e) { + log.warn("Exception[%s] waiting for job change notification for job[%s]. Error message[%s]", e.getClass().getName(), taskId.getK8sJobName(), e.getMessage()); } } while (System.currentTimeMillis() - startTime < timeoutMs); - log.warn("Timed out waiting for K8s job [%s] to complete", taskId); + log.warn("Timed out waiting for K8s job[%s] to complete", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); } @@ -172,75 +125,93 @@ public Optional getPeonPod(String jobName) } @Override + public Optional getPeonJob(String jobName) + { + return clientApi.executeJobCacheRequest(informer -> { + List jobs = informer.getIndexer().byIndex("byJobName", jobName); + return jobs.isEmpty() ? Optional.absent() : Optional.of(jobs.get(0)); + }); + } + + @Override + @Nullable protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) { - return clientApi.executePodCacheRequest(informer -> { - long timeoutMs = timeUnit.toMillis(howLong); - long startTime = System.currentTimeMillis(); - long pollInterval = 2000; // Poll every 2 seconds - - boolean podSeenInCache = false; - String podName = null; - - do { - if (!informer.hasSynced()) { - // Wait for informer to sync - try { - Thread.sleep(pollInterval); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); + long timeoutMs = timeUnit.toMillis(howLong); + long startTime = System.currentTimeMillis(); + + String podName = "unknown"; + boolean podSeenInCache = false; + CompletableFuture podFuture = clientApi.getEventNotifier().waitForPodChange(jobName); + do { + try { + // First check to see if pod is already in cache and ready in case our completion future started after the update event fired + Optional maybePod = getPeonPod(jobName); + if (maybePod.isPresent()) { + podSeenInCache = true; + // Check if pod is ready (has IP) + Pod pod = maybePod.get(); + if (isPodReady(pod)) { + log.info("Pod[%s] for job[%s] is ready with IP[%s]", podName, jobName, pod.getStatus().getPodIP()); + return pod; + } else { + log.debug("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); } - continue; } - List pods = informer.getIndexer().byIndex("byJobName", jobName); - - if (pods.isEmpty()) { - // If we've seen the pod before, and now it's gone, it was deleted - if (podSeenInCache) { - log.warn("Pod for job[%s] disappeared after being seen in cache", jobName); - return null; - } - // Otherwise keep waiting for it to appear + Pod pod = podFuture.get(timeoutMs, TimeUnit.MILLISECONDS); + podFuture = clientApi.getEventNotifier().waitForPodChange(jobName); + log.debug("Received pod[%s] change notification for job[%s]", podName, jobName); + if (pod == null) { + throw DruidException.defensive("Pod[%s] for job[%s] is null. This is unusual. Investigate Druid and k8s logs.", podName, jobName); } else { - Pod currentPod = pods.get(0); podSeenInCache = true; - podName = currentPod.getMetadata().getName(); - - // Check if pod is ready (has IP) - if (currentPod.getStatus() != null && currentPod.getStatus().getPodIP() != null) { - log.info("Pod[%s] for job[%s] is ready with IP: %s", podName, jobName, currentPod.getStatus().getPodIP()); - return currentPod; + if (isPodReady(pod)) { + log.info("Pod[%s] for job[%s] is ready with IP[%s]", podName, jobName, pod.getStatus().getPodIP()); + return pod; + } else { + log.debug("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); } - - log.debug("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); } + } + catch (Throwable e) { + log.warn("Exception[%s] waiting for pod change notification for job [%s]. Error message[%s]", e.getClass().getName(), jobName, e.getMessage()); + } + } while (System.currentTimeMillis() - startTime < timeoutMs); - // Wait before polling again - long remainingTime = timeoutMs - (System.currentTimeMillis() - startTime); - if (remainingTime <= 0) { - break; - } + // Timeout + if (podSeenInCache) { + log.warn("Timeout waiting for pod[%s] for job[%s] to become ready", podName, jobName); + return null; + } else { + throw DruidException.defensive("Timeout waiting for pod for job[%s] to be created", jobName); + } + } - try { - Thread.sleep(Math.min(pollInterval, remainingTime)); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } while (System.currentTimeMillis() - startTime < timeoutMs); + /** + * Check if the pod is ready. For our purposes, this means it has been assigned an IP address. + */ + private boolean isPodReady(Pod pod) + { + return pod.getStatus() != null && pod.getStatus().getPodIP() != null; + } - // Timeout - if (podSeenInCache) { - log.warn("Timeout waiting for pod[%s] for job[%s] to become ready", podName, jobName); - return null; + /** + * Determine the JobResponse based on the current state of the Job. + */ + private JobResponse determineJobResponse(Job job) + { + if (job.getStatus().getSucceeded() != null || job.getStatus().getFailed() != null) { + if (job.getStatus().getSucceeded() != null && job.getStatus().getSucceeded() > 0) { + log.info("K8s job[%s] completed successfully", job.getMetadata().getName()); + return new JobResponse(job, PeonPhase.SUCCEEDED); } else { - throw DruidException.defensive("Timeout waiting for pod for job[%s] to be created", jobName); + log.warn("K8s job[%s] failed with status %s", job.getMetadata().getName(), job.getStatus()); + return new JobResponse(job, PeonPhase.FAILED); } - }); + } else { + log.debug("K8s job[%s] is still active.", job.getMetadata().getName()); + return new JobResponse(job, PeonPhase.RUNNING); + } } - } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java index 052000498c5e..3a9a572ad204 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java @@ -128,6 +128,12 @@ public Optional getPeonPod(String jobName) return clientApi.executeRequest(client -> getPeonPod(client, jobName)); } + @Override + public Optional getPeonJob(String jobName) + { + return clientApi.executeRequest(client -> getPeonJob(client, jobName)); + } + @Nullable @Override protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) @@ -224,6 +230,17 @@ private Optional getPeonPod(KubernetesClient client, String jobName) return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); } + private Optional getPeonJob(KubernetesClient client, String jobName) + { + Job job = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(jobName) + .get(); + return job == null ? Optional.absent() : Optional.of(job); + } + /** * Determines if this exception, specifically when containing Kubernetes job event messages, permits a retry attempt. *

    diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 78a9ded617bc..f02da1517da6 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -44,6 +44,7 @@ public class DruidKubernetesClient implements KubernetesClientApi private final KubernetesClient kubernetesClient; private final SharedIndexInformer podInformer; private final SharedIndexInformer jobInformer; + private final KubernetesResourceEventNotifier eventNotifier; public DruidKubernetesClient( DruidKubernetesHttpClientConfig httpClientConfig, @@ -56,9 +57,11 @@ public DruidKubernetesClient( .withConfig(kubernetesClientConfig) .build(); if (enableCache) { + this.eventNotifier = new KubernetesResourceEventNotifier(); this.podInformer = setupPodInformer(kubernetesClient.getNamespace()); this.jobInformer = setupJobInformer(kubernetesClient.getNamespace()); } else { + this.eventNotifier = null; this.podInformer = null; this.jobInformer = null; } @@ -113,6 +116,12 @@ public SharedIndexInformer getJobInformer() return jobInformer; } + @Override + public KubernetesResourceEventNotifier getEventNotifier() + { + return eventNotifier; + } + private SharedIndexInformer setupPodInformer(String namespace) { SharedIndexInformer podInformer = @@ -124,19 +133,22 @@ private SharedIndexInformer setupPodInformer(String namespace) @Override public void onAdd(Pod pod) { - log.info("Pod " + pod.getMetadata().getName() + " got added"); + log.debug("Pod[%s] got added", pod.getMetadata().getName()); + notifyPodChange(pod); } @Override public void onUpdate(Pod oldPod, Pod newPod) { - log.info("Pod " + oldPod.getMetadata().getName() + " got updated"); + log.debug("Pod[%s] got updated", oldPod.getMetadata().getName()); + notifyPodChange(newPod); } @Override public void onDelete(Pod pod, boolean deletedFinalStateUnknown) { - log.info("Pod " + pod.getMetadata().getName() + " got deleted"); + log.debug("Pod[%s] got deleted", pod.getMetadata().getName()); + notifyPodChange(pod); } }, INFORMER_RESYNC_PERIOD_MS ); @@ -172,19 +184,22 @@ private SharedIndexInformer setupJobInformer(String namespace) @Override public void onAdd(Job job) { - log.info("Job " + job.getMetadata().getName() + " got added"); + log.debug("Job[%s] got added", job.getMetadata().getName()); + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); } @Override public void onUpdate(Job oldJob, Job newJob) { - log.info("Job " + oldJob.getMetadata().getName() + " got updated"); + log.debug("Job[%s] got updated", newJob.getMetadata().getName()); + eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); } @Override public void onDelete(Job job, boolean deletedFinalStateUnknown) { - log.info("Job " + job.getMetadata().getName() + " got deleted"); + log.debug("Job[%s] got deleted", job.getMetadata().getName()); + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); } }, INFORMER_RESYNC_PERIOD_MS ); @@ -214,4 +229,15 @@ public void onDelete(Job job, boolean deletedFinalStateUnknown) return jobInformer; } + + private void notifyPodChange(Pod pod) + { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + // Prevents us from trying to notify pod changes that are not indexing jobs + eventNotifier.notifyPodChange(jobName, pod); + } + } + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java index 1569430cf1a7..68f16f19aec6 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java @@ -41,4 +41,6 @@ public interface KubernetesClientApi SharedIndexInformer getPodInformer(); SharedIndexInformer getJobInformer(); + + KubernetesResourceEventNotifier getEventNotifier(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java new file mode 100644 index 000000000000..e360ad4770a0 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; + +/** + * Manages event notifications for Kubernetes resources (Jobs and Pods). + * Allows tasks to wait for specific resource changes without polling, + * improving efficiency and responsiveness. + */ +public class KubernetesResourceEventNotifier +{ + private static final EmittingLogger log = new EmittingLogger(KubernetesResourceEventNotifier.class); + + private final ConcurrentHashMap>> jobWatchers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap>> podWatchers = new ConcurrentHashMap<>(); + + /** + * Register to be notified when a job with the given name changes. + * The returned future will complete when the job is added, updated, or deleted. + * + * @param jobName The name of the job to watch + * @return A future that completes when the job changes + */ + public CompletableFuture waitForJobChange(String jobName) + { + CompletableFuture future = new CompletableFuture<>(); + jobWatchers.computeIfAbsent(jobName, k -> new CopyOnWriteArrayList<>()).add(future); + log.debug("Registered watcher for job [%s]. Total watchers: %d", jobName, jobWatchers.get(jobName).size()); + return future; + } + + /** + * Register to be notified when a pod for the given job name changes. + * The returned future will complete when a pod with the job-name label changes. + * + * @param jobName The job-name label value to watch for + * @return A future that completes when a matching pod changes + */ + public CompletableFuture waitForPodChange(String jobName) + { + CompletableFuture future = new CompletableFuture<>(); + podWatchers.computeIfAbsent(jobName, k -> new CopyOnWriteArrayList<>()).add(future); + log.debug("Registered watcher for pod with job-name [%s]. Total watchers: %d", jobName, podWatchers.get(jobName).size()); + return future; + } + + /** + * Notify all watchers that a job with the given name has changed. + * Completes all pending futures for this job and clears the watcher list. + * + * @param jobName The name of the job that changed + */ + public void notifyJobChange(String jobName, Job job) + { + List> futures = jobWatchers.get(jobName); + if (futures != null && !futures.isEmpty()) { + log.debug("Notifying %d watchers of job [%s] change", futures.size(), jobName); + futures.forEach(f -> f.complete(job)); + futures.clear(); + } + } + + /** + * Notify all watchers that a pod for the given job name has changed. + * Completes all pending futures for pods with this job-name label and clears the watcher list. + * + * @param jobName The job-name label value that changed + */ + public void notifyPodChange(String jobName, Pod pod) + { + List> futures = podWatchers.get(jobName); + if (futures != null && !futures.isEmpty()) { + log.debug("Notifying %d watchers of pod change for job-name [%s]", futures.size(), jobName); + futures.forEach(f -> f.complete(pod)); + futures.clear(); + } + } + + /** + * Cancel all pending watchers. Used during shutdown. + */ + public void cancelAll() + { + log.info("Cancelling all pending watchers"); + jobWatchers.values().forEach(futures -> futures.forEach(f -> f.cancel(true))); + podWatchers.values().forEach(futures -> futures.forEach(f -> f.cancel(true))); + jobWatchers.clear(); + podWatchers.clear(); + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index db23c2b73a4c..e4f8c056eae7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -25,210 +25,667 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; import io.fabric8.kubernetes.client.informers.SharedIndexInformer; -import io.fabric8.kubernetes.client.informers.cache.Indexer; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.easymock.EasyMock; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; @EnableKubernetesMockClient(crud = true) public class CachingKubernetesPeonClientTest { - private static final String ID = "id"; - private static final String TASK_NAME_PREFIX = ""; - private static final String JOB_NAME = ID; - private static final String KUBERNETES_JOB_NAME = KubernetesOverlordUtils.convertTaskIdToJobName(JOB_NAME); - private static final String POD_NAME = "name"; - private static final String NAMESPACE = "namespace"; + private static final String NAMESPACE = "test-namespace"; + private static final String OVERLORD_NAMESPACE = "overlord-test"; + private static final String JOB_NAME = "test-job-abc123"; + private static final String POD_NAME = "test-job-abc123-pod"; private KubernetesClient client; private KubernetesMockServer server; - private KubernetesClientApi clientApi; - private CachingKubernetesPeonClient instance; + private TestKubernetesClientApi clientApi; + private CachingKubernetesPeonClient peonClient; private StubServiceEmitter serviceEmitter; @BeforeEach - public void setup() + public void setup() throws Exception { serviceEmitter = new StubServiceEmitter("service", "host"); + + // Set up real informers with the mock client + clientApi = new TestKubernetesClientApi(client, NAMESPACE); + clientApi.start(); + + peonClient = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); } - @Test - void test_getPeonPod_withPodInCache_returnsPresentOptional() + @AfterEach + public void teardown() { - // Create mocks - SharedIndexInformer podInformer = EasyMock.createMock(SharedIndexInformer.class); - Indexer indexer = EasyMock.createMock(Indexer.class); + if (clientApi != null) { + clientApi.stop(); + } + } + @Test + public void test_getPeonPod_withPodInCache_returnsPresentOptional() throws Exception + { + // Create pod in mock server Pod pod = new PodBuilder() .withNewMetadata() .withName(POD_NAME) - .addToLabels("job-name", KUBERNETES_JOB_NAME) + .withNamespace(NAMESPACE) + .addToLabels("job-name", JOB_NAME) .endMetadata() + .withNewStatus() + .withPodIP("10.0.0.1") + .endStatus() .build(); - // Set up expectations - EasyMock.expect(podInformer.getIndexer()).andReturn(indexer); - EasyMock.expect(indexer.byIndex("byJobName", KUBERNETES_JOB_NAME)) - .andReturn(List.of(pod)); - - EasyMock.replay(podInformer, indexer); + client.pods().inNamespace(NAMESPACE).resource(pod).create(); - clientApi = new TestKubernetesClient(this.client, podInformer, null); - instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + // Wait for informer to sync + clientApi.waitForSync(); - Optional result = instance.getPeonPod(KUBERNETES_JOB_NAME); + // Query from cache + Optional result = peonClient.getPeonPod(JOB_NAME); Assertions.assertTrue(result.isPresent()); Assertions.assertEquals(POD_NAME, result.get().getMetadata().getName()); - - EasyMock.verify(podInformer, indexer); } @Test - void test_getPeonPod_withoutPodInCache_returnsAbsentOptional() + public void test_getPeonPod_withoutPodInCache_returnsAbsentOptional() throws Exception { - SharedIndexInformer podInformer = EasyMock.createMock(SharedIndexInformer.class); - Indexer indexer = EasyMock.createMock(Indexer.class); - - EasyMock.expect(podInformer.getIndexer()).andReturn(indexer); - EasyMock.expect(indexer.byIndex("byJobName", KUBERNETES_JOB_NAME)) - .andReturn(Collections.emptyList()); + // Wait for informer to sync (empty cache) + clientApi.waitForSync(); - EasyMock.replay(podInformer, indexer); - - clientApi = new TestKubernetesClient(this.client, podInformer, null); - instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); - - Optional result = instance.getPeonPod(KUBERNETES_JOB_NAME); + Optional result = peonClient.getPeonPod(JOB_NAME); Assertions.assertFalse(result.isPresent()); - - EasyMock.verify(podInformer, indexer); } @Test - void test_getPeonPod_withMultiplePodsForSameJob_returnsFirstOne() + public void test_getPeonPod_withMultiplePodsForSameJob_returnsFirstOne() throws Exception { - SharedIndexInformer podInformer = EasyMock.createMock(SharedIndexInformer.class); - Indexer indexer = EasyMock.createMock(Indexer.class); - Pod pod1 = new PodBuilder() .withNewMetadata() .withName("pod-1") - .addToLabels("job-name", KUBERNETES_JOB_NAME) + .withNamespace(NAMESPACE) + .addToLabels("job-name", JOB_NAME) .endMetadata() .build(); Pod pod2 = new PodBuilder() .withNewMetadata() .withName("pod-2") - .addToLabels("job-name", KUBERNETES_JOB_NAME) + .withNamespace(NAMESPACE) + .addToLabels("job-name", JOB_NAME) .endMetadata() .build(); - EasyMock.expect(podInformer.getIndexer()).andReturn(indexer); - EasyMock.expect(indexer.byIndex("byJobName", KUBERNETES_JOB_NAME)) - .andReturn(List.of(pod1, pod2)); + client.pods().inNamespace(NAMESPACE).resource(pod1).create(); + client.pods().inNamespace(NAMESPACE).resource(pod2).create(); - EasyMock.replay(podInformer, indexer); + clientApi.waitForSync(); - clientApi = new TestKubernetesClient(this.client, podInformer, null); - instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + Optional result = peonClient.getPeonPod(JOB_NAME); - Optional result = instance.getPeonPod(KUBERNETES_JOB_NAME); + Assertions.assertTrue(result.isPresent()); + // Should return one of them (order may vary) + String podName = result.get().getMetadata().getName(); + Assertions.assertTrue("pod-1".equals(podName) || "pod-2".equals(podName)); + } + + @Test + public void test_getPeonJob_withJobInCache_returnsPresentOptional() throws Exception + { + Job job = new JobBuilder() + .withNewMetadata() + .withName(JOB_NAME) + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); + + clientApi.waitForSync(); + + Optional result = peonClient.getPeonJob(JOB_NAME); Assertions.assertTrue(result.isPresent()); - Assertions.assertEquals("pod-1", result.get().getMetadata().getName()); + Assertions.assertEquals(JOB_NAME, result.get().getMetadata().getName()); + } + + @Test + public void test_getPeonJob_withoutJobInCache_returnsAbsentOptional() throws Exception + { + clientApi.waitForSync(); + + Optional result = peonClient.getPeonJob(JOB_NAME); - EasyMock.verify(podInformer, indexer); + Assertions.assertFalse(result.isPresent()); } @Test - void test_getPeonJobs_withoutOverlordNamespace_returnsAllJobsFromCache() + public void test_getPeonJobs_withoutOverlordNamespace_returnsAllJobsFromCache() throws Exception { - SharedIndexInformer jobInformer = EasyMock.createMock(SharedIndexInformer.class); - Indexer indexer = EasyMock.createMock(Indexer.class); + Job job1 = new JobBuilder() + .withNewMetadata() + .withName("job-1") + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .build(); - Job job = new JobBuilder() + Job job2 = new JobBuilder() .withNewMetadata() - .withName(KUBERNETES_JOB_NAME) + .withName("job-2") + .withNamespace(NAMESPACE) .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .endMetadata() .build(); - EasyMock.expect(jobInformer.getIndexer()).andReturn(indexer); - EasyMock.expect(indexer.list()).andReturn(List.of(job)); + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job1).create(); + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job2).create(); - EasyMock.replay(jobInformer, indexer); + clientApi.waitForSync(); - clientApi = new TestKubernetesClient(this.client, null, jobInformer); - instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + List jobs = peonClient.getPeonJobs(); - List jobs = instance.getPeonJobs(); + Assertions.assertEquals(2, jobs.size()); + } - Assertions.assertEquals(1, jobs.size()); - Assertions.assertEquals(KUBERNETES_JOB_NAME, jobs.get(0).getMetadata().getName()); + @Test + public void test_getPeonJobs_withOverlordNamespace_returnsFilteredJobs() throws Exception + { + peonClient = new CachingKubernetesPeonClient(clientApi, NAMESPACE, OVERLORD_NAMESPACE, false, serviceEmitter); + + Job matchingJob = new JobBuilder() + .withNewMetadata() + .withName("matching-job") + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .addToLabels(DruidK8sConstants.OVERLORD_NAMESPACE_KEY, OVERLORD_NAMESPACE) + .endMetadata() + .build(); + + Job nonMatchingJob = new JobBuilder() + .withNewMetadata() + .withName("non-matching-job") + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .addToLabels(DruidK8sConstants.OVERLORD_NAMESPACE_KEY, "other-namespace") + .endMetadata() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(matchingJob).create(); + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(nonMatchingJob).create(); + + clientApi.waitForSync(); - EasyMock.verify(jobInformer, indexer); + List jobs = peonClient.getPeonJobs(); + + Assertions.assertEquals(1, jobs.size()); + Assertions.assertEquals("matching-job", jobs.get(0).getMetadata().getName()); } @Test - void test_getPeonJobs_withOverlordNamespace_returnsFilteredJobs() + public void test_getPeonJobs_whenCacheEmpty_returnsEmptyList() throws Exception { - SharedIndexInformer jobInformer = EasyMock.createMock(SharedIndexInformer.class); - Indexer indexer = EasyMock.createMock(Indexer.class); + clientApi.waitForSync(); + + List jobs = peonClient.getPeonJobs(); + + Assertions.assertEquals(0, jobs.size()); + } + @Test + public void test_waitForPeonJobCompletion_jobSucceeds() throws Exception + { + // Create job in running state + K8sTaskId taskId = new K8sTaskId("", "original-task-id"); Job job = new JobBuilder() .withNewMetadata() - .withName(KUBERNETES_JOB_NAME) + .withName(taskId.getK8sJobName()) + .withNamespace(NAMESPACE) .addToLabels(DruidK8sConstants.LABEL_KEY, "true") - .addToLabels(DruidK8sConstants.OVERLORD_NAMESPACE_KEY, "overlord-ns") .endMetadata() + .withNewStatus() + .withActive(1) + .endStatus() .build(); - EasyMock.expect(jobInformer.getIndexer()).andReturn(indexer); - EasyMock.expect(indexer.byIndex("byOverlordNamespace", "overlord-ns")) - .andReturn(List.of(job)); + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); + clientApi.waitForSync(); - EasyMock.replay(jobInformer, indexer); - clientApi = new TestKubernetesClient(this.client, null, jobInformer); - instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, "overlord-ns", false, serviceEmitter); + // Start waiting in background + CompletableFuture futureResponse = CompletableFuture.supplyAsync(() -> + peonClient.waitForPeonJobCompletion(taskId, 60, TimeUnit.SECONDS) + ); - List jobs = instance.getPeonJobs(); + // Give it a moment to start waiting + Thread.sleep(500); - Assertions.assertEquals(1, jobs.size()); - Assertions.assertEquals(KUBERNETES_JOB_NAME, jobs.get(0).getMetadata().getName()); + // Update job to succeeded state + Job succeededJob = new JobBuilder() + .withNewMetadata() + .withName(taskId.getK8sJobName()) + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .withNewStatus() + .withSucceeded(1) + .withActive(0) + .endStatus() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(succeededJob).update(); + + // Wait for response + JobResponse response = futureResponse.get(60, TimeUnit.SECONDS); - EasyMock.verify(jobInformer, indexer); + Assertions.assertEquals(PeonPhase.SUCCEEDED, response.getPhase()); + Assertions.assertNotNull(response.getJob()); } @Test - void test_getPeonJobs_whenCacheEmpty_returnsEmptyList() + public void test_waitUntilPeonPodCreatedAndReady_podBecomesReady() throws Exception { - SharedIndexInformer jobInformer = EasyMock.createMock(SharedIndexInformer.class); - Indexer indexer = EasyMock.createMock(Indexer.class); + // Create pod without IP (not ready) + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .withNamespace(NAMESPACE) + .addToLabels("job-name", JOB_NAME) + .endMetadata() + .withNewStatus() + .endStatus() + .build(); - EasyMock.expect(jobInformer.getIndexer()).andReturn(indexer); - EasyMock.expect(indexer.list()).andReturn(Collections.emptyList()); + client.pods().inNamespace(NAMESPACE).resource(pod).create(); + clientApi.waitForSync(); - EasyMock.replay(jobInformer, indexer); + // Start waiting for pod to be ready in background + CompletableFuture futurePod = CompletableFuture.supplyAsync(() -> + peonClient.waitUntilPeonPodCreatedAndReady(JOB_NAME, 10, TimeUnit.SECONDS) + ); - clientApi = new TestKubernetesClient(this.client, null, jobInformer); - instance = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + // Give it a moment to start waiting + Thread.sleep(500); - List jobs = instance.getPeonJobs(); + // Update pod with IP (becomes ready) + Pod readyPod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .withNamespace(NAMESPACE) + .addToLabels("job-name", JOB_NAME) + .endMetadata() + .withNewStatus() + .withPodIP("10.0.0.1") + .endStatus() + .build(); - Assertions.assertEquals(0, jobs.size()); + client.pods().inNamespace(NAMESPACE).resource(readyPod).update(); - EasyMock.verify(jobInformer, indexer); + // Wait for result + Pod result = futurePod.get(5, TimeUnit.SECONDS); + + Assertions.assertNotNull(result); + Assertions.assertEquals("10.0.0.1", result.getStatus().getPodIP()); + } + + @Test + public void test_waitUntilPeonPodCreatedAndReady_timeoutWhenPodNotReady() throws Exception + { + // Create pod without IP (never becomes ready) + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .withNamespace(NAMESPACE) + .addToLabels("job-name", JOB_NAME) + .endMetadata() + .withNewStatus() + .endStatus() + .build(); + + client.pods().inNamespace(NAMESPACE).resource(pod).create(); + clientApi.waitForSync(); + + // Wait for pod to be ready with short timeout + Pod result = peonClient.waitUntilPeonPodCreatedAndReady(JOB_NAME, 1, TimeUnit.SECONDS); + + // Should return null on timeout + Assertions.assertNull(result); + } + + @Test + public void test_waitUntilPeonPodCreatedAndReady_throwsWhenPodNeverCreated() throws Exception + { + clientApi.waitForSync(); + + // Wait for pod that is never created with short timeout + Exception exception = Assertions.assertThrows( + RuntimeException.class, + () -> peonClient.waitUntilPeonPodCreatedAndReady(JOB_NAME, 1, TimeUnit.SECONDS) + ); + + // Should throw DruidException about timeout waiting for pod creation + Assertions.assertTrue( + exception.getMessage().contains("Timeout waiting for pod") || + exception.getCause().getMessage().contains("Timeout waiting for pod") + ); + } + + @Test + public void test_waitForPeonJobCompletion_timeoutWhenJobNeverCompletes() throws Exception + { + // Create job that stays in running state + K8sTaskId taskId = new K8sTaskId("", "timeout-task-id"); + Job job = new JobBuilder() + .withNewMetadata() + .withName(taskId.getK8sJobName()) + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .withNewStatus() + .withActive(1) + .endStatus() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); + clientApi.waitForSync(); + + // Wait with short timeout - job never completes + JobResponse response = peonClient.waitForPeonJobCompletion(taskId, 500, TimeUnit.MILLISECONDS); + + // Should return FAILED phase on timeout + Assertions.assertEquals(PeonPhase.FAILED, response.getPhase()); + Assertions.assertNull(response.getJob()); + } + + @Test + public void test_waitForPeonJobCompletion_jobFails() throws Exception + { + // Create job in running state + K8sTaskId taskId = new K8sTaskId("", "failing-task-id"); + Job job = new JobBuilder() + .withNewMetadata() + .withName(taskId.getK8sJobName()) + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .withNewStatus() + .withActive(1) + .endStatus() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); + clientApi.waitForSync(); + + // Start waiting in background + CompletableFuture futureResponse = CompletableFuture.supplyAsync(() -> + peonClient.waitForPeonJobCompletion(taskId, 60, TimeUnit.SECONDS) + ); + + // Give it a moment to start waiting + Thread.sleep(500); + + // Update job to failed state + Job failedJob = new JobBuilder() + .withNewMetadata() + .withName(taskId.getK8sJobName()) + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .withNewStatus() + .withFailed(1) + .withActive(0) + .endStatus() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(failedJob).update(); + + // Wait for response + JobResponse response = futureResponse.get(60, TimeUnit.SECONDS); + + Assertions.assertEquals(PeonPhase.FAILED, response.getPhase()); + Assertions.assertNotNull(response.getJob()); + } + + /** + * Test implementation of KubernetesClientApi that uses real informers with the mock server + */ + private static class TestKubernetesClientApi implements KubernetesClientApi + { + private final KubernetesClient client; + private final SharedIndexInformer podInformer; + private final SharedIndexInformer jobInformer; + private final KubernetesResourceEventNotifier eventNotifier; + private final CountDownLatch syncLatch; + + public TestKubernetesClientApi(KubernetesClient client, String namespace) + { + this.client = client; + this.eventNotifier = new KubernetesResourceEventNotifier(); + this.syncLatch = new CountDownLatch(2); // Wait for both informers + + // Set up pod informer with real event handlers + this.podInformer = client.pods() + .inNamespace(namespace) + .inform(new ResourceEventHandler() + { + @Override + public void onAdd(Pod pod) + { + notifyPodChange(pod); + } + + @Override + public void onUpdate(Pod oldPod, Pod newPod) + { + notifyPodChange(newPod); + } + + @Override + public void onDelete(Pod pod, boolean deletedFinalStateUnknown) + { + notifyPodChange(pod); + } + }, 1000L); + + // Add pod indexer + Map>> podIndexers = new HashMap<>(); + podIndexers.put("byJobName", pod -> { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + return Collections.singletonList(jobName); + } + } + return Collections.emptyList(); + }); + podInformer.addIndexers(podIndexers); + + // Set up job informer with real event handlers + this.jobInformer = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .inform(new ResourceEventHandler() + { + @Override + public void onAdd(Job job) + { + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + + @Override + public void onUpdate(Job oldJob, Job newJob) + { + eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); + } + + @Override + public void onDelete(Job job, boolean deletedFinalStateUnknown) + { + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + }, 1000L); + + // Add job indexers + Map>> jobIndexers = new HashMap<>(); + jobIndexers.put("byJobName", job -> { + if (job.getMetadata() != null && job.getMetadata().getName() != null) { + return Collections.singletonList(job.getMetadata().getName()); + } + return Collections.emptyList(); + }); + jobIndexers.put("byOverlordNamespace", job -> { + if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { + String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); + if (overlordNamespace != null) { + return Collections.singletonList(overlordNamespace); + } + } + return Collections.emptyList(); + }); + jobInformer.addIndexers(jobIndexers); + } + + public void start() + { + // Add ready callbacks to count down latch + podInformer.addEventHandlerWithResyncPeriod(new ResourceEventHandler() + { + @Override + public void onAdd(Pod obj) + { + + } + @Override + public void onUpdate(Pod oldObj, Pod newObj) + { + + } + @Override + public void onDelete(Pod obj, boolean deletedFinalStateUnknown) + { + + } + }, 1000L); + + jobInformer.addEventHandlerWithResyncPeriod(new ResourceEventHandler() + { + @Override + public void onAdd(Job obj) + { + + } + @Override + public void onUpdate(Job oldObj, Job newObj) + { + + } + @Override + public void onDelete(Job obj, boolean deletedFinalStateUnknown) + { + + } + }, 1000L); + + podInformer.run(); + jobInformer.run(); + + // Count down after starting + syncLatch.countDown(); + syncLatch.countDown(); + } + + public void stop() + { + if (podInformer != null) { + podInformer.stop(); + } + if (jobInformer != null) { + jobInformer.stop(); + } + if (eventNotifier != null) { + eventNotifier.cancelAll(); + } + } + + public void waitForSync() throws InterruptedException + { + syncLatch.await(5, TimeUnit.SECONDS); + // Give informers a bit more time to process + Thread.sleep(200); + } + + private void notifyPodChange(Pod pod) + { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + eventNotifier.notifyPodChange(jobName, pod); + } + } + } + + @Override + public T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException + { + return executor.executeRequest(client); + } + + @Override + public T executePodCacheRequest(KubernetesInformerExecutor executor) + { + return executor.executeRequest(podInformer); + } + + @Override + public T executeJobCacheRequest(KubernetesInformerExecutor executor) + { + return executor.executeRequest(jobInformer); + } + + @Override + public KubernetesClient getClient() + { + return client; + } + + @Override + public SharedIndexInformer getPodInformer() + { + return podInformer; + } + + @Override + public SharedIndexInformer getJobInformer() + { + return jobInformer; + } + + @Override + public KubernetesResourceEventNotifier getEventNotifier() + { + return eventNotifier; + } } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java new file mode 100644 index 000000000000..09dec764e4ff --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java @@ -0,0 +1,313 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class KubernetesResourceEventNotifierTest +{ + private KubernetesResourceEventNotifier notifier; + + @BeforeEach + public void setUp() + { + notifier = new KubernetesResourceEventNotifier(); + } + + @AfterEach + public void tearDown() + { + notifier.cancelAll(); + } + + @Test + public void testWaitForJobChange_CompletesOnNotification() throws Exception + { + String jobName = "test-job"; + Job mockJob = createMockJob(jobName); + + CompletableFuture future = notifier.waitForJobChange(jobName); + assertFalse(future.isDone()); + + notifier.notifyJobChange(jobName, mockJob); + + Job result = future.get(1, TimeUnit.SECONDS); + assertSame(mockJob, result); + assertTrue(future.isDone()); + } + + @Test + public void testWaitForPodChange_CompletesOnNotification() throws Exception + { + String jobName = "test-job"; + Pod mockPod = createMockPod(jobName); + + CompletableFuture future = notifier.waitForPodChange(jobName); + assertFalse(future.isDone()); + + notifier.notifyPodChange(jobName, mockPod); + + Pod result = future.get(1, TimeUnit.SECONDS); + assertSame(mockPod, result); + assertTrue(future.isDone()); + } + + @Test + public void testMultipleWatchers_AllNotified() throws Exception + { + String jobName = "test-job"; + Job mockJob = createMockJob(jobName); + + CompletableFuture future1 = notifier.waitForJobChange(jobName); + CompletableFuture future2 = notifier.waitForJobChange(jobName); + CompletableFuture future3 = notifier.waitForJobChange(jobName); + + assertFalse(future1.isDone()); + assertFalse(future2.isDone()); + assertFalse(future3.isDone()); + + notifier.notifyJobChange(jobName, mockJob); + + Job result1 = future1.get(1, TimeUnit.SECONDS); + Job result2 = future2.get(1, TimeUnit.SECONDS); + Job result3 = future3.get(1, TimeUnit.SECONDS); + + assertSame(mockJob, result1); + assertSame(mockJob, result2); + assertSame(mockJob, result3); + } + + @Test + public void testMultiplePodWatchers_AllNotified() throws Exception + { + String jobName = "test-job"; + Pod mockPod = createMockPod(jobName); + + CompletableFuture future1 = notifier.waitForPodChange(jobName); + CompletableFuture future2 = notifier.waitForPodChange(jobName); + CompletableFuture future3 = notifier.waitForPodChange(jobName); + + assertFalse(future1.isDone()); + assertFalse(future2.isDone()); + assertFalse(future3.isDone()); + + notifier.notifyPodChange(jobName, mockPod); + + Pod result1 = future1.get(1, TimeUnit.SECONDS); + Pod result2 = future2.get(1, TimeUnit.SECONDS); + Pod result3 = future3.get(1, TimeUnit.SECONDS); + + assertSame(mockPod, result1); + assertSame(mockPod, result2); + assertSame(mockPod, result3); + } + + @Test + public void testNotifyWithoutWatchers_NoException() + { + String jobName = "test-job"; + Job mockJob = createMockJob(jobName); + + // Should not throw exception + notifier.notifyJobChange(jobName, mockJob); + notifier.notifyPodChange(jobName, createMockPod(jobName)); + } + + @Test + public void testDifferentJobNames_IndependentNotifications() throws Exception + { + String jobName1 = "job-1"; + String jobName2 = "job-2"; + Job mockJob1 = createMockJob(jobName1); + Job mockJob2 = createMockJob(jobName2); + + CompletableFuture future1 = notifier.waitForJobChange(jobName1); + CompletableFuture future2 = notifier.waitForJobChange(jobName2); + + notifier.notifyJobChange(jobName1, mockJob1); + + Job result1 = future1.get(1, TimeUnit.SECONDS); + assertSame(mockJob1, result1); + assertFalse(future2.isDone()); + + notifier.notifyJobChange(jobName2, mockJob2); + + Job result2 = future2.get(1, TimeUnit.SECONDS); + assertSame(mockJob2, result2); + } + + @Test + public void testDifferentPodJobNames_IndependentNotifications() throws Exception + { + String jobName1 = "job-1"; + String jobName2 = "job-2"; + Pod mockPod1 = createMockPod(jobName1); + Pod mockPod2 = createMockPod(jobName2); + + CompletableFuture future1 = notifier.waitForPodChange(jobName1); + CompletableFuture future2 = notifier.waitForPodChange(jobName2); + + notifier.notifyPodChange(jobName1, mockPod1); + + Pod result1 = future1.get(1, TimeUnit.SECONDS); + assertSame(mockPod1, result1); + assertFalse(future2.isDone()); + + notifier.notifyPodChange(jobName2, mockPod2); + + Pod result2 = future2.get(1, TimeUnit.SECONDS); + assertSame(mockPod2, result2); + } + + @Test + public void testCancelAll_CancelsAllPendingWatchers() + { + String jobName1 = "job-1"; + String jobName2 = "job-2"; + + CompletableFuture jobFuture1 = notifier.waitForJobChange(jobName1); + CompletableFuture jobFuture2 = notifier.waitForJobChange(jobName2); + CompletableFuture podFuture1 = notifier.waitForPodChange(jobName1); + CompletableFuture podFuture2 = notifier.waitForPodChange(jobName2); + + assertFalse(jobFuture1.isDone()); + assertFalse(jobFuture2.isDone()); + assertFalse(podFuture1.isDone()); + assertFalse(podFuture2.isDone()); + + notifier.cancelAll(); + + assertTrue(jobFuture1.isCancelled()); + assertTrue(jobFuture2.isCancelled()); + assertTrue(podFuture1.isCancelled()); + assertTrue(podFuture2.isCancelled()); + } + + @Test + public void testCancelAll_CancelledFuturesThrowException() + { + String jobName = "test-job"; + + CompletableFuture future = notifier.waitForJobChange(jobName); + notifier.cancelAll(); + + assertThrows(CancellationException.class, future::get); + } + + @Test + public void testSequentialNotifications_WatchersAreCleared() throws Exception + { + String jobName = "test-job"; + Job mockJob1 = createMockJob(jobName); + Job mockJob2 = createMockJob(jobName); + + // First notification + CompletableFuture future1 = notifier.waitForJobChange(jobName); + notifier.notifyJobChange(jobName, mockJob1); + Job result1 = future1.get(1, TimeUnit.SECONDS); + assertSame(mockJob1, result1); + + // Second notification - should require new watcher + CompletableFuture future2 = notifier.waitForJobChange(jobName); + assertFalse(future2.isDone()); + notifier.notifyJobChange(jobName, mockJob2); + Job result2 = future2.get(1, TimeUnit.SECONDS); + assertSame(mockJob2, result2); + } + + @Test + public void testJobAndPodWatchers_Independent() throws Exception + { + String jobName = "test-job"; + Job mockJob = createMockJob(jobName); + Pod mockPod = createMockPod(jobName); + + CompletableFuture jobFuture = notifier.waitForJobChange(jobName); + CompletableFuture podFuture = notifier.waitForPodChange(jobName); + + // Notify job change - should not affect pod watcher + notifier.notifyJobChange(jobName, mockJob); + Job jobResult = jobFuture.get(1, TimeUnit.SECONDS); + assertSame(mockJob, jobResult); + assertFalse(podFuture.isDone()); + + // Notify pod change + notifier.notifyPodChange(jobName, mockPod); + Pod podResult = podFuture.get(1, TimeUnit.SECONDS); + assertSame(mockPod, podResult); + } + + @Test + public void testWaitForJobChange_ReturnsNewFutureEachTime() + { + String jobName = "test-job"; + + CompletableFuture future1 = notifier.waitForJobChange(jobName); + CompletableFuture future2 = notifier.waitForJobChange(jobName); + + // Should be different future instances + assertNotEquals(future1, future2); + } + + @Test + public void testWaitForPodChange_ReturnsNewFutureEachTime() + { + String jobName = "test-job"; + + CompletableFuture future1 = notifier.waitForPodChange(jobName); + CompletableFuture future2 = notifier.waitForPodChange(jobName); + + // Should be different future instances + assertNotEquals(future1, future2); + } + + private Job createMockJob(String name) + { + Job job = new Job(); + ObjectMeta metadata = new ObjectMeta(); + metadata.setName(name); + job.setMetadata(metadata); + return job; + } + + private Pod createMockPod(String jobName) + { + Pod pod = new Pod(); + ObjectMeta metadata = new ObjectMeta(); + metadata.setName(jobName + "-pod"); + pod.setMetadata(metadata); + return pod; + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java index 4cb2a97432e9..1c670afef9d3 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -76,4 +76,10 @@ public SharedIndexInformer getJobInformer() { return jobInformer; } + + @Override + public KubernetesResourceEventNotifier getEventNotifier() + { + return null; + } } From 37788831f29e5e528c9044f2a1b01e70006f98bd Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 8 Oct 2025 10:29:48 -0500 Subject: [PATCH 06/58] some config and instantion cleanup along with basic docs --- docs/development/extensions-core/k8s-jobs.md | 16 +++++ .../overlord/KubernetesOverlordModule.java | 5 +- .../overlord/KubernetesTaskRunnerConfig.java | 63 ++++++++++++------- .../overlord/KubernetesTaskRunnerFactory.java | 2 +- .../common/DruidKubernetesClient.java | 17 +++-- .../KubernetesTaskRunnerFactoryTest.java | 2 +- .../DruidPeonClientIntegrationTest.java | 2 +- 7 files changed, 76 insertions(+), 31 deletions(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index 4d75a65ba6ff..307e833f2f68 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -33,6 +33,20 @@ Consider this an [EXPERIMENTAL](../experimental.md) feature mostly because it ha The K8s extension builds a pod spec for each task using the specified pod adapter. All jobs are natively restorable, they are decoupled from the Druid deployment, thus restarting pods or doing upgrades has no effect on tasks in flight. They will continue to run and when the overlord comes back up it will start tracking them again. +## Kubernetes Client Mode + +### "Direct" K8s API Interaction per task *(Default)* + +Task lifecycle code in Druid talks directly to the Kubernetes API server for all operations that require interaction with the Kubernetes cluster. + +### `SharedInformer` "Caching" *(Experimental)* + +Enabled by setting `druid.indexer.runner.k8s.useSharedInformer=true`, this mode uses Fabric8 `SharedInformer` objects for monitoring state changes in the remote K8s cluster, reducing the number of direct API calls to the Kubernetes API server. This can greatly reduce load on the API server, especially in environments with a high volume of tasks. + +This mode is experimental and should be used with caution in production until it has been vetted more thoroughly by the community. + +The core idea is to use two SharedInformers—one for jobs and one for pods—to watch for changes in the remote K8s cluster. These informers maintain a local cache of jobs and pods that tasks can query. The informers can also notify listeners when changes occur, allowing tasks to react to state changes without polling the API server or creating per-task watches on the K8s cluster. + ## Configuration @@ -793,6 +807,8 @@ Should you require the needed permissions for interacting across Kubernetes name | `druid.indexer.runner.capacity` | `Integer` | Number of concurrent jobs that can be sent to Kubernetes. | `2147483647` | No | | `druid.indexer.runner.cpuCoreInMicro` | `Integer` | Number of CPU micro core for the task. | `1000` | No | | `druid.indexer.runner.logSaveTimeout` | `Duration` | How long to wait for task logs to be saved before giving up. | `PT300S` | NO | +| `druid.indexer.runner.enableKubernetesClientSharedInformers` | `boolean` | Whether to use shared informers to watch for pod/job changes. This is more efficient on the Kubernetes API server, but may use more memory in the Overlord. | `false` | No | +| `druid.indexer.runner.kubernetesClientInformerResyncPeriod` | `Duration` | When using shared informers, controls how frequently the informers resync with the Kubernetes API server. | `PT30S` | No | ### Metrics added diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index 5f32005f0c0f..b330ea2a23e9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -135,7 +135,10 @@ public DruidKubernetesClient makeKubernetesClient( config.setNamespace(kubernetesTaskRunnerConfig.getNamespace()); - client = new DruidKubernetesClient(httpClientConfig, config, kubernetesTaskRunnerConfig.isEnablePeonClientCache()); + config.setAdditionalProperty(DruidKubernetesClient.ENABLE_INFORMERS_KEY, kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers()); + config.setAdditionalProperty(DruidKubernetesClient.INFORMER_RESYNC_PERIOD_MS_KEY, kubernetesTaskRunnerConfig.getKubernetesClientInformerResyncPeriod()); + + client = new DruidKubernetesClient(httpClientConfig, config); lifecycle.addHandler( new Lifecycle.Handler() diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index 26f5e51aa6eb..df70487ac8b3 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -82,7 +82,10 @@ public class KubernetesTaskRunnerConfig @JsonProperty // enable using kubernetes informer cache for peon client operations - private boolean enableKubernetesClientCaching = false; + private boolean enableKubernetesClientSharedInformers = false; + + @JsonProperty + private Period kubernetesClientInformerResyncPeriod = new Period("PT30S"); @JsonProperty @NotNull @@ -157,7 +160,6 @@ private KubernetesTaskRunnerConfig( String kubexitImage, Long graceTerminationPeriodSeconds, boolean disableClientProxy, - boolean enableKubernetesClientCaching, Period maxTaskDuration, Period taskCleanupDelay, Period taskCleanupInterval, @@ -169,7 +171,9 @@ private KubernetesTaskRunnerConfig( Map labels, Map annotations, Integer capacity, - Period taskJoinTimeout + Period taskJoinTimeout, + boolean enableKubernetesClientSharedInformers, + Period kubernetesClientInformerResyncPeriod ) { this.namespace = namespace; @@ -199,10 +203,6 @@ private KubernetesTaskRunnerConfig( this.graceTerminationPeriodSeconds ); this.disableClientProxy = disableClientProxy; - this.enableKubernetesClientCaching = ObjectUtils.defaultIfNull( - enableKubernetesClientCaching, - this.enableKubernetesClientCaching - ); this.maxTaskDuration = ObjectUtils.defaultIfNull( maxTaskDuration, this.maxTaskDuration @@ -251,6 +251,14 @@ private KubernetesTaskRunnerConfig( capacity, this.capacity ); + this.enableKubernetesClientSharedInformers = ObjectUtils.defaultIfNull( + enableKubernetesClientSharedInformers, + this.enableKubernetesClientSharedInformers + ); + this.kubernetesClientInformerResyncPeriod = ObjectUtils.defaultIfNull( + kubernetesClientInformerResyncPeriod, + this.kubernetesClientInformerResyncPeriod + ); } public String getNamespace() @@ -299,11 +307,6 @@ public boolean isDisableClientProxy() return disableClientProxy; } - public boolean isEnablePeonClientCache() - { - return enableKubernetesClientCaching; - } - public Period getTaskTimeout() { return maxTaskDuration; @@ -365,6 +368,16 @@ public Integer getCapacity() return capacity; } + public boolean isEnableKubernetesClientSharedInformers() + { + return enableKubernetesClientSharedInformers; + } + + public Period getKubernetesClientInformerResyncPeriod() + { + return kubernetesClientInformerResyncPeriod; + } + public static Builder builder() { return new Builder(); @@ -381,7 +394,6 @@ public static class Builder private String kubexitImage; private Long graceTerminationPeriodSeconds; private boolean disableClientProxy; - private boolean enableKubernetesClientCaching; private Period maxTaskDuration; private Period taskCleanupDelay; private Period taskCleanupInterval; @@ -394,6 +406,8 @@ public static class Builder private Integer capacity; private Period taskJoinTimeout; private Period logSaveTimeout; + private boolean enableKubernetesClientCaching; + private Period kubernetesClientInformerResyncPeriod; public Builder() { @@ -453,12 +467,6 @@ public Builder withDisableClientProxy(boolean disableClientProxy) return this; } - public Builder withEnablePeonClientCache(boolean enableKubernetesClientCaching) - { - this.enableKubernetesClientCaching = enableKubernetesClientCaching; - return this; - } - public Builder withTaskTimeout(Period taskTimeout) { this.maxTaskDuration = taskTimeout; @@ -532,6 +540,18 @@ public Builder withLogSaveTimeout(Period logSaveTimeout) return this; } + public Builder withEnablePeonClientCache(boolean enableKubernetesClientCaching) + { + this.enableKubernetesClientCaching = enableKubernetesClientCaching; + return this; + } + + public Builder withKubernetesClientInformerResyncPeriod(Period kubernetesClientInformerResyncPeriod) + { + this.kubernetesClientInformerResyncPeriod = kubernetesClientInformerResyncPeriod; + return this; + } + public KubernetesTaskRunnerConfig build() { return new KubernetesTaskRunnerConfig( @@ -544,7 +564,6 @@ public KubernetesTaskRunnerConfig build() this.kubexitImage, this.graceTerminationPeriodSeconds, this.disableClientProxy, - this.enableKubernetesClientCaching, this.maxTaskDuration, this.taskCleanupDelay, this.taskCleanupInterval, @@ -556,7 +575,9 @@ public KubernetesTaskRunnerConfig build() this.labels, this.annotations, this.capacity, - this.taskJoinTimeout + this.taskJoinTimeout, + this.enableKubernetesClientCaching, + this.kubernetesClientInformerResyncPeriod ); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index dbb58b325c4a..b660dcc27247 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -73,7 +73,7 @@ public KubernetesTaskRunnerFactory( public KubernetesTaskRunner build() { AbstractKubernetesPeonClient peonClient; - boolean enableCache = kubernetesTaskRunnerConfig.isEnablePeonClientCache(); + boolean enableCache = kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers(); boolean useOverlordNamespace = adapterTypeAllowingTasksInDifferentNamespaces.contains(taskAdapter.getAdapterType()); if (enableCache) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index f02da1517da6..87bcfd975e40 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -39,24 +39,29 @@ public class DruidKubernetesClient implements KubernetesClientApi { private static final EmittingLogger log = new EmittingLogger(DruidKubernetesClient.class); - private static final long INFORMER_RESYNC_PERIOD_MS = 30 * 1000L; // todo make this configurable by operator + public static final String ENABLE_INFORMERS_KEY = "druid.k8s.informers.enabled"; + public static final String INFORMER_RESYNC_PERIOD_MS_KEY = "druid.k8s.informers.resyncPeriodMs"; + private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 3000L; private final KubernetesClient kubernetesClient; private final SharedIndexInformer podInformer; private final SharedIndexInformer jobInformer; private final KubernetesResourceEventNotifier eventNotifier; + private final long informerResyncPeriodMillis; public DruidKubernetesClient( DruidKubernetesHttpClientConfig httpClientConfig, - Config kubernetesClientConfig, - boolean enableCache + Config kubernetesClientConfig ) { this.kubernetesClient = new KubernetesClientBuilder() .withHttpClientFactory(new DruidKubernetesHttpClientFactory(httpClientConfig)) .withConfig(kubernetesClientConfig) .build(); - if (enableCache) { + + informerResyncPeriodMillis = (long) kubernetesClientConfig + .getAdditionalProperties().getOrDefault(INFORMER_RESYNC_PERIOD_MS_KEY, DEFAULT_INFORMER_RESYNC_PERIOD_MS); + if ((boolean) kubernetesClientConfig.getAdditionalProperties().getOrDefault(ENABLE_INFORMERS_KEY, false)) { this.eventNotifier = new KubernetesResourceEventNotifier(); this.podInformer = setupPodInformer(kubernetesClient.getNamespace()); this.jobInformer = setupJobInformer(kubernetesClient.getNamespace()); @@ -150,7 +155,7 @@ public void onDelete(Pod pod, boolean deletedFinalStateUnknown) log.debug("Pod[%s] got deleted", pod.getMetadata().getName()); notifyPodChange(pod); } - }, INFORMER_RESYNC_PERIOD_MS + }, informerResyncPeriodMillis ); Function> jobNameIndexer = pod -> { @@ -201,7 +206,7 @@ public void onDelete(Job job, boolean deletedFinalStateUnknown) log.debug("Job[%s] got deleted", job.getMetadata().getName()); eventNotifier.notifyJobChange(job.getMetadata().getName(), job); } - }, INFORMER_RESYNC_PERIOD_MS + }, informerResyncPeriodMillis ); Function> overlordNamespaceIndexer = job -> { diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index c81cdb0a45d9..49720642216f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -57,7 +57,7 @@ public void setup() .build(); taskLogs = new NoopTaskLogs(); druidKubernetesClient = - new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build(), false); + new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build()); taskAdapter = new TestTaskAdapter(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 5220eeadd13c..d1796868dc6b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -88,7 +88,7 @@ public void setup() new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), new NamedType(IndexTask.IndexTuningConfig.class, "index") ); - k8sClient = new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build(), false); + k8sClient = new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build()); peonClient = new DirectKubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); druidNode = new DruidNode( "test", From 5dcc64dd5056b00bf2463cac4cdfc7537fed3a7d Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 8 Oct 2025 12:40:36 -0500 Subject: [PATCH 07/58] extnd the K8s task runner docker test to run with both direct and caching mode for the k8s client --- .../k8s/KubernetesTaskRunnerDockerTest.java | 41 +++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDockerTest.java index 014186cdd2fc..dfdd8b4c457a 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDockerTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDockerTest.java @@ -27,12 +27,17 @@ import org.junit.jupiter.api.BeforeEach; /** - * Runs some basic ingestion tests against latest image Druid containers running - * on a K3s cluster with druid-operator and using {@code k8s} task runner type. + * Base class for Kubernetes task runner tests. Subclasses configure whether to use + * SharedInformers for caching. */ -public class KubernetesTaskRunnerDockerTest extends IngestionSmokeTest implements LatestImageDockerTest +abstract class BaseKubernetesTaskRunnerDockerTest extends IngestionSmokeTest implements LatestImageDockerTest { - private static final String MANIFEST_TEMPLATE = "manifests/druid-service-with-operator.yaml"; + protected static final String MANIFEST_TEMPLATE = "manifests/druid-service-with-operator.yaml"; + + /** + * Subclasses override to enable/disable SharedInformer caching. + */ + protected abstract boolean useSharedInformers(); @Override protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) @@ -45,6 +50,7 @@ protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) .addProperty("druid.indexer.runner.type", "k8s") .addProperty("druid.indexer.runner.namespace", "druid") .addProperty("druid.indexer.runner.capacity", "4") + .addProperty("druid.indexer.runner.enableKubernetesClientSharedInformers", String.valueOf(useSharedInformers())) .usingPort(30090); final K3sClusterResource k3sCluster = new K3sClusterWithOperatorResource() @@ -84,3 +90,30 @@ public void verifyOverlordLeader() ); } } + +/** + * Runs ingestion tests using direct K8s API interaction (default mode). + * Each task makes direct API calls to the Kubernetes API server. + */ +class KubernetesTaskRunnerDirectModeDockerTest extends BaseKubernetesTaskRunnerDockerTest +{ + @Override + protected boolean useSharedInformers() + { + return false; + } +} + +/** + * Runs ingestion tests using SharedInformer caching mode. + * Uses Fabric8 SharedInformers to maintain a local cache of Jobs and Pods, + * reducing load on the Kubernetes API server. + */ +class KubernetesTaskRunnerCachingModeDockerTest extends BaseKubernetesTaskRunnerDockerTest +{ + @Override + protected boolean useSharedInformers() + { + return true; + } +} From af306e183c68f6ff8210040783f35597dc3c8427 Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 8 Oct 2025 14:13:09 -0500 Subject: [PATCH 08/58] fix spelling and add resync to dictionary --- docs/development/extensions-core/k8s-jobs.md | 4 ++-- website/.spelling | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index 9926096869f1..cb688c811036 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -41,11 +41,11 @@ Task lifecycle code in Druid talks directly to the Kubernetes API server for all ### `SharedInformer` "Caching" *(Experimental)* -Enabled by setting `druid.indexer.runner.k8s.useSharedInformer=true`, this mode uses Fabric8 `SharedInformer` objects for monitoring state changes in the remote K8s cluster, reducing the number of direct API calls to the Kubernetes API server. This can greatly reduce load on the API server, especially in environments with a high volume of tasks. +Enabled by setting `druid.indexer.runner.k8s.useSharedInformer=true`, this mode uses `Fabric8` `SharedInformer` objects for monitoring state changes in the remote K8s cluster, reducing the number of direct API calls to the Kubernetes API server. This can greatly reduce load on the API server, especially in environments with a high volume of tasks. This mode is experimental and should be used with caution in production until it has been vetted more thoroughly by the community. -The core idea is to use two SharedInformers—one for jobs and one for pods—to watch for changes in the remote K8s cluster. These informers maintain a local cache of jobs and pods that tasks can query. The informers can also notify listeners when changes occur, allowing tasks to react to state changes without polling the API server or creating per-task watches on the K8s cluster. +The core idea is to use two `SharedInformers`, one for jobs and one for pods, to watch for changes in the remote K8s cluster. These informers maintain a local cache of jobs and pods that tasks can query. The informers can also notify listeners when changes occur, allowing tasks to react to state changes without polling the API server or creating per-task watches on the K8s cluster. ## Configuration diff --git a/website/.spelling b/website/.spelling index 767ef4249aba..4e58b1f54bb1 100644 --- a/website/.spelling +++ b/website/.spelling @@ -522,6 +522,7 @@ reindexing reingest reingesting reingestion +resync repo requireSSL rollup From 08764c5efd139c165005b0b792df6ba8dc1fcf40 Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 8 Oct 2025 14:43:06 -0500 Subject: [PATCH 09/58] fix strict compile issues --- ...> BaseKubernetesTaskRunnerDockerTest.java} | 27 --------------- ...rnetesTaskRunnerCachingModeDockerTest.java | 34 +++++++++++++++++++ ...ernetesTaskRunnerDirectModeDockerTest.java | 33 ++++++++++++++++++ 3 files changed, 67 insertions(+), 27 deletions(-) rename embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/{KubernetesTaskRunnerDockerTest.java => BaseKubernetesTaskRunnerDockerTest.java} (85%) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java similarity index 85% rename from embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDockerTest.java rename to embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java index dfdd8b4c457a..9691716049e4 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDockerTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java @@ -90,30 +90,3 @@ public void verifyOverlordLeader() ); } } - -/** - * Runs ingestion tests using direct K8s API interaction (default mode). - * Each task makes direct API calls to the Kubernetes API server. - */ -class KubernetesTaskRunnerDirectModeDockerTest extends BaseKubernetesTaskRunnerDockerTest -{ - @Override - protected boolean useSharedInformers() - { - return false; - } -} - -/** - * Runs ingestion tests using SharedInformer caching mode. - * Uses Fabric8 SharedInformers to maintain a local cache of Jobs and Pods, - * reducing load on the Kubernetes API server. - */ -class KubernetesTaskRunnerCachingModeDockerTest extends BaseKubernetesTaskRunnerDockerTest -{ - @Override - protected boolean useSharedInformers() - { - return true; - } -} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java new file mode 100644 index 000000000000..2a2bbac982d0 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testing.embedded.k8s; + +/** + * Runs ingestion tests using SharedInformer caching mode. + * Uses Fabric8 SharedInformers to maintain a local cache of Jobs and Pods, + * reducing load on the Kubernetes API server. + */ +public class KubernetesTaskRunnerCachingModeDockerTest extends BaseKubernetesTaskRunnerDockerTest +{ + @Override + protected boolean useSharedInformers() + { + return true; + } +} \ No newline at end of file diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java new file mode 100644 index 000000000000..a43d3f091175 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testing.embedded.k8s; + +/** + * Runs ingestion tests using direct K8s API interaction (default mode). + * Each task makes direct API calls to the Kubernetes API server. + */ +public class KubernetesTaskRunnerDirectModeDockerTest extends BaseKubernetesTaskRunnerDockerTest +{ + @Override + protected boolean useSharedInformers() + { + return false; + } +} \ No newline at end of file From a90b2d23215ed984b1c5535554bdb305c96e9d7e Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 8 Oct 2025 15:33:34 -0500 Subject: [PATCH 10/58] fixup checkstyle --- .../embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java | 2 +- .../embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java index 2a2bbac982d0..a23b89172948 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerCachingModeDockerTest.java @@ -31,4 +31,4 @@ protected boolean useSharedInformers() { return true; } -} \ No newline at end of file +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java index a43d3f091175..7142f35bbb3f 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/KubernetesTaskRunnerDirectModeDockerTest.java @@ -30,4 +30,4 @@ protected boolean useSharedInformers() { return false; } -} \ No newline at end of file +} From 0d9672ca9b23597062fc80008eb8298d86fda00a Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 8 Oct 2025 16:19:33 -0500 Subject: [PATCH 11/58] fix k8s overlord module setup --- .../druid/k8s/overlord/KubernetesOverlordModule.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index b330ea2a23e9..f1a14f3b7eab 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -135,8 +135,12 @@ public DruidKubernetesClient makeKubernetesClient( config.setNamespace(kubernetesTaskRunnerConfig.getNamespace()); - config.setAdditionalProperty(DruidKubernetesClient.ENABLE_INFORMERS_KEY, kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers()); - config.setAdditionalProperty(DruidKubernetesClient.INFORMER_RESYNC_PERIOD_MS_KEY, kubernetesTaskRunnerConfig.getKubernetesClientInformerResyncPeriod()); + config.setAdditionalProperty( + DruidKubernetesClient.ENABLE_INFORMERS_KEY, + kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers()); + config.setAdditionalProperty( + DruidKubernetesClient.INFORMER_RESYNC_PERIOD_MS_KEY, + kubernetesTaskRunnerConfig.getKubernetesClientInformerResyncPeriod().toStandardDuration().getMillis()); client = new DruidKubernetesClient(httpClientConfig, config); From 1fe0f8707392bdabcaaf395f0515e3136ffce3d0 Mon Sep 17 00:00:00 2001 From: capistrant Date: Thu, 9 Oct 2025 13:04:48 -0500 Subject: [PATCH 12/58] few small fixups --- .../k8s/overlord/common/CachingKubernetesPeonClient.java | 5 ++++- .../k8s/overlord/common/DirectKubernetesPeonClient.java | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 879266e7a0ad..989526472ab6 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -201,7 +201,10 @@ private boolean isPodReady(Pod pod) */ private JobResponse determineJobResponse(Job job) { - if (job.getStatus().getSucceeded() != null || job.getStatus().getFailed() != null) { + if (job.getStatus() != null && + (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) && + (job.getStatus().getFailed() != null || job.getStatus().getSucceeded() != null)) { + if (job.getStatus().getSucceeded() != null && job.getStatus().getSucceeded() > 0) { log.info("K8s job[%s] completed successfully", job.getMetadata().getName()); return new JobResponse(job, PeonPhase.SUCCEEDED); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java index 3a9a572ad204..a251ad1cb098 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java @@ -46,7 +46,7 @@ */ public class DirectKubernetesPeonClient extends AbstractKubernetesPeonClient { - protected static final EmittingLogger log = new EmittingLogger(CachingKubernetesPeonClient.class); + protected static final EmittingLogger log = new EmittingLogger(DirectKubernetesPeonClient.class); public DirectKubernetesPeonClient( KubernetesClientApi clientApi, From e9b39477db4f45711220a7b47b58edffc6575d04 Mon Sep 17 00:00:00 2001 From: capistrant Date: Thu, 9 Oct 2025 13:12:46 -0500 Subject: [PATCH 13/58] fix checkstyle --- .../druid/k8s/overlord/common/CachingKubernetesPeonClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 989526472ab6..baacc093aaf8 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -202,7 +202,7 @@ private boolean isPodReady(Pod pod) private JobResponse determineJobResponse(Job job) { if (job.getStatus() != null && - (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) && + (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) && (job.getStatus().getFailed() != null || job.getStatus().getSucceeded() != null)) { if (job.getStatus().getSucceeded() != null && job.getStatus().getSucceeded() > 0) { From d62bbc701bc870401d82fac3eef97b832cd21d44 Mon Sep 17 00:00:00 2001 From: capistrant Date: Thu, 9 Oct 2025 21:28:28 -0500 Subject: [PATCH 14/58] fix up some issues with wait for job completion --- .../BaseKubernetesTaskRunnerDockerTest.java | 1 + .../common/CachingKubernetesPeonClient.java | 21 +++++- .../common/DruidKubernetesClient.java | 6 ++ .../overlord/common/KubernetesClientApi.java | 2 + .../CachingKubernetesPeonClientTest.java | 75 +++++++++++++++++++ .../overlord/common/TestKubernetesClient.java | 6 ++ 6 files changed, 109 insertions(+), 2 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java index 9691716049e4..6b003967d708 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java @@ -51,6 +51,7 @@ protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) .addProperty("druid.indexer.runner.namespace", "druid") .addProperty("druid.indexer.runner.capacity", "4") .addProperty("druid.indexer.runner.enableKubernetesClientSharedInformers", String.valueOf(useSharedInformers())) + .addProperty("druid.indexer.runner.kubernetesClientInformerResyncPeriod", "PT1S") .usingPort(30090); final K3sClusterResource k3sCluster = new K3sClusterWithOperatorResource() diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index baacc093aaf8..a9498b352fd0 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -69,21 +69,38 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time long timeoutMs = unit.toMillis(howLong); long startTime = System.currentTimeMillis(); + // Give the informer 2 resync periodd to see the job. if it isn't seen by then, we assume the job was canceled. + // This is to prevent us from waiting for entire max job runtime on a job that was canceled before it even started. + long jobSeenBy = startTime + (clientApi.getInformerResyncPeriodMillis() * 2); + boolean jobSeenInCache = false; + CompletableFuture jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); do { try { Optional maybeJob = getPeonJob(taskId.getK8sJobName()); if (maybeJob.isPresent()) { + jobSeenInCache = true; Job job = maybeJob.get(); JobResponse currentResponse = determineJobResponse(job); if (currentResponse.getPhase() != PeonPhase.RUNNING) { return currentResponse; } + } else if (jobSeenInCache) { + // Job was in cache before, but now it's gone - it was deleted + log.warn("K8s Job[%s] was not found. It can happen if the task was canceled", taskId.getK8sJobName()); + return new JobResponse(null, PeonPhase.FAILED); + } + Job job; + if (jobSeenInCache) { + job = jobFuture.get(timeoutMs, TimeUnit.MILLISECONDS); + } else { + // We haven't seen the job in cache yet, wait a resync cycles instead of the full max runtime allowed + job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); } - Job job = jobFuture.get(timeoutMs, TimeUnit.MILLISECONDS); // Immediately set up to watch for the next change in case we need to wait again jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); log.debug("Received job[%s] change notification", taskId.getK8sJobName()); + jobSeenInCache = true; if (job == null) { log.warn("K8s job for the task[%s] was not found. It can happen if the task was canceled", taskId); return new JobResponse(null, PeonPhase.FAILED); @@ -97,7 +114,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time catch (Throwable e) { log.warn("Exception[%s] waiting for job change notification for job[%s]. Error message[%s]", e.getClass().getName(), taskId.getK8sJobName(), e.getMessage()); } - } while (System.currentTimeMillis() - startTime < timeoutMs); + } while ((System.currentTimeMillis() - startTime < timeoutMs) && (jobSeenInCache || System.currentTimeMillis() < jobSeenBy)); log.warn("Timed out waiting for K8s job[%s] to complete", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 87bcfd975e40..56eb95feff39 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -245,4 +245,10 @@ private void notifyPodChange(Pod pod) } } } + + @Override + public long getInformerResyncPeriodMillis() + { + return informerResyncPeriodMillis; + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java index 68f16f19aec6..bcd6c1ed914f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java @@ -42,5 +42,7 @@ public interface KubernetesClientApi SharedIndexInformer getJobInformer(); + long getInformerResyncPeriodMillis(); + KubernetesResourceEventNotifier getEventNotifier(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index e4f8c056eae7..a55f49b53b68 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -34,6 +34,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.Collections; import java.util.HashMap; @@ -466,6 +467,74 @@ public void test_waitForPeonJobCompletion_jobFails() throws Exception Assertions.assertNotNull(response.getJob()); } + @Test + public void test_waitForPeonJobCompletion_jobGetsDeleted() throws Exception + { + // Create job in running state + K8sTaskId taskId = new K8sTaskId("", "deleted-task-id"); + Job job = new JobBuilder() + .withNewMetadata() + .withName(taskId.getK8sJobName()) + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .withNewStatus() + .withActive(1) + .endStatus() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); + clientApi.waitForSync(); + + // Start waiting in background + CompletableFuture futureResponse = CompletableFuture.supplyAsync(() -> + peonClient.waitForPeonJobCompletion(taskId, 60, TimeUnit.SECONDS) + ); + + // Give it a moment to start waiting + Thread.sleep(500); + + // Delete the job (simulates task cancellation/shutdown) + client.batch().v1().jobs().inNamespace(NAMESPACE).withName(taskId.getK8sJobName()).delete(); + + // Wait for response + JobResponse response = futureResponse.get(10, TimeUnit.SECONDS); + + // Should detect deletion and return FAILED + Assertions.assertEquals(PeonPhase.FAILED, response.getPhase()); + Assertions.assertNull(response.getJob()); + } + + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + public void test_waitForPeonJobCompletion_jobDeletedBeforeSeenInCache() throws Exception + { + // Create job + K8sTaskId taskId = new K8sTaskId("", "quick-delete-task-id"); + Job job = new JobBuilder() + .withNewMetadata() + .withName(taskId.getK8sJobName()) + .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") + .endMetadata() + .withNewStatus() + .withActive(1) + .endStatus() + .build(); + + client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); + + // Delete immediately before informer syncs + client.batch().v1().jobs().inNamespace(NAMESPACE).withName(taskId.getK8sJobName()).delete(); + + clientApi.waitForSync(); + + JobResponse response = peonClient.waitForPeonJobCompletion(taskId, 10, TimeUnit.SECONDS); + + // Should timeout or detect job was never seen and return FAILED + Assertions.assertEquals(PeonPhase.FAILED, response.getPhase()); + } + /** * Test implementation of KubernetesClientApi that uses real informers with the mock server */ @@ -682,6 +751,12 @@ public SharedIndexInformer getJobInformer() return jobInformer; } + @Override + public long getInformerResyncPeriodMillis() + { + return 1000L; + } + @Override public KubernetesResourceEventNotifier getEventNotifier() { diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java index 1c670afef9d3..e17f23aa70bf 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -77,6 +77,12 @@ public SharedIndexInformer getJobInformer() return jobInformer; } + @Override + public long getInformerResyncPeriodMillis() + { + return 0; + } + @Override public KubernetesResourceEventNotifier getEventNotifier() { From 5f6920b0e005805a4cc71c748364d56a8b6dff56 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 10 Oct 2025 11:09:02 -0500 Subject: [PATCH 15/58] cleanup and fix some tests --- .../CachingKubernetesPeonClientTest.java | 233 +----------------- .../DirectKubernetesPeonClientTest.java | 17 +- .../overlord/common/TestKubernetesClient.java | 204 ++++++++++++++- .../taskadapter/K8sTaskAdapterTest.java | 22 +- .../MultiContainerTaskAdapterTest.java | 6 +- .../SingleContainerTaskAdapterTest.java | 2 +- 6 files changed, 219 insertions(+), 265 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index a55f49b53b68..cd745dfadb2b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -55,7 +55,7 @@ public class CachingKubernetesPeonClientTest private KubernetesClient client; private KubernetesMockServer server; - private TestKubernetesClientApi clientApi; + private TestKubernetesClient clientApi; private CachingKubernetesPeonClient peonClient; private StubServiceEmitter serviceEmitter; @@ -65,7 +65,7 @@ public void setup() throws Exception serviceEmitter = new StubServiceEmitter("service", "host"); // Set up real informers with the mock client - clientApi = new TestKubernetesClientApi(client, NAMESPACE); + clientApi = new TestKubernetesClient(client, NAMESPACE); clientApi.start(); peonClient = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); @@ -534,233 +534,4 @@ public void test_waitForPeonJobCompletion_jobDeletedBeforeSeenInCache() throws E // Should timeout or detect job was never seen and return FAILED Assertions.assertEquals(PeonPhase.FAILED, response.getPhase()); } - - /** - * Test implementation of KubernetesClientApi that uses real informers with the mock server - */ - private static class TestKubernetesClientApi implements KubernetesClientApi - { - private final KubernetesClient client; - private final SharedIndexInformer podInformer; - private final SharedIndexInformer jobInformer; - private final KubernetesResourceEventNotifier eventNotifier; - private final CountDownLatch syncLatch; - - public TestKubernetesClientApi(KubernetesClient client, String namespace) - { - this.client = client; - this.eventNotifier = new KubernetesResourceEventNotifier(); - this.syncLatch = new CountDownLatch(2); // Wait for both informers - - // Set up pod informer with real event handlers - this.podInformer = client.pods() - .inNamespace(namespace) - .inform(new ResourceEventHandler() - { - @Override - public void onAdd(Pod pod) - { - notifyPodChange(pod); - } - - @Override - public void onUpdate(Pod oldPod, Pod newPod) - { - notifyPodChange(newPod); - } - - @Override - public void onDelete(Pod pod, boolean deletedFinalStateUnknown) - { - notifyPodChange(pod); - } - }, 1000L); - - // Add pod indexer - Map>> podIndexers = new HashMap<>(); - podIndexers.put("byJobName", pod -> { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - return Collections.singletonList(jobName); - } - } - return Collections.emptyList(); - }); - podInformer.addIndexers(podIndexers); - - // Set up job informer with real event handlers - this.jobInformer = client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .inform(new ResourceEventHandler() - { - @Override - public void onAdd(Job job) - { - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - - @Override - public void onUpdate(Job oldJob, Job newJob) - { - eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); - } - - @Override - public void onDelete(Job job, boolean deletedFinalStateUnknown) - { - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - }, 1000L); - - // Add job indexers - Map>> jobIndexers = new HashMap<>(); - jobIndexers.put("byJobName", job -> { - if (job.getMetadata() != null && job.getMetadata().getName() != null) { - return Collections.singletonList(job.getMetadata().getName()); - } - return Collections.emptyList(); - }); - jobIndexers.put("byOverlordNamespace", job -> { - if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { - String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); - if (overlordNamespace != null) { - return Collections.singletonList(overlordNamespace); - } - } - return Collections.emptyList(); - }); - jobInformer.addIndexers(jobIndexers); - } - - public void start() - { - // Add ready callbacks to count down latch - podInformer.addEventHandlerWithResyncPeriod(new ResourceEventHandler() - { - @Override - public void onAdd(Pod obj) - { - - } - @Override - public void onUpdate(Pod oldObj, Pod newObj) - { - - } - @Override - public void onDelete(Pod obj, boolean deletedFinalStateUnknown) - { - - } - }, 1000L); - - jobInformer.addEventHandlerWithResyncPeriod(new ResourceEventHandler() - { - @Override - public void onAdd(Job obj) - { - - } - @Override - public void onUpdate(Job oldObj, Job newObj) - { - - } - @Override - public void onDelete(Job obj, boolean deletedFinalStateUnknown) - { - - } - }, 1000L); - - podInformer.run(); - jobInformer.run(); - - // Count down after starting - syncLatch.countDown(); - syncLatch.countDown(); - } - - public void stop() - { - if (podInformer != null) { - podInformer.stop(); - } - if (jobInformer != null) { - jobInformer.stop(); - } - if (eventNotifier != null) { - eventNotifier.cancelAll(); - } - } - - public void waitForSync() throws InterruptedException - { - syncLatch.await(5, TimeUnit.SECONDS); - // Give informers a bit more time to process - Thread.sleep(200); - } - - private void notifyPodChange(Pod pod) - { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - eventNotifier.notifyPodChange(jobName, pod); - } - } - } - - @Override - public T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException - { - return executor.executeRequest(client); - } - - @Override - public T executePodCacheRequest(KubernetesInformerExecutor executor) - { - return executor.executeRequest(podInformer); - } - - @Override - public T executeJobCacheRequest(KubernetesInformerExecutor executor) - { - return executor.executeRequest(jobInformer); - } - - @Override - public KubernetesClient getClient() - { - return client; - } - - @Override - public SharedIndexInformer getPodInformer() - { - return podInformer; - } - - @Override - public SharedIndexInformer getJobInformer() - { - return jobInformer; - } - - @Override - public long getInformerResyncPeriodMillis() - { - return 1000L; - } - - @Override - public KubernetesResourceEventNotifier getEventNotifier() - { - return eventNotifier; - } - } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index 95a6af3b1c09..ab0fc9327114 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -63,7 +63,7 @@ public class DirectKubernetesPeonClientTest @BeforeEach public void setup() { - clientApi = new TestKubernetesClient(this.client, null, null); + clientApi = new TestKubernetesClient(this.client, NAMESPACE); serviceEmitter = new StubServiceEmitter("service", "host"); instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); } @@ -236,7 +236,7 @@ void test_deletePeonJob_withoutJob_returnsFalse() void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() { DirectKubernetesPeonClient instance = new DirectKubernetesPeonClient( - new TestKubernetesClient(this.client, null, null), + new TestKubernetesClient(this.client, NAMESPACE), NAMESPACE, true, serviceEmitter @@ -261,7 +261,7 @@ void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() { DirectKubernetesPeonClient instance = new DirectKubernetesPeonClient( - new TestKubernetesClient(this.client, null, null), + new TestKubernetesClient(this.client, NAMESPACE), NAMESPACE, true, serviceEmitter @@ -823,15 +823,10 @@ void test_waitForPodResultWithRetries_withNonRetryableFailure_throwsDruidExcepti .endStatus() .build(); - String podPath = "/api/v1/namespaces/" + NAMESPACE + "/pods/" + POD_NAME; - - // Mock server to return the pod without IP, causing timeout - server.expect().get() - .withPath(podPath + "?watch=true") - .andReturn(HttpURLConnection.HTTP_INTERNAL_ERROR, "Internal server error") - .once(); + // Create the pod in the mock client without IP - it will remain unready + client.pods().inNamespace(NAMESPACE).resource(pod).create(); - // Should throw DruidException after failure + // Should throw DruidException after timeout due to pod never becoming ready DruidException e = Assertions.assertThrows( DruidException.class, () -> instance.waitForPodResultWithRetries( diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java index e17f23aa70bf..55ab00eec8c7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -22,23 +22,211 @@ import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + public class TestKubernetesClient implements KubernetesClientApi { private final KubernetesClient client; private final SharedIndexInformer podInformer; private final SharedIndexInformer jobInformer; + private final KubernetesResourceEventNotifier eventNotifier; + private final CountDownLatch syncLatch; - public TestKubernetesClient(KubernetesClient client, - SharedIndexInformer podInformer, - SharedIndexInformer jobInformer - ) + public TestKubernetesClient(KubernetesClient client, String namespace) { this.client = client; - this.podInformer = podInformer; - this.jobInformer = jobInformer; + this.eventNotifier = new KubernetesResourceEventNotifier(); + this.syncLatch = new CountDownLatch(2); // Wait for both informers + + // Set up pod informer with real event handlers + this.podInformer = client.pods() + .inNamespace(namespace) + .inform( + new ResourceEventHandler() + { + @Override + public void onAdd(Pod pod) + { + notifyPodChange(pod); + } + + @Override + public void onUpdate(Pod oldPod, Pod newPod) + { + notifyPodChange(newPod); + } + + @Override + public void onDelete(Pod pod, boolean deletedFinalStateUnknown) + { + notifyPodChange(pod); + } + }, 1000L + ); + + // Add pod indexer + Map>> podIndexers = new HashMap<>(); + podIndexers.put( + "byJobName", pod -> { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + return Collections.singletonList(jobName); + } + } + return Collections.emptyList(); + } + ); + podInformer.addIndexers(podIndexers); + + // Set up job informer with real event handlers + this.jobInformer = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .inform( + new ResourceEventHandler() + { + @Override + public void onAdd(Job job) + { + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + + @Override + public void onUpdate(Job oldJob, Job newJob) + { + eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); + } + + @Override + public void onDelete(Job job, boolean deletedFinalStateUnknown) + { + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + }, 1000L + ); + + // Add job indexers + Map>> jobIndexers = new HashMap<>(); + jobIndexers.put( + "byJobName", job -> { + if (job.getMetadata() != null && job.getMetadata().getName() != null) { + return Collections.singletonList(job.getMetadata().getName()); + } + return Collections.emptyList(); + } + ); + jobIndexers.put( + "byOverlordNamespace", job -> { + if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { + String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); + if (overlordNamespace != null) { + return Collections.singletonList(overlordNamespace); + } + } + return Collections.emptyList(); + } + ); + jobInformer.addIndexers(jobIndexers); + } + + public void start() + { + // Add ready callbacks to count down latch + podInformer.addEventHandlerWithResyncPeriod( + new ResourceEventHandler() + { + @Override + public void onAdd(Pod obj) + { + + } + + @Override + public void onUpdate(Pod oldObj, Pod newObj) + { + + } + + @Override + public void onDelete(Pod obj, boolean deletedFinalStateUnknown) + { + + } + }, 1000L + ); + + jobInformer.addEventHandlerWithResyncPeriod( + new ResourceEventHandler() + { + @Override + public void onAdd(Job obj) + { + + } + + @Override + public void onUpdate(Job oldObj, Job newObj) + { + + } + + @Override + public void onDelete(Job obj, boolean deletedFinalStateUnknown) + { + + } + }, 1000L + ); + + podInformer.run(); + jobInformer.run(); + + // Count down after starting + syncLatch.countDown(); + syncLatch.countDown(); + } + + public void stop() + { + if (podInformer != null) { + podInformer.stop(); + } + if (jobInformer != null) { + jobInformer.stop(); + } + if (eventNotifier != null) { + eventNotifier.cancelAll(); + } + } + + public void waitForSync() throws InterruptedException + { + syncLatch.await(5, TimeUnit.SECONDS); + // Give informers a bit more time to process + Thread.sleep(200); + } + + private void notifyPodChange(Pod pod) + { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + eventNotifier.notifyPodChange(jobName, pod); + } + } } @Override @@ -80,12 +268,12 @@ public SharedIndexInformer getJobInformer() @Override public long getInformerResyncPeriodMillis() { - return 0; + return 1000L; } @Override public KubernetesResourceEventNotifier getEventNotifier() { - return null; + return eventNotifier; } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 3df5ac272ace..3d955be3dbf8 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -121,7 +121,7 @@ public K8sTaskAdapterTest() void testAddingLabelsAndAnnotations() throws IOException { final PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null) + TestKubernetesClient testClient = new TestKubernetesClient(client, "namespace") { @SuppressWarnings("unchecked") @Override @@ -174,7 +174,7 @@ public PodSpec getSpec() public void serializingAndDeserializingATask() throws IOException { // given a task create a k8s job - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "test"); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); @@ -212,7 +212,7 @@ public void serializingAndDeserializingATask() throws IOException public void fromTask_dontSetTaskJSON() throws IOException { final PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null) + TestKubernetesClient testClient = new TestKubernetesClient(client, "test") { @SuppressWarnings("unchecked") @Override @@ -276,7 +276,7 @@ public PodSpec getSpec() @Test public void toTask_useTaskPayloadManager() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "test"); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); @@ -308,7 +308,7 @@ public void toTask_useTaskPayloadManager() throws IOException @Test public void getTaskId() { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "test"); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -330,7 +330,7 @@ public void getTaskId() @Test public void getTaskId_noAnnotations() { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "test"); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -352,7 +352,7 @@ public void getTaskId_noAnnotations() @Test public void getTaskId_missingTaskIdAnnotation() { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "test"); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -451,7 +451,7 @@ void testNoPrimaryFound() @Test void testAddingMonitors() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "test"); PeonCommandContext context = new PeonCommandContext( new ArrayList<>(), new ArrayList<>(), @@ -530,7 +530,7 @@ void testAddingMonitors() throws IOException @Test void testEphemeralStorageIsRespected() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "namespace"); Pod pod = K8sTestUtils.fileToResource("ephemeralPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") @@ -580,7 +580,7 @@ void testEphemeralStorageIsRespected() throws IOException @Test void testProbesRemoved() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "test"); Pod pod = K8sTestUtils.fileToResource("probesPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") @@ -630,7 +630,7 @@ void testProbesRemoved() throws IOException @Test void testCPUResourceIsRespected() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "namespace"); Pod pod = K8sTestUtils.fileToResource("ephemeralPodSpec.yaml", Pod.class); List javaOpts = new ArrayList<>(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java index 844053667d17..f784d8467d6b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java @@ -85,7 +85,7 @@ public void setup() @Test public void testMultiContainerSupport() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "namespace"); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") @@ -136,7 +136,7 @@ public void testMultiContainerSupport() throws IOException @Test public void testMultiContainerSupportWithNamedContainer() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "namespace"); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpecOrder.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") @@ -190,7 +190,7 @@ public void testMultiContainerSupportWithNamedContainer() throws IOException @Test public void testOverridingPeonMonitors() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "namespace"); Pod pod = K8sTestUtils.fileToResource("podSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java index 60fa5eb31ed8..791a60dd64aa 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java @@ -84,7 +84,7 @@ public void setup() @Test public void testSingleContainerSupport() throws IOException { - TestKubernetesClient testClient = new TestKubernetesClient(client, null, null); + TestKubernetesClient testClient = new TestKubernetesClient(client, "namespace"); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") From 5b6c8f3846e2cc47048c402ba1fc7cab5aff49fb Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 10 Oct 2025 11:12:27 -0500 Subject: [PATCH 16/58] Make DruidKubernetesClient defend against invalid use if caching is off --- .../druid/k8s/overlord/common/DruidKubernetesClient.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 56eb95feff39..de521725a762 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -124,6 +124,9 @@ public SharedIndexInformer getJobInformer() @Override public KubernetesResourceEventNotifier getEventNotifier() { + if (eventNotifier == null) { + throw DruidException.defensive("Event notifier is not initialized, caching is disabled"); + } return eventNotifier; } @@ -249,6 +252,9 @@ private void notifyPodChange(Pod pod) @Override public long getInformerResyncPeriodMillis() { + if (jobInformer == null || podInformer == null) { + throw DruidException.defensive("Informers are not initialized, caching is disabled"); + } return informerResyncPeriodMillis; } } From 11db5adee4c27d8f95de95ff11e393aa7869974f Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 10 Oct 2025 13:13:16 -0500 Subject: [PATCH 17/58] cleanup checkstyle --- .../overlord/common/CachingKubernetesPeonClientTest.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index cd745dfadb2b..cf605ee2fdac 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -25,8 +25,6 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import io.fabric8.kubernetes.client.KubernetesClient; -import io.fabric8.kubernetes.client.informers.ResourceEventHandler; -import io.fabric8.kubernetes.client.informers.SharedIndexInformer; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer; import org.apache.druid.java.util.metrics.StubServiceEmitter; @@ -36,14 +34,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.function.Function; @EnableKubernetesMockClient(crud = true) public class CachingKubernetesPeonClientTest From 144dd4916c712df7ad14d94998cedc505e6a566f Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 10 Oct 2025 13:19:10 -0500 Subject: [PATCH 18/58] dont use deprecated method --- .../apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index df70487ac8b3..35d94a4c2140 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -251,11 +251,11 @@ private KubernetesTaskRunnerConfig( capacity, this.capacity ); - this.enableKubernetesClientSharedInformers = ObjectUtils.defaultIfNull( + this.enableKubernetesClientSharedInformers = ObjectUtils.getIfNull( enableKubernetesClientSharedInformers, this.enableKubernetesClientSharedInformers ); - this.kubernetesClientInformerResyncPeriod = ObjectUtils.defaultIfNull( + this.kubernetesClientInformerResyncPeriod = ObjectUtils.getIfNull( kubernetesClientInformerResyncPeriod, this.kubernetesClientInformerResyncPeriod ); From 7fcbcec144a4552d3bd6132befd3ada358067199 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 13 Oct 2025 09:06:53 -0500 Subject: [PATCH 19/58] doc update --- docs/development/extensions-core/k8s-jobs.md | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index cb688c811036..3b9f7ce594e7 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -41,12 +41,23 @@ Task lifecycle code in Druid talks directly to the Kubernetes API server for all ### `SharedInformer` "Caching" *(Experimental)* -Enabled by setting `druid.indexer.runner.k8s.useSharedInformer=true`, this mode uses `Fabric8` `SharedInformer` objects for monitoring state changes in the remote K8s cluster, reducing the number of direct API calls to the Kubernetes API server. This can greatly reduce load on the API server, especially in environments with a high volume of tasks. +Enabled by setting `druid.indexer.runner.enableKubernetesClientSharedInformers=true`, this mode uses `Fabric8` `SharedInformer` objects for monitoring state changes in the remote K8s cluster, reducing the number of direct API calls to the Kubernetes API server. This can greatly reduce load on the API server, especially in environments with a high volume of tasks. This mode is experimental and should be used with caution in production until it has been vetted more thoroughly by the community. The core idea is to use two `SharedInformers`, one for jobs and one for pods, to watch for changes in the remote K8s cluster. These informers maintain a local cache of jobs and pods that tasks can query. The informers can also notify listeners when changes occur, allowing tasks to react to state changes without polling the API server or creating per-task watches on the K8s cluster. +#### Architecture: Direct vs. Caching Mode + +**Key Differences:** + +- **DirectKubernetesPeonClient** (Default): Every read operation makes a direct HTTP call to the K8s API server. With 100 concurrent tasks, this results in 100+ active API connections with continuous polling. + +- **CachingKubernetesPeonClient** (Experimental): All read operations query an in-memory cache maintained by SharedInformers. With 100 concurrent tasks, only 2 persistent watch connections are used (one for Jobs, one for Pods), achieving a large reduction in API calls. + +**Shared Operations**: + +Both implementations share the same write (job creation, deletion) and log read operations code, which always use direct API calls. ## Configuration From 2301dde4e4b63f22e551ee30d9a643b61ecb547a Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 14 Oct 2025 14:35:56 -0500 Subject: [PATCH 20/58] fix spelling --- docs/development/extensions-core/k8s-jobs.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index 3b9f7ce594e7..697b5992b5b8 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -51,9 +51,9 @@ The core idea is to use two `SharedInformers`, one for jobs and one for pods, to **Key Differences:** -- **DirectKubernetesPeonClient** (Default): Every read operation makes a direct HTTP call to the K8s API server. With 100 concurrent tasks, this results in 100+ active API connections with continuous polling. +- `DirectKubernetesPeonClient` (Default): Every read operation makes a direct HTTP call to the K8s API server. With 100 concurrent tasks, this results in 100+ active API connections with continuous polling. -- **CachingKubernetesPeonClient** (Experimental): All read operations query an in-memory cache maintained by SharedInformers. With 100 concurrent tasks, only 2 persistent watch connections are used (one for Jobs, one for Pods), achieving a large reduction in API calls. +- `CachingKubernetesPeonClient` (Experimental): All read operations query an in-memory cache maintained by `SharedInformers`. With 100 concurrent tasks, only 2 persistent watch connections are used (one for Jobs, one for Pods), achieving a large reduction in API calls. **Shared Operations**: From 898347720b4baa71245ea0e50c481561a7eb2489 Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 15 Oct 2025 12:46:15 -0500 Subject: [PATCH 21/58] fix checkstyle after merge with master --- .../apache/druid/k8s/overlord/common/DruidKubernetesClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index f7c70fe2118e..238f9f2d3c39 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -24,11 +24,11 @@ import io.fabric8.kubernetes.client.Config; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientBuilder; -import org.apache.druid.k8s.overlord.common.httpclient.DruidKubernetesHttpClientFactory; import io.fabric8.kubernetes.client.informers.ResourceEventHandler; import io.fabric8.kubernetes.client.informers.SharedIndexInformer; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.k8s.overlord.common.httpclient.DruidKubernetesHttpClientFactory; import java.util.Collections; import java.util.HashMap; From 06ed5282d508241f9dacf9dd35bb6db774ef2f7d Mon Sep 17 00:00:00 2001 From: capistrant Date: Thu, 16 Oct 2025 15:44:46 -0500 Subject: [PATCH 22/58] Improve reliability of the Caching K8s Peon Client code and associated embedded tests --- docs/development/extensions-core/k8s-jobs.md | 2 +- .../embedded/indexing/IngestionSmokeTest.java | 26 +++-- .../BaseKubernetesTaskRunnerDockerTest.java | 2 +- .../kafka/simulate/KafkaResource.java | 3 +- .../overlord/KubernetesTaskRunnerConfig.java | 2 +- .../common/CachingKubernetesPeonClient.java | 105 ++++++++++++------ .../CachingKubernetesPeonClientTest.java | 1 + 7 files changed, 91 insertions(+), 50 deletions(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index 251b37f986c2..94e5893846c4 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -819,7 +819,7 @@ Should you require the needed permissions for interacting across Kubernetes name | `druid.indexer.runner.cpuCoreInMicro` | `Integer` | Number of CPU micro core for the task. | `1000` | No | | `druid.indexer.runner.logSaveTimeout` | `Duration` | The peon executing the ingestion task makes a best effort to persist the pod logs from `k8s` to persistent task log storage. The timeout ensures that `k8s` connection issues do not cause the pod to hang indefinitely thereby blocking Overlord operations. If the timeout occurs before the logs are saved, those logs will not be available in Druid. | `PT300S` | NO | | `druid.indexer.runner.enableKubernetesClientSharedInformers` | `boolean` | Whether to use shared informers to watch for pod/job changes. This is more efficient on the Kubernetes API server, but may use more memory in the Overlord. | `false` | No | -| `druid.indexer.runner.kubernetesClientInformerResyncPeriod` | `Duration` | When using shared informers, controls how frequently the informers resync with the Kubernetes API server. | `PT30S` | No | +| `druid.indexer.runner.kubernetesClientInformerResyncPeriod` | `Duration` | When using shared informers, controls how frequently the informers resync with the Kubernetes API server. This prevents change events from being missed, keeping the informer cache clean and accurate. | `PT90S` | No | ### Metrics added diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java index 13245227d2aa..db63d9ca76a0 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java @@ -25,7 +25,6 @@ import org.apache.druid.common.utils.IdUtils; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.IndexTask; @@ -243,7 +242,7 @@ public void test_runIndexParallelTask_andCompactData() .dynamicPartitionWithMaxRows(5000) .withId(compactTaskId); cluster.callApi().onLeaderOverlord(o -> o.runTask(compactTaskId, compactionTask)); - cluster.callApi().waitForTaskToSucceed(taskId, eventCollector.latchableEmitter()); + cluster.callApi().waitForTaskToSucceed(compactTaskId, eventCollector.latchableEmitter()); // Verify the compacted data final int numCompactedSegments = 5; @@ -308,13 +307,11 @@ public void test_runKafkaSupervisor() Assertions.assertEquals("RUNNING", supervisorStatus.getState()); Assertions.assertEquals(topic, supervisorStatus.getSource()); - // Get the task statuses - List taskStatuses = ImmutableList.copyOf( - (CloseableIterator) - cluster.callApi().onLeaderOverlord(o -> o.taskStatuses(null, dataSource, 1)) - ); - Assertions.assertFalse(taskStatuses.isEmpty()); - Assertions.assertEquals(TaskState.RUNNING, taskStatuses.get(0).getStatusCode()); + // Confirm tasks are being created and running + // This more forgiving assertion avoids weird race conditions with super specific checks like "exactly 1 running task" + int runningTasks = getTaskCount("running", dataSource); + int completedTasks = getTaskCount("complete", dataSource); + Assertions.assertTrue(runningTasks + completedTasks > 0); // Suspend the supervisor and verify the state cluster.callApi().onLeaderOverlord( @@ -408,6 +405,17 @@ private void waitForSegmentsToBeQueryable(int numSegments) ); } + /** + * Gets the count of tasks with the given status for the specified datasource. + */ + private int getTaskCount(String status, String dataSource) + { + return ImmutableList.copyOf( + (CloseableIterator) + cluster.callApi().onLeaderOverlord(o -> o.taskStatuses(status, dataSource, 100)) + ).size(); + } + /** * Verifies the total number of used segments in {@link #dataSource}. */ diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java index 6b003967d708..2677612f1bf4 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java @@ -51,7 +51,7 @@ protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) .addProperty("druid.indexer.runner.namespace", "druid") .addProperty("druid.indexer.runner.capacity", "4") .addProperty("druid.indexer.runner.enableKubernetesClientSharedInformers", String.valueOf(useSharedInformers())) - .addProperty("druid.indexer.runner.kubernetesClientInformerResyncPeriod", "PT1S") + .addProperty("druid.indexer.runner.kubernetesClientInformerResyncPeriod", "PT1s") .usingPort(30090); final K3sClusterResource k3sCluster = new K3sClusterWithOperatorResource() diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java index 6a1d5abe546b..cb56adc00a8b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java @@ -40,7 +40,8 @@ */ public class KafkaResource extends TestcontainerResource { - private static final String KAFKA_IMAGE = "apache/kafka:4.0.0"; + // Offering an opportunity to override can help some local devs whose system struggles to run apache/kafka. Overriding with apache/kafka-native can help. + private static final String KAFKA_IMAGE = System.getProperty("druid.kafka.test.image", "apache/kafka:4.0.0"); private EmbeddedDruidCluster cluster; diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index cfeccf9f4d1e..34bf38764616 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -85,7 +85,7 @@ public class KubernetesTaskRunnerConfig private boolean enableKubernetesClientSharedInformers = false; @JsonProperty - private Period kubernetesClientInformerResyncPeriod = new Period("PT30S"); + private Period kubernetesClientInformerResyncPeriod = new Period("PT90s"); @JsonProperty @NotNull diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index a9498b352fd0..2cd6833b0f92 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -22,6 +22,7 @@ import com.google.common.base.Optional; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.utils.PodStatusUtil; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -30,6 +31,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; /** * A KubernetesPeonClient implementation that uses cached informers to read Job and Pod state. @@ -69,9 +71,9 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time long timeoutMs = unit.toMillis(howLong); long startTime = System.currentTimeMillis(); - // Give the informer 2 resync periodd to see the job. if it isn't seen by then, we assume the job was canceled. + // Give the informer 2 resync periods to see the job. if it isn't seen by then, we assume the job was canceled. // This is to prevent us from waiting for entire max job runtime on a job that was canceled before it even started. - long jobSeenBy = startTime + (clientApi.getInformerResyncPeriodMillis() * 2); + long jobMustBeSeenBy = startTime + (clientApi.getInformerResyncPeriodMillis() * 2); boolean jobSeenInCache = false; CompletableFuture jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); @@ -84,37 +86,47 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time JobResponse currentResponse = determineJobResponse(job); if (currentResponse.getPhase() != PeonPhase.RUNNING) { return currentResponse; + } else { + log.debug("K8s job[%s] found in cache and is still running", taskId.getK8sJobName()); } } else if (jobSeenInCache) { - // Job was in cache before, but now it's gone - it was deleted + // Job was in cache before, but now it's gone - it was deleted and will never complete. log.warn("K8s Job[%s] was not found. It can happen if the task was canceled", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); } - Job job; - if (jobSeenInCache) { - job = jobFuture.get(timeoutMs, TimeUnit.MILLISECONDS); - } else { - // We haven't seen the job in cache yet, wait a resync cycles instead of the full max runtime allowed - job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); - } + + // We wake up every informer resync period to avoid event notifier misses. + Job job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); + // Immediately set up to watch for the next change in case we need to wait again jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); log.debug("Received job[%s] change notification", taskId.getK8sJobName()); jobSeenInCache = true; + if (job == null) { - log.warn("K8s job for the task[%s] was not found. It can happen if the task was canceled", taskId); + log.warn("K8s job[%s] was not found. It can happen if the task was canceled", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); } JobResponse currentResponse = determineJobResponse(job); if (currentResponse.getPhase() != PeonPhase.RUNNING) { return currentResponse; + } else { + log.debug("K8s job[%s] is still running", taskId.getK8sJobName()); } } + catch (TimeoutException e) { + // A timeout here is not a problem, it forces us to loop around and check the cache again. + // This prevents the case where we miss a notification and wait forever. + log.debug("Timeout waiting for job change notification for job[%s], checking cache again", taskId.getK8sJobName()); + } + catch (InterruptedException e) { + throw DruidException.defensive(e, "Interrupted waiting for job change notification for job[%s]", taskId.getK8sJobName()); + } catch (Throwable e) { log.warn("Exception[%s] waiting for job change notification for job[%s]. Error message[%s]", e.getClass().getName(), taskId.getK8sJobName(), e.getMessage()); } - } while ((System.currentTimeMillis() - startTime < timeoutMs) && (jobSeenInCache || System.currentTimeMillis() < jobSeenBy)); + } while ((System.currentTimeMillis() - startTime < timeoutMs) && (jobSeenInCache || System.currentTimeMillis() < jobMustBeSeenBy)); log.warn("Timed out waiting for K8s job[%s] to complete", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); @@ -166,33 +178,47 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time Optional maybePod = getPeonPod(jobName); if (maybePod.isPresent()) { podSeenInCache = true; - // Check if pod is ready (has IP) Pod pod = maybePod.get(); - if (isPodReady(pod)) { - log.info("Pod[%s] for job[%s] is ready with IP[%s]", podName, jobName, pod.getStatus().getPodIP()); + podName = pod.getMetadata().getName(); + + if (isPodRunningOrComplete(pod)) { + log.info("Pod[%s] for job[%s] is running or complete", podName, jobName); return pod; } else { - log.debug("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); + log.debug("Pod[%s] for job[%s] exists but not ready yet", podName, jobName); } + } else { + log.info("Pod for job[%s] not created yet", jobName); } - Pod pod = podFuture.get(timeoutMs, TimeUnit.MILLISECONDS); + // We wake up every informer resync period to avoid event notifier misses. + Pod pod = podFuture.get(clientApi.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); + podFuture = clientApi.getEventNotifier().waitForPodChange(jobName); log.debug("Received pod[%s] change notification for job[%s]", podName, jobName); if (pod == null) { throw DruidException.defensive("Pod[%s] for job[%s] is null. This is unusual. Investigate Druid and k8s logs.", podName, jobName); } else { podSeenInCache = true; - if (isPodReady(pod)) { - log.info("Pod[%s] for job[%s] is ready with IP[%s]", podName, jobName, pod.getStatus().getPodIP()); + podName = pod.getMetadata().getName(); + if (isPodRunningOrComplete(pod)) { + log.info("Pod[%s] for job[%s] is running or complete", podName, jobName); return pod; } else { - log.debug("Pod[%s] for job[%s] exists but not ready yet (no IP assigned)", podName, jobName); + log.debug("Pod[%s] for job[%s] exists but not ready yet", podName, jobName); } } } + catch (TimeoutException e) { + // A timeout here is not a problem, it forces us to loop around and check the cache again. + // This prevents the case where we miss a notification and wait forever. + log.debug("Timeout waiting for pod change notification for job[%s], checking cache again", jobName); + } + catch (InterruptedException e) { + throw DruidException.defensive(e, "Interrupted waiting for pod change notification for job[%s]", jobName); + } catch (Throwable e) { - log.warn("Exception[%s] waiting for pod change notification for job [%s]. Error message[%s]", e.getClass().getName(), jobName, e.getMessage()); + log.warn("Unexpected exception[%s] waiting for pod change notification for job [%s]. Error message[%s]", e.getClass().getName(), jobName, e.getMessage()); } } while (System.currentTimeMillis() - startTime < timeoutMs); @@ -206,11 +232,15 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time } /** - * Check if the pod is ready. For our purposes, this means it has been assigned an IP address. + * Check if the pod is in Running, Succeeded or Failed phase. */ - private boolean isPodReady(Pod pod) + private boolean isPodRunningOrComplete(Pod pod) { - return pod.getStatus() != null && pod.getStatus().getPodIP() != null; + // I could not find constants for Pod phases in fabric8, so hardcoding them here. + // They are documented here: https://kubernetes.io/docs/concepts/workloads/pods/pod-lifecycle/#pod-phase + List matchingPhases = List.of("Running", "Succeeded", "Failed"); + return pod.getStatus() != null && + matchingPhases.contains(pod.getStatus().getPhase()); } /** @@ -218,20 +248,21 @@ private boolean isPodReady(Pod pod) */ private JobResponse determineJobResponse(Job job) { - if (job.getStatus() != null && - (job.getStatus().getActive() == null || job.getStatus().getActive() == 0) && - (job.getStatus().getFailed() != null || job.getStatus().getSucceeded() != null)) { - - if (job.getStatus().getSucceeded() != null && job.getStatus().getSucceeded() > 0) { - log.info("K8s job[%s] completed successfully", job.getMetadata().getName()); - return new JobResponse(job, PeonPhase.SUCCEEDED); - } else { - log.warn("K8s job[%s] failed with status %s", job.getMetadata().getName(), job.getStatus()); - return new JobResponse(job, PeonPhase.FAILED); + if (job.getStatus() != null) { + Integer active = job.getStatus().getActive(); + Integer succeeded = job.getStatus().getSucceeded(); + Integer failed = job.getStatus().getFailed(); + if ((active == null || active == 0) && (succeeded != null || failed != null)) { + if (succeeded != null && succeeded > 0) { + log.info("K8s job[%s] completed successfully", job.getMetadata().getName()); + return new JobResponse(job, PeonPhase.SUCCEEDED); + } else { + log.warn("K8s job[%s] failed with status %s", job.getMetadata().getName(), job.getStatus()); + return new JobResponse(job, PeonPhase.FAILED); + } } - } else { - log.debug("K8s job[%s] is still active.", job.getMetadata().getName()); - return new JobResponse(job, PeonPhase.RUNNING); } + log.debug("K8s job[%s] is still active.", job.getMetadata().getName()); + return new JobResponse(job, PeonPhase.RUNNING); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index cf605ee2fdac..abd35ff3b184 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -329,6 +329,7 @@ public void test_waitUntilPeonPodCreatedAndReady_podBecomesReady() throws Except .addToLabels("job-name", JOB_NAME) .endMetadata() .withNewStatus() + .withPhase("Running") .withPodIP("10.0.0.1") .endStatus() .build(); From 34510265d1e737d1fcbf73e26b113a2378587e6e Mon Sep 17 00:00:00 2001 From: capistrant Date: Thu, 16 Oct 2025 15:49:23 -0500 Subject: [PATCH 23/58] fix checkstyle --- .../druid/k8s/overlord/common/CachingKubernetesPeonClient.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 2cd6833b0f92..0e41938a4f90 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -22,7 +22,6 @@ import com.google.common.base.Optional; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; -import io.fabric8.kubernetes.client.utils.PodStatusUtil; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; From d963d55df17968562bf709e53fe4636fc63d6c0e Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 22 Oct 2025 17:20:49 -0500 Subject: [PATCH 24/58] Modifications to try and reduce caching client api impact even more --- docs/development/extensions-core/k8s-jobs.md | 2 +- .../overlord/KubernetesTaskRunnerConfig.java | 2 +- .../common/AbstractKubernetesPeonClient.java | 38 ++++++++++++++----- .../common/DruidKubernetesClient.java | 3 +- 4 files changed, 32 insertions(+), 13 deletions(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index 94e5893846c4..f0a23e25bb7e 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -819,7 +819,7 @@ Should you require the needed permissions for interacting across Kubernetes name | `druid.indexer.runner.cpuCoreInMicro` | `Integer` | Number of CPU micro core for the task. | `1000` | No | | `druid.indexer.runner.logSaveTimeout` | `Duration` | The peon executing the ingestion task makes a best effort to persist the pod logs from `k8s` to persistent task log storage. The timeout ensures that `k8s` connection issues do not cause the pod to hang indefinitely thereby blocking Overlord operations. If the timeout occurs before the logs are saved, those logs will not be available in Druid. | `PT300S` | NO | | `druid.indexer.runner.enableKubernetesClientSharedInformers` | `boolean` | Whether to use shared informers to watch for pod/job changes. This is more efficient on the Kubernetes API server, but may use more memory in the Overlord. | `false` | No | -| `druid.indexer.runner.kubernetesClientInformerResyncPeriod` | `Duration` | When using shared informers, controls how frequently the informers resync with the Kubernetes API server. This prevents change events from being missed, keeping the informer cache clean and accurate. | `PT90S` | No | +| `druid.indexer.runner.kubernetesClientInformerResyncPeriod` | `Duration` | When using shared informers, controls how frequently the informers resync with the Kubernetes API server. This prevents change events from being missed, keeping the informer cache clean and accurate. | `PT300S` | No | ### Metrics added diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index 34bf38764616..ec6bf194a710 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -85,7 +85,7 @@ public class KubernetesTaskRunnerConfig private boolean enableKubernetesClientSharedInformers = false; @JsonProperty - private Period kubernetesClientInformerResyncPeriod = new Period("PT90s"); + private Period kubernetesClientInformerResyncPeriod = new Period("PT5M"); @JsonProperty @NotNull diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index c28c73fd652f..63a9dc38ed43 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -164,13 +164,22 @@ public boolean deletePeonJob(K8sTaskId taskId) public Optional getPeonLogWatcher(K8sTaskId taskId) { + // First, get the pod and if it exists set up log watching + Optional maybePod = getPeonPod(taskId.getK8sJobName()); + if (!maybePod.isPresent()) { + log.debug("Pod for job[%s] not found in cache, cannot watch logs", taskId.getK8sJobName()); + return Optional.absent(); + } + + Pod pod = maybePod.get(); + String podName = pod.getMetadata().getName(); + KubernetesClient k8sClient = clientApi.getClient(); try { - LogWatch logWatch = k8sClient.batch() - .v1() - .jobs() + // Use resource() to pass the pod object directly, avoiding any lookups + LogWatch logWatch = k8sClient.pods() .inNamespace(namespace) - .withName(taskId.getK8sJobName()) + .resource(pod) .inContainer("main") .watchLog(); if (logWatch == null) { @@ -179,20 +188,29 @@ public Optional getPeonLogWatcher(K8sTaskId taskId) return Optional.of(logWatch); } catch (Exception e) { - log.error(e, "Error watching logs from task: %s", taskId); + log.error(e, "Error watching logs from task: %s, pod: %s", taskId, podName); return Optional.absent(); } } public Optional getPeonLogs(K8sTaskId taskId) { + // First, get the pod from cache to avoid unnecessary API calls to look up the job and pod + Optional maybePod = getPeonPod(taskId.getK8sJobName()); + if (!maybePod.isPresent()) { + log.debug("Pod for job[%s] not found in cache, cannot stream logs", taskId.getK8sJobName()); + return Optional.absent(); + } + + Pod pod = maybePod.get(); + String podName = pod.getMetadata().getName(); + KubernetesClient k8sClient = clientApi.getClient(); try { - InputStream logStream = k8sClient.batch() - .v1() - .jobs() + // Use resource() to pass the pod object directly, avoiding any lookups + InputStream logStream = k8sClient.pods() .inNamespace(namespace) - .withName(taskId.getK8sJobName()) + .resource(pod) .inContainer("main") .getLogInputStream(); if (logStream == null) { @@ -201,7 +219,7 @@ public Optional getPeonLogs(K8sTaskId taskId) return Optional.of(logStream); } catch (Exception e) { - log.error(e, "Error streaming logs from task: %s", taskId); + log.error(e, "Error streaming logs from task: %s, pod: %s", taskId, podName); return Optional.absent(); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 238f9f2d3c39..99a98ac95916 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -42,7 +42,7 @@ public class DruidKubernetesClient implements KubernetesClientApi public static final String ENABLE_INFORMERS_KEY = "druid.k8s.informers.enabled"; public static final String INFORMER_RESYNC_PERIOD_MS_KEY = "druid.k8s.informers.resyncPeriodMs"; - private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 3000L; + private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 300000L; // 5 minutes private final KubernetesClient kubernetesClient; private final SharedIndexInformer podInformer; @@ -133,6 +133,7 @@ private SharedIndexInformer setupPodInformer(String namespace) SharedIndexInformer podInformer = kubernetesClient.pods() .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) .inform( new ResourceEventHandler<>() { From 09ab11f3a773b27d161d3df25e8538079a2a392c Mon Sep 17 00:00:00 2001 From: capistrant Date: Thu, 23 Oct 2025 10:45:26 -0500 Subject: [PATCH 25/58] Fixup tests now that we have refactored log fetching --- .../DirectKubernetesPeonClientTest.java | 88 +++---------------- 1 file changed, 14 insertions(+), 74 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index ab0fc9327114..143c0dc9a3d7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -273,44 +273,14 @@ void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() @Test void test_getPeonLogs_withJob_returnsInputStreamInOptional() { - server.expect().get() - .withPath("/apis/batch/v1/namespaces/namespace/jobs/" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder() - .withNewMetadata() - .withName(KUBERNETES_JOB_NAME) - .withUid("uid") - .endMetadata() - .withNewSpec() - .withNewTemplate() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endTemplate() - .endSpec() - .build() - ).once(); + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .endMetadata() + .build(); - server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=controller-uid%3Duid") - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addNewOwnerReference() - .withUid("uid") - .withController(true) - .endOwnerReference() - .endMetadata() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endItem() - .build() - ).once(); + client.pods().inNamespace(NAMESPACE).resource(pod).create(); server.expect().get() .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") @@ -642,44 +612,14 @@ void test_getPeonPodWithRetries_withoutPod_noRestartForBlacklistedEvent_raisesKu @Test void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() { - server.expect().get() - .withPath("/apis/batch/v1/namespaces/namespace/jobs/" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new JobBuilder() - .withNewMetadata() - .withName(KUBERNETES_JOB_NAME) - .withUid("uid") - .endMetadata() - .withNewSpec() - .withNewTemplate() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endTemplate() - .endSpec() - .build() - ).once(); + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .endMetadata() + .build(); - server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=controller-uid%3Duid") - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addNewOwnerReference() - .withUid("uid") - .withController(true) - .endOwnerReference() - .endMetadata() - .withNewSpec() - .addNewContainer() - .withName("main") - .endContainer() - .endSpec() - .endItem() - .build() - ).once(); + client.pods().inNamespace(NAMESPACE).resource(pod).create(); server.expect().get() .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") From 7dd0fe898f246d1e5a72b86d8a6c75bf5a69f89b Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 16:48:55 -0500 Subject: [PATCH 26/58] remove some whitespace from the diff. Can be corrected in a future formatting patch --- .../DirectKubernetesPeonClientTest.java | 78 +++++++++---------- .../DruidPeonClientIntegrationTest.java | 4 +- 2 files changed, 41 insertions(+), 41 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index 143c0dc9a3d7..7d4e1ae7b8dc 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -105,16 +105,16 @@ void test_launchPeonJobAndWaitForStart_withDisappearingPod_throwIllegalStateExce .build(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addToLabels("job-name", JOB_NAME) - .endMetadata() - .endItem() - .build() - ).once(); + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() + .addNewItem() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", JOB_NAME) + .endMetadata() + .endItem() + .build() + ).once(); Assertions.assertThrows( IllegalStateException.class, @@ -283,9 +283,9 @@ void test_getPeonLogs_withJob_returnsInputStreamInOptional() client.pods().inNamespace(NAMESPACE).resource(pod).create(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") - .andReturn(HttpURLConnection.HTTP_OK, "data") - .once(); + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); Optional maybeInputStream = instance.getPeonLogs(new K8sTaskId(TASK_NAME_PREFIX, ID)); Assertions.assertTrue(maybeInputStream.isPresent()); @@ -537,21 +537,21 @@ void test_getPeonPod_withoutPod_returnsEmptyOptional() void test_getPeonPodWithRetries_withPod_returnsPod() { server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder().build()) - .once(); + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder().build()) + .once(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) - .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() - .addNewItem() - .withNewMetadata() - .withName(POD_NAME) - .addToLabels("job-name", KUBERNETES_JOB_NAME) - .endMetadata() - .endItem() - .build() - ).once(); + .withPath("/api/v1/namespaces/namespace/pods?labelSelector=job-name%3D" + KUBERNETES_JOB_NAME) + .andReturn(HttpURLConnection.HTTP_OK, new PodListBuilder() + .addNewItem() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .endMetadata() + .endItem() + .build() + ).once(); Pod pod = instance.getPeonPodWithRetries(clientApi.getClient(), new K8sTaskId(TASK_NAME_PREFIX, ID).getK8sJobName(), 0, 2); @@ -578,10 +578,10 @@ void test_getPeonPodWithRetries_withoutPod_noRestartForBlacklistedEvent_raisesKu String blacklistedMessage = DruidK8sConstants.BLACKLISTED_PEON_POD_ERROR_MESSAGES.get(0); final String eventsPath = "/api/v1/namespaces/namespace/events?fieldSelector=" + - "involvedObject.name%3D" + k8sJobName + - "%2CinvolvedObject.namespace%3D" + NAMESPACE + - "%2CinvolvedObject.kind%3DJob" + - "%2CinvolvedObject.apiVersion%3Dbatch%2Fv1"; + "involvedObject.name%3D" + k8sJobName + + "%2CinvolvedObject.namespace%3D" + NAMESPACE + + "%2CinvolvedObject.kind%3DJob" + + "%2CinvolvedObject.apiVersion%3Dbatch%2Fv1"; server.expect().get() .withPath(eventsPath) @@ -622,9 +622,9 @@ void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() client.pods().inNamespace(NAMESPACE).resource(pod).create(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") - .andReturn(HttpURLConnection.HTTP_OK, "data") - .once(); + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); Optional maybeLogWatch = instance.getPeonLogWatcher(new K8sTaskId(TASK_NAME_PREFIX, ID)); Assertions.assertTrue(maybeLogWatch.isPresent()); @@ -684,9 +684,9 @@ void test_createK8sJobWithRetries_withNonRetryableException_failsImmediately() // Return 403 Forbidden - this is not a retryable exception server.expect().post() - .withPath(jobPath) - .andReturn(HttpURLConnection.HTTP_FORBIDDEN, "Forbidden: insufficient permissions") - .once(); + .withPath(jobPath) + .andReturn(HttpURLConnection.HTTP_FORBIDDEN, "Forbidden: insufficient permissions") + .once(); // Should fail immediately without retries DruidException e = Assertions.assertThrows( @@ -711,9 +711,9 @@ void test_createK8sJobWithRetries_withJobAlreadyExists_succeedsGracefully() // Return 409 Conflict - job already exists server.expect().post() - .withPath(jobPath) - .andReturn(HttpURLConnection.HTTP_CONFLICT, "Job already exists") - .once(); + .withPath(jobPath) + .andReturn(HttpURLConnection.HTTP_CONFLICT, "Job already exists") + .once(); // Should succeed gracefully without throwing exception Assertions.assertDoesNotThrow( diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 44575b1cbcdc..2dee4d3d9617 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -112,8 +112,8 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception Task task = K8sTestUtils.getTask(); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("default") - .build(); + .withNamespace("default") + .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( k8sClient, config, From baf50159b360153e8aa6da7bdd5bacabc3ad7f31 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 16:51:26 -0500 Subject: [PATCH 27/58] one more whitespace cleanup --- .../common/DirectKubernetesPeonClientTest.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index 7d4e1ae7b8dc..a39c2518fb9f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -283,9 +283,9 @@ void test_getPeonLogs_withJob_returnsInputStreamInOptional() client.pods().inNamespace(NAMESPACE).resource(pod).create(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") - .andReturn(HttpURLConnection.HTTP_OK, "data") - .once(); + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); Optional maybeInputStream = instance.getPeonLogs(new K8sTaskId(TASK_NAME_PREFIX, ID)); Assertions.assertTrue(maybeInputStream.isPresent()); @@ -622,9 +622,9 @@ void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() client.pods().inNamespace(NAMESPACE).resource(pod).create(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") - .andReturn(HttpURLConnection.HTTP_OK, "data") - .once(); + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); Optional maybeLogWatch = instance.getPeonLogWatcher(new K8sTaskId(TASK_NAME_PREFIX, ID)); Assertions.assertTrue(maybeLogWatch.isPresent()); @@ -711,9 +711,9 @@ void test_createK8sJobWithRetries_withJobAlreadyExists_succeedsGracefully() // Return 409 Conflict - job already exists server.expect().post() - .withPath(jobPath) - .andReturn(HttpURLConnection.HTTP_CONFLICT, "Job already exists") - .once(); + .withPath(jobPath) + .andReturn(HttpURLConnection.HTTP_CONFLICT, "Job already exists") + .once(); // Should succeed gracefully without throwing exception Assertions.assertDoesNotThrow( From 4109b6c74fe9b4d644d7dab6dd8e61ddcdbd3714 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 16:57:44 -0500 Subject: [PATCH 28/58] more diff cleanup --- .../common/DirectKubernetesPeonClientTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index a39c2518fb9f..3ae4c7ea8d8a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -283,9 +283,9 @@ void test_getPeonLogs_withJob_returnsInputStreamInOptional() client.pods().inNamespace(NAMESPACE).resource(pod).create(); server.expect().get() - .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") - .andReturn(HttpURLConnection.HTTP_OK, "data") - .once(); + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); Optional maybeInputStream = instance.getPeonLogs(new K8sTaskId(TASK_NAME_PREFIX, ID)); Assertions.assertTrue(maybeInputStream.isPresent()); @@ -684,9 +684,9 @@ void test_createK8sJobWithRetries_withNonRetryableException_failsImmediately() // Return 403 Forbidden - this is not a retryable exception server.expect().post() - .withPath(jobPath) - .andReturn(HttpURLConnection.HTTP_FORBIDDEN, "Forbidden: insufficient permissions") - .once(); + .withPath(jobPath) + .andReturn(HttpURLConnection.HTTP_FORBIDDEN, "Forbidden: insufficient permissions") + .once(); // Should fail immediately without retries DruidException e = Assertions.assertThrows( @@ -766,7 +766,7 @@ void test_waitForPodResultWithRetries_withNonRetryableFailure_throwsDruidExcepti // Create the pod in the mock client without IP - it will remain unready client.pods().inNamespace(NAMESPACE).resource(pod).create(); - // Should throw DruidException after timeout due to pod never becoming ready + // Should throw DruidException after failure DruidException e = Assertions.assertThrows( DruidException.class, () -> instance.waitForPodResultWithRetries( From d96cabe95e53ee3b6fac5c33a7e587c82ee20ae0 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 19:07:57 -0500 Subject: [PATCH 29/58] Make another api usage optimization for the caching client. Clean up code and javadocs --- .../common/AbstractKubernetesPeonClient.java | 113 +++++++++++++----- .../DirectKubernetesPeonClientTest.java | 4 +- 2 files changed, 85 insertions(+), 32 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index 63a9dc38ed43..61f3bfa91004 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -19,7 +19,6 @@ package org.apache.druid.k8s.overlord.common; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; @@ -73,47 +72,64 @@ public abstract class AbstractKubernetesPeonClient } /** - * Wait for the K8s job associated with the given taskId to complete, or until the given timeout is reached + * Wait for the K8s job associated with the given taskId to complete, or until the given timeout is reached. + *

    + * If the job completes, the {@link JobResponse} is returned in accordance with success or failure. If the timeout + * is reached before job completion, a FAILED {@link JobResponse} is returned. + *

    * - * @return JobResponse indicating the result of the job + * @param taskId K8sTaskId representing the job to wait for + * @param howLong maximum time to wait + * @param unit time unit for the timeout + * @return {@link JobResponse} indicating the result of the job */ public abstract JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit); /** - * Get the list of all peon jobs in the namespace associated with this client + * Get the list of all peon jobs in the namespace that this client is associated with. + * + * @return List of {@link Job} objects representing the peon jobs */ public abstract List getPeonJobs(); /** * Get the Pod associated with the given job name, if it exists * - * @return an Optional containing the Pod if it exists, or absent if not found + * @return an Optional containing the {@link Pod} if it exists, or absent if not found */ public abstract Optional getPeonPod(String jobName); /** * Get the Job with the given name, if it exists * - * @return an Optional containing the Job if it exists, or absent if not found + * @return an Optional containing the {@link Job} if it exists, or absent if not found */ public abstract Optional getPeonJob(String jobName); /** - * Waits for a pod associated with a job to be created and reach ready state using the pod cache. - * This method polls the informer cache until the pod appears and has a pod IP assigned. + * Waits until a pod for the given job is created and ready to be monitored. + *

    + * A pod can appear and dissapear in some cases, such as the task being canceled. In this case, null is returned and + * the caller should handle accordingly. + *

    * * @param jobName the name of the job whose pod we're waiting for * @param howLong the maximum time to wait * @param timeUnit the time unit for the timeout - * @return the pod in ready state, or null if the pod disappeared after being seen + * @return the {@link Pod} which was waited for or null if the pod appeared and dissapeared * @throws DruidException if the pod never appears within the timeout period */ @Nullable protected abstract Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit); /** - * Launches the given Job. Waits for the associated pod and job to be created and start running. + * Launches the given Kubernetes job for the specified task and waits for its associated pod to be created and ready.:w * + * @param job {@link Job} being launched in k8s + * @param task {@link Task} indexing task associated with the underlying job + * @param howLong maximum time to wait for the pod to be created and ready to monitor + * @param timeUnit time unit for the timeout + * @return the {@link Pod} associated with the launched job once it is created and ready */ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) { @@ -131,29 +147,44 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn // Evaluate result of job launch if (result == null) { throw new ISE( - "K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", + "K8s pod for the task[%s] appeared and disappeared. It can happen if the task was canceled", task.getId() ); } - log.info("Pod for job[%s] is in state [%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); + log.info("Pod for job[%s] is in state[%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); long duration = System.currentTimeMillis() - start; emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); return result; } + /** + * Deletes the Kubernetes job associated with the given taskId. + *

    + * If the debugJobs flag is set to true, the job will not be deleted and a log message will be emitted instead. + *

    + * + * @return true if the job was deleted successfully or debugJobs is true, false if the job did not exist + */ public boolean deletePeonJob(K8sTaskId taskId) { if (!debugJobs) { + Optional maybeJob = getPeonJob(taskId.getK8sJobName()); + if (!maybeJob.isPresent()) { + log.info("Asked to delete a k8s job[%s] for task[%s] that does not exist?", taskId.getK8sJobName(), taskId.getOriginalTaskId()); + return false; + } + Job job = maybeJob.get(); + Boolean result = clientApi.executeRequest(client -> !client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .delete().isEmpty()); + .v1() + .jobs() + .inNamespace(namespace) + .resource(job) + .delete().isEmpty()); if (result) { - log.info("Cleaned up k8s job: %s", taskId); + log.info("Deleted k8s job[%s] for task[%s]", taskId.getK8sJobName(), taskId.getOriginalTaskId()); } else { - log.info("K8s job does not exist: %s", taskId); + log.info("Asked to delete a k8s job[%s] for task[%s] that does not exist?", taskId.getK8sJobName(), taskId.getOriginalTaskId()); } return result; } else { @@ -162,9 +193,16 @@ public boolean deletePeonJob(K8sTaskId taskId) } } + /** + * Get a LogWatch for the peon pod associated with the given taskId. Create it if it does not already exist. + *

    + * Any issues creating the LogWatch will be logged and an absent Optional will be returned. + *

    + * + * @return an Optional containing the {@link LogWatch} if it exists or was created. + */ public Optional getPeonLogWatcher(K8sTaskId taskId) { - // First, get the pod and if it exists set up log watching Optional maybePod = getPeonPod(taskId.getK8sJobName()); if (!maybePod.isPresent()) { log.debug("Pod for job[%s] not found in cache, cannot watch logs", taskId.getK8sJobName()); @@ -176,7 +214,6 @@ public Optional getPeonLogWatcher(K8sTaskId taskId) KubernetesClient k8sClient = clientApi.getClient(); try { - // Use resource() to pass the pod object directly, avoiding any lookups LogWatch logWatch = k8sClient.pods() .inNamespace(namespace) .resource(pod) @@ -193,9 +230,16 @@ public Optional getPeonLogWatcher(K8sTaskId taskId) } } + /** + * Get an InputStream for the logs of the peon pod associated with the given taskId. + *

    + * Any issues creating the InputStream will be logged and an absent Optional will be returned. + *

    + * + * @return an Optional containing the {@link InputStream} if the pod exists and logs could be streamed, or absent otherwise + */ public Optional getPeonLogs(K8sTaskId taskId) { - // First, get the pod from cache to avoid unnecessary API calls to look up the job and pod Optional maybePod = getPeonPod(taskId.getK8sJobName()); if (!maybePod.isPresent()) { log.debug("Pod for job[%s] not found in cache, cannot stream logs", taskId.getK8sJobName()); @@ -207,7 +251,6 @@ public Optional getPeonLogs(K8sTaskId taskId) KubernetesClient k8sClient = clientApi.getClient(); try { - // Use resource() to pass the pod object directly, avoiding any lookups InputStream logStream = k8sClient.pods() .inNamespace(namespace) .resource(pod) @@ -219,32 +262,42 @@ public Optional getPeonLogs(K8sTaskId taskId) return Optional.of(logStream); } catch (Exception e) { - log.error(e, "Error streaming logs from task: %s, pod: %s", taskId, podName); + log.error(e, "Error streaming logs for pod[%s] associated with task[%s]", podName, taskId.getOriginalTaskId()); return Optional.absent(); } } + /** + * Delete completed k8s jobs older than the specified time duration. + * + * @return the number of k8s jobs deleted + */ public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit) { AtomicInteger numDeleted = new AtomicInteger(); return clientApi.executeRequest(client -> { List jobs = getJobsToCleanup(getPeonJobs(), howFarBack, timeUnit); - jobs.forEach(x -> { + jobs.forEach(job -> { if (!client.batch() .v1() .jobs() .inNamespace(namespace) - .withName(x.getMetadata().getName()) + .resource(job) .delete().isEmpty()) { numDeleted.incrementAndGet(); } else { - log.error("Failed to delete job %s", x.getMetadata().getName()); + log.error("Failed to delete k8s job[%s] during completed job cleanup", job.getMetadata().getName()); } }); return numDeleted.get(); }); } + /** + * Get the list of jobs to clean up based on their completion time. + * + * @return List of {@link Job} objects that are ready for cleanup + */ private List getJobsToCleanup(List candidates, long howFarBack, TimeUnit timeUnit) { List toDelete = new ArrayList<>(); @@ -286,8 +339,7 @@ public void createK8sJobWithRetries(Job job) * @param maxTries maximum total number of retry attempts * @throws DruidException if job creation fails after all retry attempts or encounters non-retryable errors */ - @VisibleForTesting - void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, int maxTries) + private void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, int maxTries) { try { RetryUtils.retry( @@ -325,7 +377,8 @@ void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, i * and whether it contains a specific message substring, if applicable. *

    * We have experienced connections in the pool being closed by the server-side but remaining in the pool. These issues - * should be safe to retry in many cases. + * should be safe to retry because even when making mutable calls to create jobs, the k8s control plane API has + * gaurds in place preventind duplicate jobs with same job name. */ protected boolean isRetryableTransientConnectionPoolException(Throwable e) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index 3ae4c7ea8d8a..f4ddb3f63efb 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -691,7 +691,7 @@ void test_createK8sJobWithRetries_withNonRetryableException_failsImmediately() // Should fail immediately without retries DruidException e = Assertions.assertThrows( DruidException.class, - () -> instance.createK8sJobWithRetries(clientApi.getClient(), job, 0, 5) + () -> instance.createK8sJobWithRetries(job) ); // Verify the error message contains our job name @@ -717,7 +717,7 @@ void test_createK8sJobWithRetries_withJobAlreadyExists_succeedsGracefully() // Should succeed gracefully without throwing exception Assertions.assertDoesNotThrow( - () -> instance.createK8sJobWithRetries(clientApi.getClient(), job, 0, 5) + () -> instance.createK8sJobWithRetries(job) ); } From a7104a75b18ff79bc997a18fc00999f97b19e56b Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 19:08:04 -0500 Subject: [PATCH 30/58] diff cleanup --- .../java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 9155dc124e9d..287ffe5f6682 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -347,7 +347,6 @@ public void start() log.info("Starting K8sTaskRunner..."); // Load tasks from previously running jobs and wait for their statuses to start running. final List> taskStatusActiveList = new ArrayList<>(); - // Get all existing peon jobs. final List peonJobs = client.getPeonJobs(); log.info("Locating [%,d] active tasks.", peonJobs.size()); From e129bd0dec93def33feca0cb24b91470026ff4d3 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 19:14:48 -0500 Subject: [PATCH 31/58] Some better class javadocs for the k8s clients --- .../k8s/overlord/common/AbstractKubernetesPeonClient.java | 7 ++++++- .../k8s/overlord/common/CachingKubernetesPeonClient.java | 7 ++++--- .../k8s/overlord/common/DirectKubernetesPeonClient.java | 7 ++++--- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index 61f3bfa91004..81354145a0f9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -44,7 +44,12 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * Abstract base class for Kubernetes peon clients, providing common functionality for mutable and log related operations + * Abstract base class for Kubernetes peon clients providing shared implementation for mutable operations + * (job creation, deletion) and log-related operations (log streaming, log watching). + *

    + * Subclasses implement read operations (querying job and pod state) with different strategies for + * interacting with the Kubernetes API server. + *

    */ public abstract class AbstractKubernetesPeonClient { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 0e41938a4f90..e2c800f2c80b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -33,10 +33,11 @@ import java.util.concurrent.TimeoutException; /** - * A KubernetesPeonClient implementation that uses cached informers to read Job and Pod state. + * A KubernetesPeonClient implementation that uses shared informers to read Job and Pod state from a local cache. *

    - * This reduces load on the Kubernetes API server by centralizing watches allowing tasks to query cached K8s resource - * information. + * This implementation greatly reduces load on the Kubernetes API server by centralizing watches and allowing + * tasks to query cached resource state instead of making per-task API calls. Mutable operations (job creation, + * deletion) still contact the API server directly. *

    */ public class CachingKubernetesPeonClient extends AbstractKubernetesPeonClient diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java index a251ad1cb098..214cb08058d5 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java @@ -38,10 +38,11 @@ import java.util.concurrent.TimeUnit; /** - * A KubernetesPeonClient implementation that directly queries the Kubernetes API server for Job and Pod state. + * A KubernetesPeonClient implementation that directly queries the Kubernetes API server for all read and write + * operations on a per-task basis. *

    - * This implementation does not use caching and may put more load on the Kubernetes API server when many tasks - * are running. + * This implementation does not use caching and may put more load on the Kubernetes API server compared to + * {@link CachingKubernetesPeonClient}, especially when many tasks are running concurrently. *

    */ public class DirectKubernetesPeonClient extends AbstractKubernetesPeonClient From c5cf20842dbdc025482511d7a9b06ffd6889a424 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 19:50:57 -0500 Subject: [PATCH 32/58] logging and comment cleanup --- .../common/CachingKubernetesPeonClient.java | 38 +++++++++++++------ .../common/DruidKubernetesClient.java | 3 ++ .../CachingKubernetesPeonClientTest.java | 14 +------ 3 files changed, 32 insertions(+), 23 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index e2c800f2c80b..9d62f637f02b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -76,7 +76,14 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time long jobMustBeSeenBy = startTime + (clientApi.getInformerResyncPeriodMillis() * 2); boolean jobSeenInCache = false; + // Set up to watch for job changes CompletableFuture jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); + + // We will loop until the full timeout is reached if the job is seen in cache. If the job does not show up in the cache we will exit earlier. + // In this loop we first check the cache to see if our job is there and complete. This avoids missing notifications that happened before we set up the watch. + // If the job is not complete we wait for a notification of a job change or a timeout. + // If it is a timeout, we loop back to check the cache again. + // If it is a job change notification, we check the job state and exit if complete, or loop again if still running. do { try { Optional maybeJob = getPeonJob(taskId.getK8sJobName()); @@ -118,7 +125,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time catch (TimeoutException e) { // A timeout here is not a problem, it forces us to loop around and check the cache again. // This prevents the case where we miss a notification and wait forever. - log.debug("Timeout waiting for job change notification for job[%s], checking cache again", taskId.getK8sJobName()); + log.debug("Timeout waiting for job change notification for job[%s]. If full job timeout has not been reached, the job completion wait will continue", taskId.getK8sJobName()); } catch (InterruptedException e) { throw DruidException.defensive(e, "Interrupted waiting for job change notification for job[%s]", taskId.getK8sJobName()); @@ -140,7 +147,7 @@ public List getPeonJobs() } else { return clientApi.executeJobCacheRequest(informer -> informer.getIndexer() - .byIndex("byOverlordNamespace", overlordNamespace)); + .byIndex(DruidKubernetesClient.OVERLORD_NAMESPACE_INDEX, overlordNamespace)); } } @@ -148,7 +155,7 @@ public List getPeonJobs() public Optional getPeonPod(String jobName) { return clientApi.executePodCacheRequest(informer -> { - List pods = informer.getIndexer().byIndex("byJobName", jobName); + List pods = informer.getIndexer().byIndex(DruidKubernetesClient.JOB_NAME_INDEX, jobName); return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); }); } @@ -157,7 +164,7 @@ public Optional getPeonPod(String jobName) public Optional getPeonJob(String jobName) { return clientApi.executeJobCacheRequest(informer -> { - List jobs = informer.getIndexer().byIndex("byJobName", jobName); + List jobs = informer.getIndexer().byIndex(DruidKubernetesClient.JOB_NAME_INDEX, jobName); return jobs.isEmpty() ? Optional.absent() : Optional.of(jobs.get(0)); }); } @@ -168,10 +175,17 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time { long timeoutMs = timeUnit.toMillis(howLong); long startTime = System.currentTimeMillis(); - String podName = "unknown"; boolean podSeenInCache = false; + + // Set up to watch for pod changes CompletableFuture podFuture = clientApi.getEventNotifier().waitForPodChange(jobName); + + // We will loop until the specified timeout is reached, or we see the pod become ready, whichever comes first. + // We eagerly check the cache first to avoid missing notifications that happened before we set up the watch. + // If the pod is not ready we wait for a notification of a pod change or a timeout. + // If it is a timeout, we loop back to check the cache again (if there is time) + // If it is a pod change notification, we check the pod state and exit if ready, or loop again if still not ready. do { try { // First check to see if pod is already in cache and ready in case our completion future started after the update event fired @@ -197,7 +211,8 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time podFuture = clientApi.getEventNotifier().waitForPodChange(jobName); log.debug("Received pod[%s] change notification for job[%s]", podName, jobName); if (pod == null) { - throw DruidException.defensive("Pod[%s] for job[%s] is null. This is unusual. Investigate Druid and k8s logs.", podName, jobName); + log.warn("Pod[%s] for job[%s] is null. This is unusual. Investigate Druid and k8s logs.", podName, jobName); + return null; } else { podSeenInCache = true; podName = pod.getMetadata().getName(); @@ -212,7 +227,7 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time catch (TimeoutException e) { // A timeout here is not a problem, it forces us to loop around and check the cache again. // This prevents the case where we miss a notification and wait forever. - log.debug("Timeout waiting for pod change notification for job[%s], checking cache again", jobName); + log.debug("Timeout waiting for pod change notification for job[%s], If full timeout has not been reached, the pod startup wait will continue", jobName); } catch (InterruptedException e) { throw DruidException.defensive(e, "Interrupted waiting for pod change notification for job[%s]", jobName); @@ -222,13 +237,12 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time } } while (System.currentTimeMillis() - startTime < timeoutMs); - // Timeout if (podSeenInCache) { - log.warn("Timeout waiting for pod[%s] for job[%s] to become ready", podName, jobName); - return null; + log.warn("Timeout waiting for pod[%s] for job[%s] to become ready after it was created", podName, jobName); } else { - throw DruidException.defensive("Timeout waiting for pod for job[%s] to be created", jobName); + log.warn("Timeout waiting for pod for job[%s] to be created", jobName); } + return null; } /** @@ -252,6 +266,7 @@ private JobResponse determineJobResponse(Job job) Integer active = job.getStatus().getActive(); Integer succeeded = job.getStatus().getSucceeded(); Integer failed = job.getStatus().getFailed(); + if ((active == null || active == 0) && (succeeded != null || failed != null)) { if (succeeded != null && succeeded > 0) { log.info("K8s job[%s] completed successfully", job.getMetadata().getName()); @@ -262,6 +277,7 @@ private JobResponse determineJobResponse(Job job) } } } + log.debug("K8s job[%s] is still active.", job.getMetadata().getName()); return new JobResponse(job, PeonPhase.RUNNING); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 99a98ac95916..21278462dc95 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -40,6 +40,9 @@ public class DruidKubernetesClient implements KubernetesClientApi { private static final EmittingLogger log = new EmittingLogger(DruidKubernetesClient.class); + public static final String JOB_NAME_INDEX = "byJobName"; + public static final String OVERLORD_NAMESPACE_INDEX = "byOverlordNamespace"; + public static final String ENABLE_INFORMERS_KEY = "druid.k8s.informers.enabled"; public static final String INFORMER_RESYNC_PERIOD_MS_KEY = "druid.k8s.informers.resyncPeriodMs"; private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 300000L; // 5 minutes diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index abd35ff3b184..1cd89263668a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -368,21 +368,11 @@ public void test_waitUntilPeonPodCreatedAndReady_timeoutWhenPodNotReady() throws } @Test - public void test_waitUntilPeonPodCreatedAndReady_throwsWhenPodNeverCreated() throws Exception + public void test_waitUntilPeonPodCreatedAndReady_returnNullWhenPodNeverCreated() throws Exception { clientApi.waitForSync(); - // Wait for pod that is never created with short timeout - Exception exception = Assertions.assertThrows( - RuntimeException.class, - () -> peonClient.waitUntilPeonPodCreatedAndReady(JOB_NAME, 1, TimeUnit.SECONDS) - ); - - // Should throw DruidException about timeout waiting for pod creation - Assertions.assertTrue( - exception.getMessage().contains("Timeout waiting for pod") || - exception.getCause().getMessage().contains("Timeout waiting for pod") - ); + Assertions.assertNull(peonClient.waitUntilPeonPodCreatedAndReady(JOB_NAME, 1, TimeUnit.SECONDS)); } @Test From 9bf9c4a0147bb2c7a0b9ee1438241640321e891d Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 20:17:56 -0500 Subject: [PATCH 33/58] DruidKubernetesClient tidy up --- .../common/DruidKubernetesClient.java | 38 ++++++++++++++++--- .../KubernetesTaskRunnerFactoryTest.java | 9 ++++- 2 files changed, 41 insertions(+), 6 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index 21278462dc95..f2adb26119e5 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -19,6 +19,7 @@ package org.apache.druid.k8s.overlord.common; +import com.google.common.base.Preconditions; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.Config; @@ -60,9 +61,14 @@ public DruidKubernetesClient(DruidKubernetesHttpClientFactory httpClientFactory, .withConfig(kubernetesClientConfig) .build(); + // It is required that the config declares whether informers are enabled or not + Preconditions.checkNotNull(kubernetesClientConfig.getAdditionalProperties().get(ENABLE_INFORMERS_KEY), + "Kubernetes client config must contain property [%s]", + ENABLE_INFORMERS_KEY); + informerResyncPeriodMillis = (long) kubernetesClientConfig .getAdditionalProperties().getOrDefault(INFORMER_RESYNC_PERIOD_MS_KEY, DEFAULT_INFORMER_RESYNC_PERIOD_MS); - if ((boolean) kubernetesClientConfig.getAdditionalProperties().getOrDefault(ENABLE_INFORMERS_KEY, false)) { + if ((boolean) kubernetesClientConfig.getAdditionalProperties().get(ENABLE_INFORMERS_KEY)) { this.eventNotifier = new KubernetesResourceEventNotifier(); this.podInformer = setupPodInformer(kubernetesClient.getNamespace()); this.jobInformer = setupJobInformer(kubernetesClient.getNamespace()); @@ -113,12 +119,18 @@ public KubernetesClient getClient() @Override public SharedIndexInformer getPodInformer() { + if (podInformer == null) { + throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); + } return podInformer; } @Override public SharedIndexInformer getJobInformer() { + if (jobInformer == null) { + throw DruidException.defensive("Job informer is not initialized, caching is disabled"); + } return jobInformer; } @@ -131,6 +143,13 @@ public KubernetesResourceEventNotifier getEventNotifier() return eventNotifier; } + /** + * Sets up a shared informer to watch and cache Pod resources in the specified namespace. + *

    + * Registers event handlers for pod add/update/delete events and creates a custom index by job-name + * for efficient pod lookup by job. + *

    + */ private SharedIndexInformer setupPodInformer(String namespace) { SharedIndexInformer podInformer = @@ -174,12 +193,19 @@ public void onDelete(Pod pod, boolean deletedFinalStateUnknown) }; Map>> customPodIndexers = new HashMap<>(); - customPodIndexers.put("byJobName", jobNameIndexer); + customPodIndexers.put(JOB_NAME_INDEX, jobNameIndexer); podInformer.addIndexers(customPodIndexers); return podInformer; } + /** + * Sets up a shared informer to watch and cache Job resources in the specified namespace. + *

    + * Registers event handlers for job add/update/delete events and creates custom indexes by job-name + * and overlord-namespace for efficient job lookup and filtering. + *

    + */ private SharedIndexInformer setupJobInformer(String namespace) { SharedIndexInformer jobInformer = @@ -232,20 +258,22 @@ public void onDelete(Job job, boolean deletedFinalStateUnknown) }; Map>> customJobIndexers = new HashMap<>(); - customJobIndexers.put("byOverlordNamespace", overlordNamespaceIndexer); - customJobIndexers.put("byJobName", jobNameIndexer); + customJobIndexers.put(OVERLORD_NAMESPACE_INDEX, overlordNamespaceIndexer); + customJobIndexers.put(JOB_NAME_INDEX, jobNameIndexer); jobInformer.addIndexers(customJobIndexers); return jobInformer; } + /** + * Utility method to only notify pod changes for pods that are part of indexing jobs. + */ private void notifyPodChange(Pod pod) { if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { String jobName = pod.getMetadata().getLabels().get("job-name"); if (jobName != null) { - // Prevents us from trying to notify pod changes that are not indexing jobs eventNotifier.notifyPodChange(jobName, pod); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 6849f9a0ecc7..9f70f6a5af05 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.Config; import io.fabric8.kubernetes.client.ConfigBuilder; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; @@ -57,8 +58,14 @@ public void setup() .withCapacity(1) .build(); taskLogs = new NoopTaskLogs(); + + Config config = new ConfigBuilder().build(); + config.setAdditionalProperty( + DruidKubernetesClient.ENABLE_INFORMERS_KEY, + kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers()); + druidKubernetesClient = - new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), new ConfigBuilder().build()); + new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), config); taskAdapter = new TestTaskAdapter(); } From 6b51c9b36a397a969f5e3fcfa683eb664687f802 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 20:26:57 -0500 Subject: [PATCH 34/58] javadoc link add --- .../overlord/common/KubernetesResourceEventNotifier.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java index e360ad4770a0..58a704e5d4d6 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java @@ -30,8 +30,10 @@ /** * Manages event notifications for Kubernetes resources (Jobs and Pods). - * Allows tasks to wait for specific resource changes without polling, - * improving efficiency and responsiveness. + *

    + * Allows tasks to wait for specific resource changes without polling, improving efficiency and responsiveness. + * Crtical component of {@link CachingKubernetesPeonClient} functionality. + *

    */ public class KubernetesResourceEventNotifier { From ab1cd891c34c4f5c8354d20472685e7b0ab0e27c Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 24 Oct 2025 22:25:05 -0500 Subject: [PATCH 35/58] Use background propagation policy when deleting jobs to lessen load on k8s api --- .../common/AbstractKubernetesPeonClient.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index 81354145a0f9..8a21882763ca 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -20,6 +20,7 @@ package org.apache.druid.k8s.overlord.common; import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.DeletionPropagation; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; @@ -173,18 +174,12 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn public boolean deletePeonJob(K8sTaskId taskId) { if (!debugJobs) { - Optional maybeJob = getPeonJob(taskId.getK8sJobName()); - if (!maybeJob.isPresent()) { - log.info("Asked to delete a k8s job[%s] for task[%s] that does not exist?", taskId.getK8sJobName(), taskId.getOriginalTaskId()); - return false; - } - Job job = maybeJob.get(); - Boolean result = clientApi.executeRequest(client -> !client.batch() .v1() .jobs() .inNamespace(namespace) - .resource(job) + .withName(taskId.getK8sJobName()) + .withPropagationPolicy(DeletionPropagation.BACKGROUND) .delete().isEmpty()); if (result) { log.info("Deleted k8s job[%s] for task[%s]", taskId.getK8sJobName(), taskId.getOriginalTaskId()); @@ -287,7 +282,8 @@ public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit) .v1() .jobs() .inNamespace(namespace) - .resource(job) + .withName(job.getMetadata().getName()) + .withPropagationPolicy(DeletionPropagation.BACKGROUND) .delete().isEmpty()) { numDeleted.incrementAndGet(); } else { From 9495e3c90c4af4cfc290e74c86d7ddedba98c4a5 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 28 Oct 2025 10:57:15 -0500 Subject: [PATCH 36/58] fix an npe and add a test to caching client --- .../common/AbstractKubernetesPeonClient.java | 2 +- .../common/CachingKubernetesPeonClient.java | 2 +- .../CachingKubernetesPeonClientTest.java | 26 +++++++++++++++++++ .../DirectKubernetesPeonClientTest.java | 17 ++++++++++++ 4 files changed, 45 insertions(+), 2 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java index 8a21882763ca..a3260e40c62f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java @@ -216,7 +216,7 @@ public Optional getPeonLogWatcher(K8sTaskId taskId) try { LogWatch logWatch = k8sClient.pods() .inNamespace(namespace) - .resource(pod) + .withName(podName) .inContainer("main") .watchLog(); if (logWatch == null) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 9d62f637f02b..10e90052d9dc 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -253,7 +253,7 @@ private boolean isPodRunningOrComplete(Pod pod) // I could not find constants for Pod phases in fabric8, so hardcoding them here. // They are documented here: https://kubernetes.io/docs/concepts/workloads/pods/pod-lifecycle/#pod-phase List matchingPhases = List.of("Running", "Succeeded", "Failed"); - return pod.getStatus() != null && + return pod.getStatus() != null && pod.getStatus().getPhase() != null && matchingPhases.contains(pod.getStatus().getPhase()); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index 1cd89263668a..155dfbf1c765 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -25,6 +25,7 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.dsl.LogWatch; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; import io.fabric8.kubernetes.client.server.mock.KubernetesMockServer; import org.apache.druid.java.util.metrics.StubServiceEmitter; @@ -34,6 +35,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.net.HttpURLConnection; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -518,4 +520,28 @@ public void test_waitForPeonJobCompletion_jobDeletedBeforeSeenInCache() throws E // Should timeout or detect job was never seen and return FAILED Assertions.assertEquals(PeonPhase.FAILED, response.getPhase()); } + + @Test + void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() throws InterruptedException + { + K8sTaskId taskId = new K8sTaskId("", "id"); + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", taskId.getK8sJobName()) + .endMetadata() + .build(); + + client.pods().inNamespace(NAMESPACE).resource(pod).create(); + + clientApi.waitForSync(); + + server.expect().get() + .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") + .andReturn(HttpURLConnection.HTTP_OK, "data") + .once(); + + Optional maybeLogWatch = peonClient.getPeonLogWatcher(taskId); + Assertions.assertTrue(maybeLogWatch.isPresent()); + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java index f4ddb3f63efb..559e8281bb7a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java @@ -218,10 +218,27 @@ void test_deletePeonJob_withJob_returnsTrue() Job job = new JobBuilder() .withNewMetadata() .withName(KUBERNETES_JOB_NAME) + .withUid("job-uid-123") + .endMetadata() + .build(); + + Pod pod = new PodBuilder() + .withNewMetadata() + .withName(POD_NAME) + .addToLabels("job-name", KUBERNETES_JOB_NAME) + .addNewOwnerReference() + .withApiVersion("batch/v1") + .withKind("Job") + .withName(KUBERNETES_JOB_NAME) + .withUid("job-uid-123") + .withController(true) + .withBlockOwnerDeletion(true) + .endOwnerReference() .endMetadata() .build(); client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); + client.pods().inNamespace(NAMESPACE).resource(pod).create(); Assertions.assertTrue(instance.deletePeonJob(new K8sTaskId(TASK_NAME_PREFIX, ID))); } From b04e779ebe93fd2260a01f26f417aca5283ebc0f Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 5 Dec 2025 13:10:40 +0530 Subject: [PATCH 37/58] Remove AbstractK8sClient, rename DirectClient --- .../k8s/overlord/KubernetesPeonLifecycle.java | 6 +- .../KubernetesPeonLifecycleFactory.java | 6 +- .../k8s/overlord/KubernetesTaskRunner.java | 6 +- .../overlord/KubernetesTaskRunnerFactory.java | 9 +- .../common/AbstractKubernetesPeonClient.java | 400 ------------- .../common/CachingKubernetesPeonClient.java | 4 +- .../common/DirectKubernetesPeonClient.java | 294 ---------- .../overlord/common/KubernetesPeonClient.java | 531 ++++++++++++++++++ .../overlord/KubernetesPeonLifecycleTest.java | 4 +- .../overlord/KubernetesTaskRunnerTest.java | 4 +- ...est.java => KubernetesPeonClientTest.java} | 16 +- .../DruidPeonClientIntegrationTest.java | 6 +- 12 files changed, 560 insertions(+), 726 deletions(-) delete mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java delete mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java rename extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/{DirectKubernetesPeonClientTest.java => KubernetesPeonClientTest.java} (97%) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index 88062e9e87c8..d15fa3963237 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -36,10 +36,10 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; +import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @@ -95,7 +95,7 @@ protected enum State private final K8sTaskId taskId; private final TaskLogs taskLogs; private final Task task; - private final AbstractKubernetesPeonClient kubernetesClient; + private final KubernetesPeonClient kubernetesClient; private final ObjectMapper mapper; private final TaskStateListener stateListener; private final SettableFuture taskStartedSuccessfullyFuture; @@ -109,7 +109,7 @@ protected enum State protected KubernetesPeonLifecycle( Task task, K8sTaskId taskId, - AbstractKubernetesPeonClient kubernetesClient, + KubernetesPeonClient kubernetesClient, TaskLogs taskLogs, ObjectMapper mapper, TaskStateListener stateListener, diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java index ebb3e6737010..8bd7db2ebf3e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java @@ -21,19 +21,19 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.K8sTaskId; +import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory { - private final AbstractKubernetesPeonClient client; + private final KubernetesPeonClient client; private final TaskLogs taskLogs; private final ObjectMapper mapper; private final long logSaveTimeoutMs; public KubernetesPeonLifecycleFactory( - AbstractKubernetesPeonClient client, + KubernetesPeonClient client, TaskLogs taskLogs, ObjectMapper mapper, long logSaveTimeoutMs diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 287ffe5f6682..51913d13a71e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -53,8 +53,8 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.K8sTaskId; +import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.TaskLogStreamer; @@ -108,7 +108,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner protected final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); protected final TaskAdapter adapter; - private final AbstractKubernetesPeonClient client; + private final KubernetesPeonClient client; private final KubernetesTaskRunnerConfig config; private final ListeningExecutorService exec; private final HttpClient httpClient; @@ -120,7 +120,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner public KubernetesTaskRunner( TaskAdapter adapter, KubernetesTaskRunnerConfig config, - AbstractKubernetesPeonClient client, + KubernetesPeonClient client, HttpClient httpClient, PeonLifecycleFactory peonLifecycleFactory, ServiceEmitter emitter diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index b660dcc27247..99a1b9012d90 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -26,10 +26,9 @@ import org.apache.druid.indexing.overlord.TaskRunnerFactory; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.CachingKubernetesPeonClient; -import org.apache.druid.k8s.overlord.common.DirectKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; +import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.TaskLogs; @@ -72,7 +71,7 @@ public KubernetesTaskRunnerFactory( @Override public KubernetesTaskRunner build() { - AbstractKubernetesPeonClient peonClient; + KubernetesPeonClient peonClient; boolean enableCache = kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers(); boolean useOverlordNamespace = adapterTypeAllowingTasksInDifferentNamespaces.contains(taskAdapter.getAdapterType()); @@ -95,7 +94,7 @@ public KubernetesTaskRunner build() } } else { if (useOverlordNamespace) { - peonClient = new DirectKubernetesPeonClient( + peonClient = new KubernetesPeonClient( druidKubernetesClient, kubernetesTaskRunnerConfig.getNamespace(), kubernetesTaskRunnerConfig.getOverlordNamespace(), @@ -103,7 +102,7 @@ public KubernetesTaskRunner build() emitter ); } else { - peonClient = new DirectKubernetesPeonClient( + peonClient = new KubernetesPeonClient( druidKubernetesClient, kubernetesTaskRunnerConfig.getNamespace(), kubernetesTaskRunnerConfig.isDebugJobs(), diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java deleted file mode 100644 index a3260e40c62f..000000000000 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/AbstractKubernetesPeonClient.java +++ /dev/null @@ -1,400 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.k8s.overlord.common; - -import com.google.common.base.Optional; -import io.fabric8.kubernetes.api.model.DeletionPropagation; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.batch.v1.Job; -import io.fabric8.kubernetes.client.KubernetesClient; -import io.fabric8.kubernetes.client.KubernetesClientException; -import io.fabric8.kubernetes.client.dsl.LogWatch; -import io.vertx.core.http.HttpClosedException; -import org.apache.druid.error.DruidException; -import org.apache.druid.indexing.common.task.IndexTaskUtils; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.RetryUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; - -import javax.annotation.Nullable; -import java.io.InputStream; -import java.sql.Timestamp; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Abstract base class for Kubernetes peon clients providing shared implementation for mutable operations - * (job creation, deletion) and log-related operations (log streaming, log watching). - *

    - * Subclasses implement read operations (querying job and pod state) with different strategies for - * interacting with the Kubernetes API server. - *

    - */ -public abstract class AbstractKubernetesPeonClient -{ - protected static final EmittingLogger log = new EmittingLogger(AbstractKubernetesPeonClient.class); - - protected final KubernetesClientApi clientApi; - protected final String namespace; - protected final String overlordNamespace; - private final boolean debugJobs; - private final ServiceEmitter emitter; - - AbstractKubernetesPeonClient( - KubernetesClientApi clientApi, - String namespace, - String overlordNamespace, - boolean debugJobs, - ServiceEmitter emitter - ) - { - this.clientApi = clientApi; - this.namespace = namespace; - this.overlordNamespace = overlordNamespace; - this.debugJobs = debugJobs; - this.emitter = emitter; - } - - /** - * Wait for the K8s job associated with the given taskId to complete, or until the given timeout is reached. - *

    - * If the job completes, the {@link JobResponse} is returned in accordance with success or failure. If the timeout - * is reached before job completion, a FAILED {@link JobResponse} is returned. - *

    - * - * @param taskId K8sTaskId representing the job to wait for - * @param howLong maximum time to wait - * @param unit time unit for the timeout - * @return {@link JobResponse} indicating the result of the job - */ - public abstract JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit); - - /** - * Get the list of all peon jobs in the namespace that this client is associated with. - * - * @return List of {@link Job} objects representing the peon jobs - */ - public abstract List getPeonJobs(); - - /** - * Get the Pod associated with the given job name, if it exists - * - * @return an Optional containing the {@link Pod} if it exists, or absent if not found - */ - public abstract Optional getPeonPod(String jobName); - - /** - * Get the Job with the given name, if it exists - * - * @return an Optional containing the {@link Job} if it exists, or absent if not found - */ - public abstract Optional getPeonJob(String jobName); - - /** - * Waits until a pod for the given job is created and ready to be monitored. - *

    - * A pod can appear and dissapear in some cases, such as the task being canceled. In this case, null is returned and - * the caller should handle accordingly. - *

    - * - * @param jobName the name of the job whose pod we're waiting for - * @param howLong the maximum time to wait - * @param timeUnit the time unit for the timeout - * @return the {@link Pod} which was waited for or null if the pod appeared and dissapeared - * @throws DruidException if the pod never appears within the timeout period - */ - @Nullable - protected abstract Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit); - - /** - * Launches the given Kubernetes job for the specified task and waits for its associated pod to be created and ready.:w - * - * @param job {@link Job} being launched in k8s - * @param task {@link Task} indexing task associated with the underlying job - * @param howLong maximum time to wait for the pod to be created and ready to monitor - * @param timeUnit time unit for the timeout - * @return the {@link Pod} associated with the launched job once it is created and ready - */ - public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) - { - long start = System.currentTimeMillis(); - - // launch job - String jobName = job.getMetadata().getName(); - log.info("Submitting job[%s] for task[%s].", jobName, task.getId()); - createK8sJobWithRetries(job); - log.info("Submitted job[%s] for task[%s]. Waiting for POD to launch and be ready.", jobName, task.getId()); - - // Wait for the Pod to be created and then reach ready state - Pod result = waitUntilPeonPodCreatedAndReady(jobName, howLong, timeUnit); - - // Evaluate result of job launch - if (result == null) { - throw new ISE( - "K8s pod for the task[%s] appeared and disappeared. It can happen if the task was canceled", - task.getId() - ); - } - log.info("Pod for job[%s] is in state[%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); - long duration = System.currentTimeMillis() - start; - emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); - return result; - } - - /** - * Deletes the Kubernetes job associated with the given taskId. - *

    - * If the debugJobs flag is set to true, the job will not be deleted and a log message will be emitted instead. - *

    - * - * @return true if the job was deleted successfully or debugJobs is true, false if the job did not exist - */ - public boolean deletePeonJob(K8sTaskId taskId) - { - if (!debugJobs) { - Boolean result = clientApi.executeRequest(client -> !client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .withPropagationPolicy(DeletionPropagation.BACKGROUND) - .delete().isEmpty()); - if (result) { - log.info("Deleted k8s job[%s] for task[%s]", taskId.getK8sJobName(), taskId.getOriginalTaskId()); - } else { - log.info("Asked to delete a k8s job[%s] for task[%s] that does not exist?", taskId.getK8sJobName(), taskId.getOriginalTaskId()); - } - return result; - } else { - log.info("Not cleaning up job %s due to flag: debugJobs=true", taskId); - return true; - } - } - - /** - * Get a LogWatch for the peon pod associated with the given taskId. Create it if it does not already exist. - *

    - * Any issues creating the LogWatch will be logged and an absent Optional will be returned. - *

    - * - * @return an Optional containing the {@link LogWatch} if it exists or was created. - */ - public Optional getPeonLogWatcher(K8sTaskId taskId) - { - Optional maybePod = getPeonPod(taskId.getK8sJobName()); - if (!maybePod.isPresent()) { - log.debug("Pod for job[%s] not found in cache, cannot watch logs", taskId.getK8sJobName()); - return Optional.absent(); - } - - Pod pod = maybePod.get(); - String podName = pod.getMetadata().getName(); - - KubernetesClient k8sClient = clientApi.getClient(); - try { - LogWatch logWatch = k8sClient.pods() - .inNamespace(namespace) - .withName(podName) - .inContainer("main") - .watchLog(); - if (logWatch == null) { - return Optional.absent(); - } - return Optional.of(logWatch); - } - catch (Exception e) { - log.error(e, "Error watching logs from task: %s, pod: %s", taskId, podName); - return Optional.absent(); - } - } - - /** - * Get an InputStream for the logs of the peon pod associated with the given taskId. - *

    - * Any issues creating the InputStream will be logged and an absent Optional will be returned. - *

    - * - * @return an Optional containing the {@link InputStream} if the pod exists and logs could be streamed, or absent otherwise - */ - public Optional getPeonLogs(K8sTaskId taskId) - { - Optional maybePod = getPeonPod(taskId.getK8sJobName()); - if (!maybePod.isPresent()) { - log.debug("Pod for job[%s] not found in cache, cannot stream logs", taskId.getK8sJobName()); - return Optional.absent(); - } - - Pod pod = maybePod.get(); - String podName = pod.getMetadata().getName(); - - KubernetesClient k8sClient = clientApi.getClient(); - try { - InputStream logStream = k8sClient.pods() - .inNamespace(namespace) - .resource(pod) - .inContainer("main") - .getLogInputStream(); - if (logStream == null) { - return Optional.absent(); - } - return Optional.of(logStream); - } - catch (Exception e) { - log.error(e, "Error streaming logs for pod[%s] associated with task[%s]", podName, taskId.getOriginalTaskId()); - return Optional.absent(); - } - } - - /** - * Delete completed k8s jobs older than the specified time duration. - * - * @return the number of k8s jobs deleted - */ - public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit) - { - AtomicInteger numDeleted = new AtomicInteger(); - return clientApi.executeRequest(client -> { - List jobs = getJobsToCleanup(getPeonJobs(), howFarBack, timeUnit); - jobs.forEach(job -> { - if (!client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(job.getMetadata().getName()) - .withPropagationPolicy(DeletionPropagation.BACKGROUND) - .delete().isEmpty()) { - numDeleted.incrementAndGet(); - } else { - log.error("Failed to delete k8s job[%s] during completed job cleanup", job.getMetadata().getName()); - } - }); - return numDeleted.get(); - }); - } - - /** - * Get the list of jobs to clean up based on their completion time. - * - * @return List of {@link Job} objects that are ready for cleanup - */ - private List getJobsToCleanup(List candidates, long howFarBack, TimeUnit timeUnit) - { - List toDelete = new ArrayList<>(); - long cutOff = System.currentTimeMillis() - timeUnit.toMillis(howFarBack); - candidates.forEach(x -> { - // jobs that are complete - if (x.getStatus().getActive() == null) { - Timestamp timestamp = Timestamp.valueOf(x.getStatus().getCompletionTime()); - if (timestamp.before(new Timestamp(cutOff))) { - toDelete.add(x); - } - } - }); - return toDelete; - } - - public void createK8sJobWithRetries(Job job) - { - clientApi.executeRequest(client -> { - createK8sJobWithRetries(client, job, 5, RetryUtils.DEFAULT_MAX_TRIES); - return null; - }); - } - - /** - * Creates a Kubernetes job with retry logic for transient connection pool exceptions. - *

    - * This method attempts to create the specified job in Kubernetes with built-in retry logic - * for transient connection pool issues. If the job already exists (HTTP 409 conflict), - * the method returns successfully without throwing an exception, assuming the job was - * already submitted by a previous request. - *

    - * The retry logic only applies to transient connection pool exceptions. Other exceptions will cause the method to - * fail immediately. - * - * @param client the Kubernetes client to use for job creation - * @param job the Kubernetes job to create - * @param quietTries number of initial retry attempts without logging warnings - * @param maxTries maximum total number of retry attempts - * @throws DruidException if job creation fails after all retry attempts or encounters non-retryable errors - */ - private void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, int maxTries) - { - try { - RetryUtils.retry( - () -> { - try { - client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .resource(job) - .create(); - return null; - } - catch (KubernetesClientException e) { - if (e.getCode() == 409) { - // Job already exists, return successfully - log.info("K8s job[%s] already exists, skipping creation", job.getMetadata().getName()); - return null; - } - throw e; - } - }, - this::isRetryableTransientConnectionPoolException, quietTries, maxTries - ); - } - catch (Exception e) { - throw DruidException.defensive(e, "Error when creating K8s job[%s]", job.getMetadata().getName()); - } - } - - /** - * Checks if the exception is a potentially transient connection pool exception. - *

    - * This method checks if the exception is one of the known transient connection pool exceptions - * and whether it contains a specific message substring, if applicable. - *

    - * We have experienced connections in the pool being closed by the server-side but remaining in the pool. These issues - * should be safe to retry because even when making mutable calls to create jobs, the k8s control plane API has - * gaurds in place preventind duplicate jobs with same job name. - */ - protected boolean isRetryableTransientConnectionPoolException(Throwable e) - { - if (e instanceof KubernetesClientException) { - return e.getMessage() != null && e.getMessage().contains("Connection was closed"); - } else if (e instanceof HttpClosedException) { - return true; - } - return false; - } - - private void emitK8sPodMetrics(Task task, String metric, long durationMs) - { - ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); - IndexTaskUtils.setTaskDimensions(metricBuilder, task); - emitter.emit(metricBuilder.setMetric(metric, durationMs)); - } -} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 10e90052d9dc..11526ff52e1b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -40,7 +40,7 @@ * deletion) still contact the API server directly. *

    */ -public class CachingKubernetesPeonClient extends AbstractKubernetesPeonClient +public class CachingKubernetesPeonClient extends KubernetesPeonClient { protected static final EmittingLogger log = new EmittingLogger(CachingKubernetesPeonClient.class); @@ -160,7 +160,6 @@ public Optional getPeonPod(String jobName) }); } - @Override public Optional getPeonJob(String jobName) { return clientApi.executeJobCacheRequest(informer -> { @@ -169,7 +168,6 @@ public Optional getPeonJob(String jobName) }); } - @Override @Nullable protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java deleted file mode 100644 index 214cb08058d5..000000000000 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClient.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.k8s.overlord.common; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; -import io.fabric8.kubernetes.api.model.Event; -import io.fabric8.kubernetes.api.model.ObjectReference; -import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.batch.v1.Job; -import io.fabric8.kubernetes.client.KubernetesClient; -import io.fabric8.kubernetes.client.KubernetesClientException; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.RetryUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; - -import javax.annotation.Nullable; -import java.util.List; -import java.util.concurrent.TimeUnit; - -/** - * A KubernetesPeonClient implementation that directly queries the Kubernetes API server for all read and write - * operations on a per-task basis. - *

    - * This implementation does not use caching and may put more load on the Kubernetes API server compared to - * {@link CachingKubernetesPeonClient}, especially when many tasks are running concurrently. - *

    - */ -public class DirectKubernetesPeonClient extends AbstractKubernetesPeonClient -{ - protected static final EmittingLogger log = new EmittingLogger(DirectKubernetesPeonClient.class); - - public DirectKubernetesPeonClient( - KubernetesClientApi clientApi, - String namespace, - String overlordNamespace, - boolean debugJobs, - ServiceEmitter emitter - ) - { - super(clientApi, namespace, overlordNamespace, debugJobs, emitter); - } - - public DirectKubernetesPeonClient( - KubernetesClientApi clientApi, - String namespace, - boolean debugJobs, - ServiceEmitter emitter - ) - { - super(clientApi, namespace, "", debugJobs, emitter); - } - - @Override - public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) - { - return clientApi.executeRequest(client -> { - Job job = client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .waitUntilCondition( - x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null - && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), - howLong, - unit - ); - if (job == null) { - log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); - return new JobResponse(null, PeonPhase.FAILED); - } - if (job.getStatus().getSucceeded() != null) { - return new JobResponse(job, PeonPhase.SUCCEEDED); - } - log.warn("Task %s failed with status %s", taskId, job.getStatus()); - return new JobResponse(job, PeonPhase.FAILED); - }); - } - - @Override - public List getPeonJobs() - { - if (overlordNamespace.isEmpty()) { - return clientApi.executeRequest(client -> client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .list() - .getItems()); - } else { - return clientApi.executeRequest(client -> client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .withLabel( - DruidK8sConstants.OVERLORD_NAMESPACE_KEY, - overlordNamespace - ) - .list() - .getItems()); - } - } - - @Override - public Optional getPeonPod(String jobName) - { - return clientApi.executeRequest(client -> getPeonPod(client, jobName)); - } - - @Override - public Optional getPeonJob(String jobName) - { - return clientApi.executeRequest(client -> getPeonJob(client, jobName)); - } - - @Nullable - @Override - protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) - { - Pod pod = clientApi.executeRequest(client -> getPeonPodWithRetries(client, jobName, 5, RetryUtils.DEFAULT_MAX_TRIES)); - if (pod == null) { - return null; - } - return clientApi.executeRequest(client -> waitForPodResultWithRetries(client, pod, howLong, timeUnit, 5, RetryUtils.DEFAULT_MAX_TRIES)); - - } - - /** - * Waits for a Kubernetes pod to reach a ready state with retry logic for transient connection pool exceptions. - *

    - * This method waits for the specified pod to have a valid status with a pod IP assigned, indicating - * it has been scheduled and is in a ready state. The method includes retry logic to handle transient - * connection pool exceptions that may occur during the wait operation. - *

    - * The method will wait up to the specified timeout for the pod to become ready, and retry the entire wait operation - * if transient connection issues are encountered. - * - * @param client the Kubernetes client to use for pod operations - * @param pod the pod to wait for - * @param howLong the maximum time to wait for the pod to become ready - * @param timeUnit the time unit for the wait timeout - * @param quietTries number of initial retry attempts without logging warnings - * @param maxTries maximum total number of retry attempts - * @return the pod in its ready state, or null if the pod disappeared or wait operation failed - * @throws DruidException if waiting fails after all retry attempts or encounters non-retryable errors - */ - @VisibleForTesting - Pod waitForPodResultWithRetries(KubernetesClient client, Pod pod, long howLong, TimeUnit timeUnit, int quietTries, int maxTries) - { - try { - return RetryUtils.retry( - () -> client.pods() - .inNamespace(namespace) - .withName(pod.getMetadata().getName()) - .waitUntilCondition( - p -> { - if (p == null) { - return true; - } - return p.getStatus() != null && p.getStatus().getPodIP() != null; - }, howLong, timeUnit), - this::isRetryableTransientConnectionPoolException, quietTries, maxTries); - } - catch (Exception e) { - throw DruidException.defensive(e, "Error when waiting for pod[%s] to start", pod.getMetadata().getName()); - } - } - - @VisibleForTesting - Pod getPeonPodWithRetries(KubernetesClient client, String jobName, int quietTries, int maxTries) - { - try { - return RetryUtils.retry( - () -> { - Optional maybePod = getPeonPod(client, jobName); - if (maybePod.isPresent()) { - return maybePod.get(); - } - - // If the pod is missing, we can take a look at job events to discover potential problems with pod creation. - List events = getPeonEvents(client, jobName); - - if (events.isEmpty()) { - throw new KubernetesResourceNotFoundException("K8s pod with label[job-name=%s] not found", jobName); - } else { - Event latestEvent = events.get(events.size() - 1); - throw new KubernetesResourceNotFoundException( - "Job[%s] failed to create pods. Message[%s]", jobName, latestEvent.getMessage()); - } - }, - this::shouldRetryWaitForStartingPeonPod, quietTries, maxTries - ); - } - catch (KubernetesResourceNotFoundException e) { - throw e; - } - catch (Exception e) { - throw DruidException.defensive(e, "Error when looking for K8s pod with label[job-name=%s]", jobName); - } - } - - private Optional getPeonPod(KubernetesClient client, String jobName) - { - List pods = client.pods() - .inNamespace(namespace) - .withLabel("job-name", jobName) - .list() - .getItems(); - return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); - } - - private Optional getPeonJob(KubernetesClient client, String jobName) - { - Job job = client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(jobName) - .get(); - return job == null ? Optional.absent() : Optional.of(job); - } - - /** - * Determines if this exception, specifically when containing Kubernetes job event messages, permits a retry attempt. - *

    - * The method checks the exception message against a predefined list of Kubernetes event messages. - * These substrings, found in {@link DruidK8sConstants#BLACKLISTED_PEON_POD_ERROR_MESSAGES}, - * represent Kubernetes event that indicate a retry for starting the Peon Pod would likely be futile. - */ - private boolean shouldRetryWaitForStartingPeonPod(Throwable e) - { - if (isRetryableTransientConnectionPoolException(e)) { - return true; - } - - if (!(e instanceof KubernetesResourceNotFoundException)) { - return false; - } - - String errorMessage = e.getMessage(); - for (String blacklistedMessage : DruidK8sConstants.BLACKLISTED_PEON_POD_ERROR_MESSAGES) { - if (errorMessage.contains(blacklistedMessage)) { - return false; - } - } - - return true; - } - - private List getPeonEvents(KubernetesClient client, String jobName) - { - ObjectReference objectReference = new ObjectReferenceBuilder() - .withApiVersion("batch/v1") - .withKind("Job") - .withName(jobName) - .withNamespace(this.namespace) - .build(); - - try { - return client.v1() - .events() - .inNamespace(this.namespace) - .withInvolvedObject(objectReference) - .list() - .getItems(); - } - catch (KubernetesClientException e) { - log.warn("Failed to get events for job[%s]; %s", jobName, e.getMessage()); - return List.of(); - } - } -} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java new file mode 100644 index 000000000000..3269780584a6 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -0,0 +1,531 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import io.fabric8.kubernetes.api.model.Event; +import io.fabric8.kubernetes.api.model.ObjectReference; +import io.fabric8.kubernetes.api.model.ObjectReferenceBuilder; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.KubernetesClientException; +import io.fabric8.kubernetes.client.dsl.LogWatch; +import io.vertx.core.http.HttpClosedException; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + +import java.io.InputStream; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A KubernetesPeonClient implementation that directly queries the Kubernetes API server for all read and write + * operations on a per-task basis. + *

    + * This implementation does not use caching and may put more load on the Kubernetes API server compared to + * {@link CachingKubernetesPeonClient}, especially when many tasks are running concurrently. + *

    + */ +public class KubernetesPeonClient +{ + private static final EmittingLogger log = new EmittingLogger(KubernetesPeonClient.class); + + protected final KubernetesClientApi clientApi; + protected final String namespace; + protected final String overlordNamespace; + protected final boolean debugJobs; + private final ServiceEmitter emitter; + + public KubernetesPeonClient( + KubernetesClientApi clientApi, + String namespace, + String overlordNamespace, + boolean debugJobs, + ServiceEmitter emitter + ) + { + this.clientApi = clientApi; + this.namespace = namespace; + this.overlordNamespace = overlordNamespace; + this.debugJobs = debugJobs; + this.emitter = emitter; + } + + public KubernetesPeonClient( + KubernetesClientApi clientApi, + String namespace, + boolean debugJobs, + ServiceEmitter emitter + ) + { + this(clientApi, namespace, "", debugJobs, emitter); + } + + public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) throws IllegalStateException + { + long start = System.currentTimeMillis(); + // launch job + return clientApi.executeRequest(client -> { + String jobName = job.getMetadata().getName(); + + log.info("Submitting job[%s] for task[%s].", jobName, task.getId()); + createK8sJobWithRetries(job); + log.info("Submitted job[%s] for task[%s]. Waiting for POD to launch.", jobName, task.getId()); + + // Wait for the pod to be available + Pod mainPod = getPeonPodWithRetries(jobName); + log.info("Pod for job[%s] launched for task[%s]. Waiting for pod to be in running state.", jobName, task.getId()); + + // Wait for the pod to be in state running, completed, or failed. + Pod result = waitForPodResultWithRetries(mainPod, howLong, timeUnit); + + if (result == null) { + throw new ISE("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", task.getId()); + } + log.info("Pod for job[%s] is in state [%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); + long duration = System.currentTimeMillis() - start; + emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); + return result; + }); + } + + public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) + { + return clientApi.executeRequest(client -> { + Job job = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .waitUntilCondition( + x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null + && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), + howLong, + unit + ); + if (job == null) { + log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); + return new JobResponse(null, PeonPhase.FAILED); + } + if (job.getStatus().getSucceeded() != null) { + return new JobResponse(job, PeonPhase.SUCCEEDED); + } + log.warn("Task %s failed with status %s", taskId, job.getStatus()); + return new JobResponse(job, PeonPhase.FAILED); + }); + } + + public boolean deletePeonJob(K8sTaskId taskId) + { + if (!debugJobs) { + Boolean result = clientApi.executeRequest(client -> !client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .delete().isEmpty()); + if (result) { + log.info("Cleaned up k8s job: %s", taskId); + } else { + log.info("K8s job does not exist: %s", taskId); + } + return result; + } else { + log.info("Not cleaning up job %s due to flag: debugJobs=true", taskId); + return true; + } + } + + public Optional getPeonLogWatcher(K8sTaskId taskId) + { + KubernetesClient k8sClient = clientApi.getClient(); + try { + LogWatch logWatch = k8sClient.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .inContainer("main") + .watchLog(); + if (logWatch == null) { + return Optional.absent(); + } + return Optional.of(logWatch); + } + catch (Exception e) { + log.error(e, "Error watching logs from task: %s", taskId); + return Optional.absent(); + } + } + + public Optional getPeonLogs(K8sTaskId taskId) + { + KubernetesClient k8sClient = clientApi.getClient(); + try { + InputStream logStream = k8sClient.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .inContainer("main") + .getLogInputStream(); + if (logStream == null) { + return Optional.absent(); + } + return Optional.of(logStream); + } + catch (Exception e) { + log.error(e, "Error streaming logs from task: %s", taskId); + return Optional.absent(); + } + } + + public List getPeonJobs() + { + return this.overlordNamespace.isEmpty() + ? getPeonJobsWithoutOverlordNamespaceKeyLabels() + : getPeonJobsWithOverlordNamespaceKeyLabels(); + } + + private List getPeonJobsWithoutOverlordNamespaceKeyLabels() + { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .list() + .getItems()); + } + + private List getPeonJobsWithOverlordNamespaceKeyLabels() + { + return clientApi.executeRequest(client -> client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .withLabel(DruidK8sConstants.OVERLORD_NAMESPACE_KEY, overlordNamespace) + .list() + .getItems()); + } + + public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit) + { + AtomicInteger numDeleted = new AtomicInteger(); + return clientApi.executeRequest(client -> { + List jobs = getJobsToCleanup(getPeonJobs(), howFarBack, timeUnit); + jobs.forEach(x -> { + if (!client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withName(x.getMetadata().getName()) + .delete().isEmpty()) { + numDeleted.incrementAndGet(); + } else { + log.error("Failed to delete job %s", x.getMetadata().getName()); + } + }); + return numDeleted.get(); + }); + } + + private List getJobsToCleanup(List candidates, long howFarBack, TimeUnit timeUnit) + { + List toDelete = new ArrayList<>(); + long cutOff = System.currentTimeMillis() - timeUnit.toMillis(howFarBack); + candidates.forEach(x -> { + // jobs that are complete + if (x.getStatus().getActive() == null) { + Timestamp timestamp = Timestamp.valueOf(x.getStatus().getCompletionTime()); + if (timestamp.before(new Timestamp(cutOff))) { + toDelete.add(x); + } + } + }); + return toDelete; + } + + public Optional getPeonPod(String jobName) + { + return clientApi.executeRequest(client -> getPeonPod(client, jobName)); + } + + private Optional getPeonPod(KubernetesClient client, String jobName) + { + List pods = client.pods() + .inNamespace(namespace) + .withLabel("job-name", jobName) + .list() + .getItems(); + return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); + } + + public Pod waitForPodResultWithRetries(final Pod pod, long howLong, TimeUnit timeUnit) + { + return clientApi.executeRequest(client -> waitForPodResultWithRetries(client, pod, howLong, timeUnit, 5, RetryUtils.DEFAULT_MAX_TRIES)); + } + + public Pod getPeonPodWithRetries(String jobName) + { + return clientApi.executeRequest(client -> getPeonPodWithRetries(client, jobName, 5, RetryUtils.DEFAULT_MAX_TRIES)); + } + + public void createK8sJobWithRetries(Job job) + { + clientApi.executeRequest(client -> { + createK8sJobWithRetries(client, job, 5, RetryUtils.DEFAULT_MAX_TRIES); + return null; + }); + } + + /** + * Creates a Kubernetes job with retry logic for transient connection pool exceptions. + *

    + * This method attempts to create the specified job in Kubernetes with built-in retry logic + * for transient connection pool issues. If the job already exists (HTTP 409 conflict), + * the method returns successfully without throwing an exception, assuming the job was + * already submitted by a previous request. + *

    + * The retry logic only applies to transient connection pool exceptions. Other exceptions will cause the method to + * fail immediately. + * + * @param client the Kubernetes client to use for job creation + * @param job the Kubernetes job to create + * @param quietTries number of initial retry attempts without logging warnings + * @param maxTries maximum total number of retry attempts + * @throws DruidException if job creation fails after all retry attempts or encounters non-retryable errors + */ + @VisibleForTesting + void createK8sJobWithRetries(KubernetesClient client, Job job, int quietTries, int maxTries) + { + try { + RetryUtils.retry( + () -> { + try { + client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .resource(job) + .create(); + return null; + } + catch (KubernetesClientException e) { + if (e.getCode() == 409) { + // Job already exists, return successfully + log.info("K8s job[%s] already exists, skipping creation", job.getMetadata().getName()); + return null; + } + throw e; + } + }, + this::isRetryableTransientConnectionPoolException, quietTries, maxTries + ); + } + catch (Exception e) { + throw DruidException.defensive(e, "Error when creating K8s job[%s]", job.getMetadata().getName()); + } + } + + /** + * Waits for a Kubernetes pod to reach a ready state with retry logic for transient connection pool exceptions. + *

    + * This method waits for the specified pod to have a valid status with a pod IP assigned, indicating + * it has been scheduled and is in a ready state. The method includes retry logic to handle transient + * connection pool exceptions that may occur during the wait operation. + *

    + * The method will wait up to the specified timeout for the pod to become ready, and retry the entire wait operation + * if transient connection issues are encountered. + * + * @param client the Kubernetes client to use for pod operations + * @param pod the pod to wait for + * @param howLong the maximum time to wait for the pod to become ready + * @param timeUnit the time unit for the wait timeout + * @param quietTries number of initial retry attempts without logging warnings + * @param maxTries maximum total number of retry attempts + * @return the pod in its ready state, or null if the pod disappeared or wait operation failed + * @throws DruidException if waiting fails after all retry attempts or encounters non-retryable errors + */ + @VisibleForTesting + Pod waitForPodResultWithRetries(KubernetesClient client, Pod pod, long howLong, TimeUnit timeUnit, int quietTries, int maxTries) + { + try { + return RetryUtils.retry( + () -> client.pods() + .inNamespace(namespace) + .withName(pod.getMetadata().getName()) + .waitUntilCondition( + p -> { + if (p == null) { + return true; + } + return p.getStatus() != null && p.getStatus().getPodIP() != null; + }, howLong, timeUnit), + this::isRetryableTransientConnectionPoolException, quietTries, maxTries); + } + catch (Exception e) { + throw DruidException.defensive(e, "Error when waiting for pod[%s] to start", pod.getMetadata().getName()); + } + } + + /** + * Retrieves the pod associated with a Kubernetes job with retry logic for transient failures. + *

    + * This method searches for a pod with the specified job name label and includes retry logic + * to handle both transient connection pool exceptions and cases where the pod may not be + * immediately available after job creation. If no pod is found, the method examines job + * events to provide detailed error information about pod creation failures. + *

    + * The retry logic applies to: + *

      + *
    • Transient connection pool exceptions
    • + *
    • Pod not found scenarios, except when blacklisted error messages from {@link DruidK8sConstants#BLACKLISTED_PEON_POD_ERROR_MESSAGES} are encountered
    • + *
    + * + * @param client the Kubernetes client to use for pod and event operations + * @param jobName the name of the job whose pod should be retrieved + * @param quietTries number of initial retry attempts without logging warnings + * @param maxTries maximum total number of retry attempts + * @return the pod associated with the job + * @throws KubernetesResourceNotFoundException if the pod cannot be found after all retry attempts + * @throws DruidException if retrieval fails due to other errors + */ + @VisibleForTesting + Pod getPeonPodWithRetries(KubernetesClient client, String jobName, int quietTries, int maxTries) + { + try { + return RetryUtils.retry( + () -> { + Optional maybePod = getPeonPod(client, jobName); + if (maybePod.isPresent()) { + return maybePod.get(); + } + + // If the pod is missing, we can take a look at job events to discover potential problems with pod creation. + List events = getPeonEvents(client, jobName); + + if (events.isEmpty()) { + throw new KubernetesResourceNotFoundException("K8s pod with label[job-name=%s] not found", jobName); + } else { + Event latestEvent = events.get(events.size() - 1); + throw new KubernetesResourceNotFoundException( + "Job[%s] failed to create pods. Message[%s]", jobName, latestEvent.getMessage()); + } + }, + this::shouldRetryWaitForStartingPeonPod, quietTries, maxTries + ); + } + catch (KubernetesResourceNotFoundException e) { + throw e; + } + catch (Exception e) { + throw DruidException.defensive(e, "Error when looking for K8s pod with label[job-name=%s]", jobName); + } + } + + /** + * Determines if this exception, specifically when containing Kubernetes job event messages, permits a retry attempt. + *

    + * The method checks the exception message against a predefined list of Kubernetes event messages. + * These substrings, found in {@link DruidK8sConstants#BLACKLISTED_PEON_POD_ERROR_MESSAGES}, + * represent Kubernetes event that indicate a retry for starting the Peon Pod would likely be futile. + */ + private boolean shouldRetryWaitForStartingPeonPod(Throwable e) + { + if (isRetryableTransientConnectionPoolException(e)) { + return true; + } + + if (!(e instanceof KubernetesResourceNotFoundException)) { + return false; + } + + String errorMessage = e.getMessage(); + for (String blacklistedMessage : DruidK8sConstants.BLACKLISTED_PEON_POD_ERROR_MESSAGES) { + if (errorMessage.contains(blacklistedMessage)) { + return false; + } + } + + return true; + } + + /** + * Checks if the exception is a potentially transient connection pool exception. + *

    + * This method checks if the exception is one of the known transient connection pool exceptions + * and whether it contains a specific message substring, if applicable. + *

    + * We have experienced connections in the pool being closed by the server-side but remaining in the pool. These issues + * should be safe to retry in many cases. + */ + private boolean isRetryableTransientConnectionPoolException(Throwable e) + { + if (e instanceof KubernetesClientException) { + return e.getMessage() != null && e.getMessage().contains("Connection was closed"); + } else if (e instanceof HttpClosedException) { + return true; + } + return false; + } + + private List getPeonEvents(KubernetesClient client, String jobName) + { + ObjectReference objectReference = new ObjectReferenceBuilder() + .withApiVersion("batch/v1") + .withKind("Job") + .withName(jobName) + .withNamespace(this.namespace) + .build(); + + try { + return client.v1() + .events() + .inNamespace(this.namespace) + .withInvolvedObject(objectReference) + .list() + .getItems(); + } + catch (KubernetesClientException e) { + log.warn("Failed to get events for job[%s]; %s", jobName, e.getMessage()); + return List.of(); + } + } + + private void emitK8sPodMetrics(Task task, String metric, long durationMs) + { + ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); + IndexTaskUtils.setTaskDimensions(metricBuilder, task); + emitter.emit(metricBuilder.setMetric(metric, durationMs)); + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index e4e1013d711f..b709bd2a02e0 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -31,10 +31,10 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; +import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; @@ -65,7 +65,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport private static final Period SHORT_LOG_SAVE_TIMEOUT = new Period("PT1S"); @Mock - AbstractKubernetesPeonClient kubernetesClient; + KubernetesPeonClient kubernetesClient; @Mock TaskLogs taskLogs; @Mock LogWatch logWatch; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 4882a31bf83c..083bf2db0e6c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -37,8 +37,8 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; -import org.apache.druid.k8s.overlord.common.AbstractKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.K8sTestUtils; +import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -74,7 +74,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport @Mock private HttpClient httpClient; @Mock private TaskAdapter taskAdapter; - @Mock private AbstractKubernetesPeonClient peonClient; + @Mock private KubernetesPeonClient peonClient; @Mock private KubernetesPeonLifecycle kubernetesPeonLifecycle; @Mock private ServiceEmitter emitter; @Mock private ListenableFuture statusFuture; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java similarity index 97% rename from extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java rename to extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java index 559e8281bb7a..5546f71a9193 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/DirectKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java @@ -45,7 +45,7 @@ import java.util.concurrent.TimeUnit; @EnableKubernetesMockClient(crud = true) -public class DirectKubernetesPeonClientTest +public class KubernetesPeonClientTest { private static final String ID = "id"; private static final String TASK_NAME_PREFIX = ""; @@ -57,7 +57,7 @@ public class DirectKubernetesPeonClientTest private KubernetesClient client; private KubernetesMockServer server; private KubernetesClientApi clientApi; - private DirectKubernetesPeonClient instance; + private KubernetesPeonClient instance; private StubServiceEmitter serviceEmitter; @BeforeEach @@ -65,7 +65,7 @@ public void setup() { clientApi = new TestKubernetesClient(this.client, NAMESPACE); serviceEmitter = new StubServiceEmitter("service", "host"); - instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + instance = new KubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); } @Test @@ -252,7 +252,7 @@ void test_deletePeonJob_withoutJob_returnsFalse() @Test void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() { - DirectKubernetesPeonClient instance = new DirectKubernetesPeonClient( + KubernetesPeonClient instance = new KubernetesPeonClient( new TestKubernetesClient(this.client, NAMESPACE), NAMESPACE, true, @@ -277,7 +277,7 @@ void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() @Test void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() { - DirectKubernetesPeonClient instance = new DirectKubernetesPeonClient( + KubernetesPeonClient instance = new KubernetesPeonClient( new TestKubernetesClient(this.client, NAMESPACE), NAMESPACE, true, @@ -350,7 +350,7 @@ void test_getPeonJobs_withJob_returnsPodList() @Test void test_getPeonJobs_withJobInDifferentNamespace_returnsPodList() { - instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); + instance = new KubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); Job job = new JobBuilder() .withNewMetadata() @@ -370,7 +370,7 @@ void test_getPeonJobs_withJobInDifferentNamespace_returnsPodList() @Test void test_getPeonJobs_withJobInDifferentNamespaceButOverlordNamespaceNotSpecified_doesNotReturnPodList() { - instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); + instance = new KubernetesPeonClient(clientApi, NAMESPACE, "ns", false, serviceEmitter); Job job = new JobBuilder() .withNewMetadata() @@ -390,7 +390,7 @@ void test_getPeonJobs_withJobInDifferentNamespaceButOverlordNamespaceNotSpecifie @Test void test_getPeonJobs_withJobInSameNamespaceWithoutLabels_doesNotReturnPodList() { - instance = new DirectKubernetesPeonClient(clientApi, NAMESPACE, NAMESPACE, false, serviceEmitter); + instance = new KubernetesPeonClient(clientApi, NAMESPACE, NAMESPACE, false, serviceEmitter); Job job = new JobBuilder() .withNewMetadata() diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 2dee4d3d9617..eb6d5a8c3de0 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -35,12 +35,12 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; -import org.apache.druid.k8s.overlord.common.DirectKubernetesPeonClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.KubernetesClientApi; +import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.k8s.overlord.common.httpclient.vertx.DruidKubernetesVertxHttpClientConfig; @@ -77,7 +77,7 @@ public class DruidPeonClientIntegrationTest private TaskConfig taskConfig; private DruidNode druidNode; private KubernetesClientApi k8sClient; - private DirectKubernetesPeonClient peonClient; + private KubernetesPeonClient peonClient; private ObjectMapper jsonMapper; @BeforeEach @@ -90,7 +90,7 @@ public void setup() new NamedType(IndexTask.IndexTuningConfig.class, "index") ); k8sClient = new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), new ConfigBuilder().build()); - peonClient = new DirectKubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); + peonClient = new KubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); druidNode = new DruidNode( "test", null, From e1c1d1327bca46e09a3e132b9b5ed05c5815d635 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 5 Dec 2025 13:15:50 +0530 Subject: [PATCH 38/58] Remove formatting changes in KubernetesPeonClient --- .../overlord/common/KubernetesPeonClient.java | 52 +++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 3269780584a6..691adf4c7097 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -127,7 +127,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time .withName(taskId.getK8sJobName()) .waitUntilCondition( x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null - && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), + && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), howLong, unit ); @@ -169,12 +169,12 @@ public Optional getPeonLogWatcher(K8sTaskId taskId) KubernetesClient k8sClient = clientApi.getClient(); try { LogWatch logWatch = k8sClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .inContainer("main") - .watchLog(); + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .inContainer("main") + .watchLog(); if (logWatch == null) { return Optional.absent(); } @@ -191,12 +191,12 @@ public Optional getPeonLogs(K8sTaskId taskId) KubernetesClient k8sClient = clientApi.getClient(); try { InputStream logStream = k8sClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .inContainer("main") - .getLogInputStream(); + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .inContainer("main") + .getLogInputStream(); if (logStream == null) { return Optional.absent(); } @@ -283,10 +283,10 @@ public Optional getPeonPod(String jobName) private Optional getPeonPod(KubernetesClient client, String jobName) { List pods = client.pods() - .inNamespace(namespace) - .withLabel("job-name", jobName) - .list() - .getItems(); + .inNamespace(namespace) + .withLabel("job-name", jobName) + .list() + .getItems(); return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); } @@ -382,15 +382,15 @@ Pod waitForPodResultWithRetries(KubernetesClient client, Pod pod, long howLong, try { return RetryUtils.retry( () -> client.pods() - .inNamespace(namespace) - .withName(pod.getMetadata().getName()) - .waitUntilCondition( - p -> { - if (p == null) { - return true; - } - return p.getStatus() != null && p.getStatus().getPodIP() != null; - }, howLong, timeUnit), + .inNamespace(namespace) + .withName(pod.getMetadata().getName()) + .waitUntilCondition( + p -> { + if (p == null) { + return true; + } + return p.getStatus() != null && p.getStatus().getPodIP() != null; + }, howLong, timeUnit), this::isRetryableTransientConnectionPoolException, quietTries, maxTries); } catch (Exception e) { From 6ea97f47030728903d6c4c5a718b4ab019afb487 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 5 Dec 2025 13:17:59 +0530 Subject: [PATCH 39/58] Remove more formatting changes --- .../k8s/overlord/common/KubernetesPeonClient.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 691adf4c7097..f9a21f2c7b63 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -127,7 +127,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time .withName(taskId.getK8sJobName()) .waitUntilCondition( x -> (x == null) || (x.getStatus() != null && x.getStatus().getActive() == null - && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), + && (x.getStatus().getFailed() != null || x.getStatus().getSucceeded() != null)), howLong, unit ); @@ -191,12 +191,12 @@ public Optional getPeonLogs(K8sTaskId taskId) KubernetesClient k8sClient = clientApi.getClient(); try { InputStream logStream = k8sClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .inContainer("main") - .getLogInputStream(); + .v1() + .jobs() + .inNamespace(namespace) + .withName(taskId.getK8sJobName()) + .inContainer("main") + .getLogInputStream(); if (logStream == null) { return Optional.absent(); } From ceb10bc88b0378db1b41250c6103aad4b60ef8c5 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 10:18:57 -0600 Subject: [PATCH 40/58] Address the more minor review comments --- .../embedded/indexing/IngestionSmokeTest.java | 19 +------- .../overlord/KubernetesOverlordModule.java | 4 +- .../overlord/KubernetesTaskRunnerConfig.java | 28 +++++------ .../overlord/KubernetesTaskRunnerFactory.java | 48 ++++++------------- .../common/CachingKubernetesPeonClient.java | 15 ++---- .../KubernetesTaskRunnerFactoryTest.java | 2 +- .../CachingKubernetesPeonClientTest.java | 2 +- .../testing/embedded/EmbeddedClusterApis.java | 9 ++++ 8 files changed, 47 insertions(+), 80 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java index db63d9ca76a0..cb0e2052d318 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java @@ -20,12 +20,10 @@ package org.apache.druid.testing.embedded.indexing; import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; import org.apache.commons.io.IOUtils; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.NoopTask; @@ -39,7 +37,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.metadata.storage.postgresql.PostgreSQLMetadataStorageModule; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.http.SqlTaskStatus; @@ -308,9 +305,8 @@ public void test_runKafkaSupervisor() Assertions.assertEquals(topic, supervisorStatus.getSource()); // Confirm tasks are being created and running - // This more forgiving assertion avoids weird race conditions with super specific checks like "exactly 1 running task" - int runningTasks = getTaskCount("running", dataSource); - int completedTasks = getTaskCount("complete", dataSource); + int runningTasks = cluster.callApi().getTaskCount("running", dataSource); + int completedTasks = cluster.callApi().getTaskCount("complete", dataSource); Assertions.assertTrue(runningTasks + completedTasks > 0); // Suspend the supervisor and verify the state @@ -405,17 +401,6 @@ private void waitForSegmentsToBeQueryable(int numSegments) ); } - /** - * Gets the count of tasks with the given status for the specified datasource. - */ - private int getTaskCount(String status, String dataSource) - { - return ImmutableList.copyOf( - (CloseableIterator) - cluster.callApi().onLeaderOverlord(o -> o.taskStatuses(status, dataSource, 100)) - ).size(); - } - /** * Verifies the total number of used segments in {@link #dataSource}. */ diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index f6d708dd7e2c..c854a5c6c5d7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -170,10 +170,10 @@ public DruidKubernetesClient makeKubernetesClient( config.setAdditionalProperty( DruidKubernetesClient.ENABLE_INFORMERS_KEY, - kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers()); + kubernetesTaskRunnerConfig.isUseK8sSharedInformers()); config.setAdditionalProperty( DruidKubernetesClient.INFORMER_RESYNC_PERIOD_MS_KEY, - kubernetesTaskRunnerConfig.getKubernetesClientInformerResyncPeriod().toStandardDuration().getMillis()); + kubernetesTaskRunnerConfig.getK8sSharedInformerResyncPeriod().toStandardDuration().getMillis()); client = new DruidKubernetesClient(httpClientFactory, config); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index ec6bf194a710..863d15236a37 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -82,10 +82,10 @@ public class KubernetesTaskRunnerConfig @JsonProperty // enable using kubernetes informer cache for peon client operations - private boolean enableKubernetesClientSharedInformers = false; + private boolean useK8sSharedInformers = false; @JsonProperty - private Period kubernetesClientInformerResyncPeriod = new Period("PT5M"); + private Period k8sSharedInformerResyncPeriod = new Period("PT5M"); @JsonProperty @NotNull @@ -172,8 +172,8 @@ private KubernetesTaskRunnerConfig( Map annotations, Integer capacity, Period taskJoinTimeout, - boolean enableKubernetesClientSharedInformers, - Period kubernetesClientInformerResyncPeriod + boolean useK8sSharedInformers, + Period k8sSharedInformerResyncPeriod ) { this.namespace = namespace; @@ -251,13 +251,13 @@ private KubernetesTaskRunnerConfig( capacity, this.capacity ); - this.enableKubernetesClientSharedInformers = ObjectUtils.getIfNull( - enableKubernetesClientSharedInformers, - this.enableKubernetesClientSharedInformers + this.useK8sSharedInformers = ObjectUtils.getIfNull( + useK8sSharedInformers, + this.useK8sSharedInformers ); - this.kubernetesClientInformerResyncPeriod = ObjectUtils.getIfNull( - kubernetesClientInformerResyncPeriod, - this.kubernetesClientInformerResyncPeriod + this.k8sSharedInformerResyncPeriod = ObjectUtils.getIfNull( + k8sSharedInformerResyncPeriod, + this.k8sSharedInformerResyncPeriod ); } @@ -368,14 +368,14 @@ public Integer getCapacity() return capacity; } - public boolean isEnableKubernetesClientSharedInformers() + public boolean isUseK8sSharedInformers() { - return enableKubernetesClientSharedInformers; + return useK8sSharedInformers; } - public Period getKubernetesClientInformerResyncPeriod() + public Period getK8sSharedInformerResyncPeriod() { - return kubernetesClientInformerResyncPeriod; + return k8sSharedInformerResyncPeriod; } public static Builder builder() diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 99a1b9012d90..c18d4a2e97c2 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -72,43 +72,25 @@ public KubernetesTaskRunnerFactory( public KubernetesTaskRunner build() { KubernetesPeonClient peonClient; - boolean enableCache = kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers(); + boolean enableCache = kubernetesTaskRunnerConfig.isUseK8sSharedInformers(); boolean useOverlordNamespace = adapterTypeAllowingTasksInDifferentNamespaces.contains(taskAdapter.getAdapterType()); if (enableCache) { - if (useOverlordNamespace) { - peonClient = new CachingKubernetesPeonClient( - druidKubernetesClient, - kubernetesTaskRunnerConfig.getNamespace(), - kubernetesTaskRunnerConfig.getOverlordNamespace(), - kubernetesTaskRunnerConfig.isDebugJobs(), - emitter - ); - } else { - peonClient = new CachingKubernetesPeonClient( - druidKubernetesClient, - kubernetesTaskRunnerConfig.getNamespace(), - kubernetesTaskRunnerConfig.isDebugJobs(), - emitter - ); - } + peonClient = new CachingKubernetesPeonClient( + druidKubernetesClient, + kubernetesTaskRunnerConfig.getNamespace(), + useOverlordNamespace ? kubernetesTaskRunnerConfig.getOverlordNamespace() : "", + kubernetesTaskRunnerConfig.isDebugJobs(), + emitter + ); } else { - if (useOverlordNamespace) { - peonClient = new KubernetesPeonClient( - druidKubernetesClient, - kubernetesTaskRunnerConfig.getNamespace(), - kubernetesTaskRunnerConfig.getOverlordNamespace(), - kubernetesTaskRunnerConfig.isDebugJobs(), - emitter - ); - } else { - peonClient = new KubernetesPeonClient( - druidKubernetesClient, - kubernetesTaskRunnerConfig.getNamespace(), - kubernetesTaskRunnerConfig.isDebugJobs(), - emitter - ); - } + peonClient = new KubernetesPeonClient( + druidKubernetesClient, + kubernetesTaskRunnerConfig.getNamespace(), + useOverlordNamespace ? kubernetesTaskRunnerConfig.getOverlordNamespace() : "", + kubernetesTaskRunnerConfig.isDebugJobs(), + emitter + ); } runner = new KubernetesTaskRunner( diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 11526ff52e1b..c2f188bf9c01 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -52,17 +52,8 @@ public CachingKubernetesPeonClient( ServiceEmitter emitter ) { - super(clientApi, namespace, overlordNamespace, debugJobs, emitter); - } - public CachingKubernetesPeonClient( - KubernetesClientApi clientApi, - String namespace, - boolean debugJobs, - ServiceEmitter emitter - ) - { - super(clientApi, namespace, "", debugJobs, emitter); + super(clientApi, namespace, overlordNamespace == null ? "" : overlordNamespace, debugJobs, emitter); } @Override @@ -125,13 +116,13 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time catch (TimeoutException e) { // A timeout here is not a problem, it forces us to loop around and check the cache again. // This prevents the case where we miss a notification and wait forever. - log.debug("Timeout waiting for job change notification for job[%s]. If full job timeout has not been reached, the job completion wait will continue", taskId.getK8sJobName()); + log.debug("Timeout waiting for change notification of job[%s]. Waiting until full job timeout.", taskId.getK8sJobName()); } catch (InterruptedException e) { throw DruidException.defensive(e, "Interrupted waiting for job change notification for job[%s]", taskId.getK8sJobName()); } catch (Throwable e) { - log.warn("Exception[%s] waiting for job change notification for job[%s]. Error message[%s]", e.getClass().getName(), taskId.getK8sJobName(), e.getMessage()); + log.noStackTrace().warn(e, "Exception while waiting for change notification of job[%s]", taskId.getK8sJobName()); } } while ((System.currentTimeMillis() - startTime < timeoutMs) && (jobSeenInCache || System.currentTimeMillis() < jobMustBeSeenBy)); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 9f70f6a5af05..786bcb09be48 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -62,7 +62,7 @@ public void setup() Config config = new ConfigBuilder().build(); config.setAdditionalProperty( DruidKubernetesClient.ENABLE_INFORMERS_KEY, - kubernetesTaskRunnerConfig.isEnableKubernetesClientSharedInformers()); + kubernetesTaskRunnerConfig.isUseK8sSharedInformers()); druidKubernetesClient = new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), config); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index 155dfbf1c765..a85d1a15f71b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -63,7 +63,7 @@ public void setup() throws Exception clientApi = new TestKubernetesClient(client, NAMESPACE); clientApi.start(); - peonClient = new CachingKubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + peonClient = new CachingKubernetesPeonClient(clientApi, NAMESPACE, "", false, serviceEmitter); } @AfterEach diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index 11b538530bdd..65bbd0ef155d 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -52,6 +52,7 @@ import java.io.Closeable; import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -230,6 +231,14 @@ public TaskStatus waitForTaskToFinish(String taskId, LatchableEmitter emitter) return getTaskStatus(taskId); } + /** + * Gets the count of tasks with the given status for the specified datasource. + */ + public int getTaskCount(String status, String dataSource) + { + return ImmutableList.copyOf((Collection) onLeaderOverlord(o -> o.taskStatuses(status, dataSource, 100))).size(); + } + /** * Retrieves all used segments from the metadata store (or cache if applicable). */ From f447b7a65eb91a289fb59537f3a13d9d7c477067 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 10:43:01 -0600 Subject: [PATCH 41/58] re-add log watch refactors to KubernetesPeonClient, they reduce API traffic --- .../overlord/common/KubernetesPeonClient.java | 62 ++++++++++++++----- 1 file changed, 46 insertions(+), 16 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index f9a21f2c7b63..0efcef92669b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -164,46 +164,76 @@ public boolean deletePeonJob(K8sTaskId taskId) } } + /** + * Get a LogWatch for the peon pod associated with the given taskId. Create it if it does not already exist. + *

    + * Any issues creating the LogWatch will be logged and an absent Optional will be returned. + *

    + * + * @return an Optional containing the {@link LogWatch} if it exists or was created. + */ public Optional getPeonLogWatcher(K8sTaskId taskId) { + Optional maybePod = getPeonPod(taskId.getK8sJobName()); + if (!maybePod.isPresent()) { + log.debug("Pod for job[%s] not found in cache, cannot watch logs", taskId.getK8sJobName()); + return Optional.absent(); + } + + Pod pod = maybePod.get(); + String podName = pod.getMetadata().getName(); + KubernetesClient k8sClient = clientApi.getClient(); try { - LogWatch logWatch = k8sClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .inContainer("main") - .watchLog(); + LogWatch logWatch = k8sClient.pods() + .inNamespace(namespace) + .withName(podName) + .inContainer("main") + .watchLog(); if (logWatch == null) { return Optional.absent(); } return Optional.of(logWatch); } catch (Exception e) { - log.error(e, "Error watching logs from task: %s", taskId); + log.error(e, "Error watching logs from task: %s, pod: %s", taskId, podName); return Optional.absent(); } } + /** + * Get an InputStream for the logs of the peon pod associated with the given taskId. + *

    + * Any issues creating the InputStream will be logged and an absent Optional will be returned. + *

    + * + * @return an Optional containing the {@link InputStream} if the pod exists and logs could be streamed, or absent otherwise + */ public Optional getPeonLogs(K8sTaskId taskId) { + Optional maybePod = getPeonPod(taskId.getK8sJobName()); + if (!maybePod.isPresent()) { + log.debug("Pod for job[%s] not found in cache, cannot stream logs", taskId.getK8sJobName()); + return Optional.absent(); + } + + Pod pod = maybePod.get(); + String podName = pod.getMetadata().getName(); + KubernetesClient k8sClient = clientApi.getClient(); try { - InputStream logStream = k8sClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withName(taskId.getK8sJobName()) - .inContainer("main") - .getLogInputStream(); + InputStream logStream = k8sClient.pods() + .inNamespace(namespace) + .resource(pod) + .inContainer("main") + .getLogInputStream(); if (logStream == null) { return Optional.absent(); } return Optional.of(logStream); } catch (Exception e) { - log.error(e, "Error streaming logs from task: %s", taskId); + log.error(e, "Error streaming logs for pod[%s] associated with task[%s]", podName, taskId.getOriginalTaskId()); return Optional.absent(); } } From 5653132bb0db2b223bf50ed6f536881ca2437fdf Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 10:43:33 -0600 Subject: [PATCH 42/58] migrate timers to stopwatch in caching k8s client per review comments --- .../common/CachingKubernetesPeonClient.java | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index c2f188bf9c01..228927eb6a18 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -23,8 +23,10 @@ import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.joda.time.Duration; import javax.annotation.Nullable; import java.util.List; @@ -59,12 +61,9 @@ public CachingKubernetesPeonClient( @Override public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) { - long timeoutMs = unit.toMillis(howLong); - long startTime = System.currentTimeMillis(); - - // Give the informer 2 resync periods to see the job. if it isn't seen by then, we assume the job was canceled. - // This is to prevent us from waiting for entire max job runtime on a job that was canceled before it even started. - long jobMustBeSeenBy = startTime + (clientApi.getInformerResyncPeriodMillis() * 2); + Duration timeout = Duration.millis(unit.toMillis(howLong)); + Duration jobMustBeSeenWithin = Duration.millis(clientApi.getInformerResyncPeriodMillis() * 2); + Stopwatch stopwatch = Stopwatch.createStarted(); boolean jobSeenInCache = false; // Set up to watch for job changes @@ -124,7 +123,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time catch (Throwable e) { log.noStackTrace().warn(e, "Exception while waiting for change notification of job[%s]", taskId.getK8sJobName()); } - } while ((System.currentTimeMillis() - startTime < timeoutMs) && (jobSeenInCache || System.currentTimeMillis() < jobMustBeSeenBy)); + } while (stopwatch.hasNotElapsed(timeout) && (jobSeenInCache || stopwatch.hasNotElapsed(jobMustBeSeenWithin))); log.warn("Timed out waiting for K8s job[%s] to complete", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); @@ -162,8 +161,8 @@ public Optional getPeonJob(String jobName) @Nullable protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) { - long timeoutMs = timeUnit.toMillis(howLong); - long startTime = System.currentTimeMillis(); + Duration timeout = Duration.millis(timeUnit.toMillis(howLong)); + Stopwatch stopwatch = Stopwatch.createStarted(); String podName = "unknown"; boolean podSeenInCache = false; @@ -224,7 +223,7 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time catch (Throwable e) { log.warn("Unexpected exception[%s] waiting for pod change notification for job [%s]. Error message[%s]", e.getClass().getName(), jobName, e.getMessage()); } - } while (System.currentTimeMillis() - startTime < timeoutMs); + } while (stopwatch.hasNotElapsed(timeout)); if (podSeenInCache) { log.warn("Timeout waiting for pod[%s] for job[%s] to become ready after it was created", podName, jobName); From aa8e12ead69b94a55b704cda0e7b96744dc0c8d7 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 10:46:40 -0600 Subject: [PATCH 43/58] Remove unused code --- .../overlord/common/DruidKubernetesClient.java | 18 ------------------ .../overlord/common/KubernetesClientApi.java | 5 ----- .../overlord/common/TestKubernetesClient.java | 12 ------------ 3 files changed, 35 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index f2adb26119e5..a6e10a0d75dd 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -116,24 +116,6 @@ public KubernetesClient getClient() return this.kubernetesClient; } - @Override - public SharedIndexInformer getPodInformer() - { - if (podInformer == null) { - throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); - } - return podInformer; - } - - @Override - public SharedIndexInformer getJobInformer() - { - if (jobInformer == null) { - throw DruidException.defensive("Job informer is not initialized, caching is disabled"); - } - return jobInformer; - } - @Override public KubernetesResourceEventNotifier getEventNotifier() { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java index bcd6c1ed914f..14c1aa3e6ce7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java @@ -22,7 +22,6 @@ import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; -import io.fabric8.kubernetes.client.informers.SharedIndexInformer; // Wraps all kubernetes api calls, to ensure you open and close connections properly public interface KubernetesClientApi @@ -38,10 +37,6 @@ public interface KubernetesClientApi // are done with the stream. Callers responsibility to clean up when using this method KubernetesClient getClient(); - SharedIndexInformer getPodInformer(); - - SharedIndexInformer getJobInformer(); - long getInformerResyncPeriodMillis(); KubernetesResourceEventNotifier getEventNotifier(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java index 55ab00eec8c7..5401b7ffed9c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -253,18 +253,6 @@ public KubernetesClient getClient() return client; } - @Override - public SharedIndexInformer getPodInformer() - { - return podInformer; - } - - @Override - public SharedIndexInformer getJobInformer() - { - return jobInformer; - } - @Override public long getInformerResyncPeriodMillis() { From 8df9dfe612b0e43274c98b2eeb5a5b260a2ffd21 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 11:09:36 -0600 Subject: [PATCH 44/58] style fix --- .../druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 786bcb09be48..0fad0156bfa6 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -62,7 +62,8 @@ public void setup() Config config = new ConfigBuilder().build(); config.setAdditionalProperty( DruidKubernetesClient.ENABLE_INFORMERS_KEY, - kubernetesTaskRunnerConfig.isUseK8sSharedInformers()); + kubernetesTaskRunnerConfig.isUseK8sSharedInformers() + ); druidKubernetesClient = new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), config); From ae29963321734dd4b85ebedd3925c4f55d1350ee Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 11:12:25 -0600 Subject: [PATCH 45/58] remove unneeded code --- .../k8s/overlord/common/KubernetesPeonClient.java | 12 +----------- .../overlord/common/KubernetesPeonClientTest.java | 4 +++- .../taskadapter/DruidPeonClientIntegrationTest.java | 2 +- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 0efcef92669b..8ecbc44d794c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -74,21 +74,11 @@ public KubernetesPeonClient( { this.clientApi = clientApi; this.namespace = namespace; - this.overlordNamespace = overlordNamespace; + this.overlordNamespace = overlordNamespace == null ? "" : overlordNamespace; this.debugJobs = debugJobs; this.emitter = emitter; } - public KubernetesPeonClient( - KubernetesClientApi clientApi, - String namespace, - boolean debugJobs, - ServiceEmitter emitter - ) - { - this(clientApi, namespace, "", debugJobs, emitter); - } - public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) throws IllegalStateException { long start = System.currentTimeMillis(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java index 5546f71a9193..7463c28805d2 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java @@ -65,7 +65,7 @@ public void setup() { clientApi = new TestKubernetesClient(this.client, NAMESPACE); serviceEmitter = new StubServiceEmitter("service", "host"); - instance = new KubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + instance = new KubernetesPeonClient(clientApi, NAMESPACE, null, false, serviceEmitter); } @Test @@ -255,6 +255,7 @@ void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() KubernetesPeonClient instance = new KubernetesPeonClient( new TestKubernetesClient(this.client, NAMESPACE), NAMESPACE, + null, true, serviceEmitter ); @@ -280,6 +281,7 @@ void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() KubernetesPeonClient instance = new KubernetesPeonClient( new TestKubernetesClient(this.client, NAMESPACE), NAMESPACE, + null, true, serviceEmitter ); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index eb6d5a8c3de0..bbc65c823a20 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -90,7 +90,7 @@ public void setup() new NamedType(IndexTask.IndexTuningConfig.class, "index") ); k8sClient = new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), new ConfigBuilder().build()); - peonClient = new KubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); + peonClient = new KubernetesPeonClient(k8sClient, "default", null, false, new NoopServiceEmitter()); druidNode = new DruidNode( "test", null, From 66f3cda43e2a0d98eb4c114cbab06a06fb62018d Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 13:59:31 -0600 Subject: [PATCH 46/58] Extract Caching client code from DruidKubernetesClient per review --- .../overlord/KubernetesOverlordModule.java | 47 +++- .../overlord/KubernetesTaskRunnerFactory.java | 11 +- .../common/CachingKubernetesPeonClient.java | 35 +-- .../common/DruidKubernetesCachingClient.java | 242 +++++++++++++++++ .../common/DruidKubernetesClient.java | 216 ---------------- .../overlord/common/KubernetesClientApi.java | 10 - .../KubernetesOverlordModuleTest.java | 1 - .../KubernetesTaskRunnerFactoryTest.java | 10 +- .../CachingKubernetesPeonClientTest.java | 50 ++-- .../common/TestCachingKubernetesClient.java | 243 ++++++++++++++++++ .../overlord/common/TestKubernetesClient.java | 222 ---------------- 11 files changed, 579 insertions(+), 508 deletions(-) create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java create mode 100644 extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index c854a5c6c5d7..0c8832943696 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -53,6 +53,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.k8s.overlord.common.DruidKubernetesCachingClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.httpclient.DruidKubernetesHttpClientFactory; import org.apache.druid.k8s.overlord.common.httpclient.jdk.DruidKubernetesJdkHttpClientConfig; @@ -73,6 +74,7 @@ import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.tasklogs.TaskLogs; +import javax.annotation.Nullable; import java.util.Locale; import java.util.Properties; @@ -150,15 +152,18 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, JDK_HTTPCLIENT_PROPERITES_PREFIX, DruidKubernetesJdkHttpClientConfig.class); } + /** + * Provides the base Kubernetes client for direct API operations. + * This is always created regardless of caching configuration. + */ @Provides @LazySingleton - public DruidKubernetesClient makeKubernetesClient( + public DruidKubernetesClient makeBaseKubernetesClient( KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig, DruidKubernetesHttpClientFactory httpClientFactory, Lifecycle lifecycle ) { - final DruidKubernetesClient client; final Config config = new ConfigBuilder().build(); if (kubernetesTaskRunnerConfig.isDisableClientProxy()) { @@ -168,14 +173,7 @@ public DruidKubernetesClient makeKubernetesClient( config.setNamespace(kubernetesTaskRunnerConfig.getNamespace()); - config.setAdditionalProperty( - DruidKubernetesClient.ENABLE_INFORMERS_KEY, - kubernetesTaskRunnerConfig.isUseK8sSharedInformers()); - config.setAdditionalProperty( - DruidKubernetesClient.INFORMER_RESYNC_PERIOD_MS_KEY, - kubernetesTaskRunnerConfig.getK8sSharedInformerResyncPeriod().toStandardDuration().getMillis()); - - client = new DruidKubernetesClient(httpClientFactory, config); + final DruidKubernetesClient client = new DruidKubernetesClient(httpClientFactory, config); lifecycle.addHandler( new Lifecycle.Handler() @@ -189,7 +187,7 @@ public void start() @Override public void stop() { - log.info("Stopping overlord Kubernetes client"); + log.info("Stopping base Kubernetes client"); client.getClient().close(); } } @@ -198,6 +196,33 @@ public void stop() return client; } + /** + * Provides the caching Kubernetes client that uses informers for efficient resource watching. + * Only created when caching is enabled via configuration. + */ + @Provides + @LazySingleton + @Nullable + public DruidKubernetesCachingClient makeCachingKubernetesClient( + KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig, + DruidKubernetesClient baseClient + ) + { + if (!kubernetesTaskRunnerConfig.isUseK8sSharedInformers()) { + log.info("Kubernetes shared informers disabled, caching client will not be created"); + return null; + } + + String namespace = kubernetesTaskRunnerConfig.getNamespace(); + long resyncPeriodMillis = kubernetesTaskRunnerConfig + .getK8sSharedInformerResyncPeriod() + .toStandardDuration() + .getMillis(); + + log.info("Creating Kubernetes caching client with informer resync period: %d ms", resyncPeriodMillis); + return new DruidKubernetesCachingClient(baseClient, namespace, resyncPeriodMillis); + } + /** * Provides a TaskRunnerFactory instance suitable for environments without Zookeeper. * In such environments, the standard RemoteTaskRunnerFactory may not be operational. diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index c18d4a2e97c2..14f96e3d5cb5 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -27,12 +27,14 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.k8s.overlord.common.CachingKubernetesPeonClient; +import org.apache.druid.k8s.overlord.common.DruidKubernetesCachingClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.TaskLogs; +import javax.annotation.Nullable; import java.util.Set; public class KubernetesTaskRunnerFactory implements TaskRunnerFactory @@ -43,6 +45,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); + CompletableFuture jobFuture = cachingClient.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); // We will loop until the full timeout is reached if the job is seen in cache. If the job does not show up in the cache we will exit earlier. // In this loop we first check the cache to see if our job is there and complete. This avoids missing notifications that happened before we set up the watch. @@ -93,10 +96,10 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time } // We wake up every informer resync period to avoid event notifier misses. - Job job = jobFuture.get(clientApi.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); + Job job = jobFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); // Immediately set up to watch for the next change in case we need to wait again - jobFuture = clientApi.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); + jobFuture = cachingClient.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); log.debug("Received job[%s] change notification", taskId.getK8sJobName()); jobSeenInCache = true; @@ -133,27 +136,27 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time public List getPeonJobs() { if (overlordNamespace.isEmpty()) { - return clientApi.executeJobCacheRequest(informer -> informer.getIndexer().list()); + return cachingClient.executeJobCacheRequest(informer -> informer.getIndexer().list()); } else { - return clientApi.executeJobCacheRequest(informer -> + return cachingClient.executeJobCacheRequest(informer -> informer.getIndexer() - .byIndex(DruidKubernetesClient.OVERLORD_NAMESPACE_INDEX, overlordNamespace)); + .byIndex(DruidKubernetesCachingClient.OVERLORD_NAMESPACE_INDEX, overlordNamespace)); } } @Override public Optional getPeonPod(String jobName) { - return clientApi.executePodCacheRequest(informer -> { - List pods = informer.getIndexer().byIndex(DruidKubernetesClient.JOB_NAME_INDEX, jobName); + return cachingClient.executePodCacheRequest(informer -> { + List pods = informer.getIndexer().byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); }); } public Optional getPeonJob(String jobName) { - return clientApi.executeJobCacheRequest(informer -> { - List jobs = informer.getIndexer().byIndex(DruidKubernetesClient.JOB_NAME_INDEX, jobName); + return cachingClient.executeJobCacheRequest(informer -> { + List jobs = informer.getIndexer().byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); return jobs.isEmpty() ? Optional.absent() : Optional.of(jobs.get(0)); }); } @@ -167,7 +170,7 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time boolean podSeenInCache = false; // Set up to watch for pod changes - CompletableFuture podFuture = clientApi.getEventNotifier().waitForPodChange(jobName); + CompletableFuture podFuture = cachingClient.getEventNotifier().waitForPodChange(jobName); // We will loop until the specified timeout is reached, or we see the pod become ready, whichever comes first. // We eagerly check the cache first to avoid missing notifications that happened before we set up the watch. @@ -194,9 +197,9 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time } // We wake up every informer resync period to avoid event notifier misses. - Pod pod = podFuture.get(clientApi.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); + Pod pod = podFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); - podFuture = clientApi.getEventNotifier().waitForPodChange(jobName); + podFuture = cachingClient.getEventNotifier().waitForPodChange(jobName); log.debug("Received pod[%s] change notification for job[%s]", podName, jobName); if (pod == null) { log.warn("Pod[%s] for job[%s] is null. This is unusual. Investigate Druid and k8s logs.", podName, jobName); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java new file mode 100644 index 000000000000..ad7c11091bbe --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +public class DruidKubernetesCachingClient +{ + private static final EmittingLogger log = new EmittingLogger(DruidKubernetesCachingClient.class); + + public static final String JOB_NAME_INDEX = "byJobName"; + public static final String OVERLORD_NAMESPACE_INDEX = "byOverlordNamespace"; + + private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 300000L; // 5 minutes + + private final KubernetesClientApi baseClient; + private final SharedIndexInformer podInformer; + private final SharedIndexInformer jobInformer; + private final KubernetesResourceEventNotifier eventNotifier; + private final long informerResyncPeriodMillis; + + public DruidKubernetesCachingClient( + KubernetesClientApi baseClient, + String namespace, + long informerResyncPeriodMillis + ) + { + this.baseClient = baseClient; + this.informerResyncPeriodMillis = informerResyncPeriodMillis; + this.eventNotifier = new KubernetesResourceEventNotifier(); + + this.podInformer = setupPodInformer(namespace); + this.jobInformer = setupJobInformer(namespace); + } + + public KubernetesClientApi getBaseClient() + { + return baseClient; + } + + // Delegate write operations to base client + public T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException + { + return baseClient.executeRequest(executor); + } + + public KubernetesClient getClient() + { + return baseClient.getClient(); + } + + public T executePodCacheRequest(KubernetesInformerExecutor executor) + { + if (podInformer == null) { + throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); + } + return executor.executeRequest(podInformer); + } + + public T executeJobCacheRequest(KubernetesInformerExecutor executor) + { + if (jobInformer == null) { + throw DruidException.defensive("Job informer is not initialized, caching is disabled"); + } + return executor.executeRequest(jobInformer); + } + + /** + * Sets up a shared informer to watch and cache Pod resources in the specified namespace. + *

    + * Registers event handlers for pod add/update/delete events and creates a custom index by job-name + * for efficient pod lookup by job. + *

    + */ + private SharedIndexInformer setupPodInformer(String namespace) + { + SharedIndexInformer podInformer = + baseClient.getClient().pods() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .inform( + new ResourceEventHandler<>() + { + @Override + public void onAdd(Pod pod) + { + log.debug("Pod[%s] got added", pod.getMetadata().getName()); + notifyPodChange(pod); + } + + @Override + public void onUpdate(Pod oldPod, Pod newPod) + { + log.debug("Pod[%s] got updated", oldPod.getMetadata().getName()); + notifyPodChange(newPod); + } + + @Override + public void onDelete(Pod pod, boolean deletedFinalStateUnknown) + { + log.debug("Pod[%s] got deleted", pod.getMetadata().getName()); + notifyPodChange(pod); + } + }, informerResyncPeriodMillis + ); + + Function> jobNameIndexer = pod -> { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + return Collections.singletonList(jobName); + } + } + return Collections.emptyList(); + }; + + Map>> customPodIndexers = new HashMap<>(); + customPodIndexers.put(JOB_NAME_INDEX, jobNameIndexer); + + podInformer.addIndexers(customPodIndexers); + return podInformer; + } + + /** + * Sets up a shared informer to watch and cache Job resources in the specified namespace. + *

    + * Registers event handlers for job add/update/delete events and creates custom indexes by job-name + * and overlord-namespace for efficient job lookup and filtering. + *

    + */ + private SharedIndexInformer setupJobInformer(String namespace) + { + SharedIndexInformer jobInformer = + baseClient.getClient().batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .inform( + new ResourceEventHandler<>() + { + @Override + public void onAdd(Job job) + { + log.debug("Job[%s] got added", job.getMetadata().getName()); + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + + @Override + public void onUpdate(Job oldJob, Job newJob) + { + log.debug("Job[%s] got updated", newJob.getMetadata().getName()); + eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); + } + + @Override + public void onDelete(Job job, boolean deletedFinalStateUnknown) + { + log.debug("Job[%s] got deleted", job.getMetadata().getName()); + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + }, informerResyncPeriodMillis + ); + + Function> overlordNamespaceIndexer = job -> { + if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { + String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); + if (overlordNamespace != null) { + return Collections.singletonList(overlordNamespace); + } + } + return Collections.emptyList(); + }; + + Function> jobNameIndexer = job -> { + if (job.getMetadata() != null && job.getMetadata().getName() != null) { + return Collections.singletonList(job.getMetadata().getName()); + } + return Collections.emptyList(); + }; + + Map>> customJobIndexers = new HashMap<>(); + customJobIndexers.put(OVERLORD_NAMESPACE_INDEX, overlordNamespaceIndexer); + customJobIndexers.put(JOB_NAME_INDEX, jobNameIndexer); + + jobInformer.addIndexers(customJobIndexers); + + return jobInformer; + } + + /** + * Utility method to only notify pod changes for pods that are part of indexing jobs. + */ + private void notifyPodChange(Pod pod) + { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + eventNotifier.notifyPodChange(jobName, pod); + } + } + } + + public KubernetesResourceEventNotifier getEventNotifier() + { + return eventNotifier; + } + + public long getInformerResyncPeriodMillis() + { + return informerResyncPeriodMillis; + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java index a6e10a0d75dd..86628673248b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesClient.java @@ -19,40 +19,14 @@ package org.apache.druid.k8s.overlord.common; -import com.google.common.base.Preconditions; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.Config; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientBuilder; -import io.fabric8.kubernetes.client.informers.ResourceEventHandler; -import io.fabric8.kubernetes.client.informers.SharedIndexInformer; -import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.k8s.overlord.common.httpclient.DruidKubernetesHttpClientFactory; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.function.Function; - public class DruidKubernetesClient implements KubernetesClientApi { - private static final EmittingLogger log = new EmittingLogger(DruidKubernetesClient.class); - - public static final String JOB_NAME_INDEX = "byJobName"; - public static final String OVERLORD_NAMESPACE_INDEX = "byOverlordNamespace"; - - public static final String ENABLE_INFORMERS_KEY = "druid.k8s.informers.enabled"; - public static final String INFORMER_RESYNC_PERIOD_MS_KEY = "druid.k8s.informers.resyncPeriodMs"; - private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 300000L; // 5 minutes - private final KubernetesClient kubernetesClient; - private final SharedIndexInformer podInformer; - private final SharedIndexInformer jobInformer; - private final KubernetesResourceEventNotifier eventNotifier; - private final long informerResyncPeriodMillis; public DruidKubernetesClient(DruidKubernetesHttpClientFactory httpClientFactory, Config kubernetesClientConfig) { @@ -60,23 +34,6 @@ public DruidKubernetesClient(DruidKubernetesHttpClientFactory httpClientFactory, .withHttpClientFactory(httpClientFactory) .withConfig(kubernetesClientConfig) .build(); - - // It is required that the config declares whether informers are enabled or not - Preconditions.checkNotNull(kubernetesClientConfig.getAdditionalProperties().get(ENABLE_INFORMERS_KEY), - "Kubernetes client config must contain property [%s]", - ENABLE_INFORMERS_KEY); - - informerResyncPeriodMillis = (long) kubernetesClientConfig - .getAdditionalProperties().getOrDefault(INFORMER_RESYNC_PERIOD_MS_KEY, DEFAULT_INFORMER_RESYNC_PERIOD_MS); - if ((boolean) kubernetesClientConfig.getAdditionalProperties().get(ENABLE_INFORMERS_KEY)) { - this.eventNotifier = new KubernetesResourceEventNotifier(); - this.podInformer = setupPodInformer(kubernetesClient.getNamespace()); - this.jobInformer = setupJobInformer(kubernetesClient.getNamespace()); - } else { - this.eventNotifier = null; - this.podInformer = null; - this.jobInformer = null; - } } @Override @@ -85,25 +42,6 @@ public T executeRequest(KubernetesExecutor executor) throws KubernetesRes return executor.executeRequest(kubernetesClient); } - @Override - public T executePodCacheRequest(KubernetesInformerExecutor executor) - { - if (podInformer == null) { - throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); - } - return executor.executeRequest(podInformer); - } - - @Override - public T executeJobCacheRequest(KubernetesInformerExecutor executor) - { - if (jobInformer == null) { - throw DruidException.defensive("Job informer is not initialized, caching is disabled"); - } - return executor.executeRequest(jobInformer); - } - - /** * This client automatically gets closed by the druid lifecycle, it should not be closed when used as it is * meant to be reused. @@ -115,158 +53,4 @@ public KubernetesClient getClient() { return this.kubernetesClient; } - - @Override - public KubernetesResourceEventNotifier getEventNotifier() - { - if (eventNotifier == null) { - throw DruidException.defensive("Event notifier is not initialized, caching is disabled"); - } - return eventNotifier; - } - - /** - * Sets up a shared informer to watch and cache Pod resources in the specified namespace. - *

    - * Registers event handlers for pod add/update/delete events and creates a custom index by job-name - * for efficient pod lookup by job. - *

    - */ - private SharedIndexInformer setupPodInformer(String namespace) - { - SharedIndexInformer podInformer = - kubernetesClient.pods() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .inform( - new ResourceEventHandler<>() - { - @Override - public void onAdd(Pod pod) - { - log.debug("Pod[%s] got added", pod.getMetadata().getName()); - notifyPodChange(pod); - } - - @Override - public void onUpdate(Pod oldPod, Pod newPod) - { - log.debug("Pod[%s] got updated", oldPod.getMetadata().getName()); - notifyPodChange(newPod); - } - - @Override - public void onDelete(Pod pod, boolean deletedFinalStateUnknown) - { - log.debug("Pod[%s] got deleted", pod.getMetadata().getName()); - notifyPodChange(pod); - } - }, informerResyncPeriodMillis - ); - - Function> jobNameIndexer = pod -> { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - return Collections.singletonList(jobName); - } - } - return Collections.emptyList(); - }; - - Map>> customPodIndexers = new HashMap<>(); - customPodIndexers.put(JOB_NAME_INDEX, jobNameIndexer); - - podInformer.addIndexers(customPodIndexers); - return podInformer; - } - - /** - * Sets up a shared informer to watch and cache Job resources in the specified namespace. - *

    - * Registers event handlers for job add/update/delete events and creates custom indexes by job-name - * and overlord-namespace for efficient job lookup and filtering. - *

    - */ - private SharedIndexInformer setupJobInformer(String namespace) - { - SharedIndexInformer jobInformer = - kubernetesClient.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .inform( - new ResourceEventHandler<>() - { - @Override - public void onAdd(Job job) - { - log.debug("Job[%s] got added", job.getMetadata().getName()); - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - - @Override - public void onUpdate(Job oldJob, Job newJob) - { - log.debug("Job[%s] got updated", newJob.getMetadata().getName()); - eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); - } - - @Override - public void onDelete(Job job, boolean deletedFinalStateUnknown) - { - log.debug("Job[%s] got deleted", job.getMetadata().getName()); - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - }, informerResyncPeriodMillis - ); - - Function> overlordNamespaceIndexer = job -> { - if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { - String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); - if (overlordNamespace != null) { - return Collections.singletonList(overlordNamespace); - } - } - return Collections.emptyList(); - }; - - Function> jobNameIndexer = job -> { - if (job.getMetadata() != null && job.getMetadata().getName() != null) { - return Collections.singletonList(job.getMetadata().getName()); - } - return Collections.emptyList(); - }; - - Map>> customJobIndexers = new HashMap<>(); - customJobIndexers.put(OVERLORD_NAMESPACE_INDEX, overlordNamespaceIndexer); - customJobIndexers.put(JOB_NAME_INDEX, jobNameIndexer); - - jobInformer.addIndexers(customJobIndexers); - - return jobInformer; - } - - /** - * Utility method to only notify pod changes for pods that are part of indexing jobs. - */ - private void notifyPodChange(Pod pod) - { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - eventNotifier.notifyPodChange(jobName, pod); - } - } - } - - @Override - public long getInformerResyncPeriodMillis() - { - if (jobInformer == null || podInformer == null) { - throw DruidException.defensive("Informers are not initialized, caching is disabled"); - } - return informerResyncPeriodMillis; - } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java index 14c1aa3e6ce7..655e4435f205 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesClientApi.java @@ -19,8 +19,6 @@ package org.apache.druid.k8s.overlord.common; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; // Wraps all kubernetes api calls, to ensure you open and close connections properly @@ -28,16 +26,8 @@ public interface KubernetesClientApi { T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException; - T executePodCacheRequest(KubernetesInformerExecutor executor); - - T executeJobCacheRequest(KubernetesInformerExecutor executor); - // use only when handling streams of data, example if you want to pass around an input stream from a pod, // then you would call this instead of executeRequest as you would want to keep the connection open until you // are done with the stream. Callers responsibility to clean up when using this method KubernetesClient getClient(); - - long getInformerResyncPeriodMillis(); - - KubernetesResourceEventNotifier getEventNotifier(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java index 59c9508005f6..4f66d970d7b7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java @@ -180,7 +180,6 @@ public void test_build_withoutSidecarSupport_returnsKubernetesTaskRunnerWithSing props.setProperty("druid.indexer.runner.namespace", "NAMESPACE"); injector = makeInjectorWithProperties(props, false, true); - TaskAdapter adapter = injector.getInstance(TaskAdapter.class); Assert.assertNotNull(adapter); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 0fad0156bfa6..f98022ce9dd5 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -26,6 +26,7 @@ import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.k8s.overlord.common.DruidKubernetesCachingClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.httpclient.vertx.DruidKubernetesVertxHttpClientConfig; @@ -47,6 +48,7 @@ public class KubernetesTaskRunnerFactoryTest private TaskLogs taskLogs; private DruidKubernetesClient druidKubernetesClient; + private DruidKubernetesCachingClient druidKubernetesCachingClient; @Mock private ServiceEmitter emitter; private TaskAdapter taskAdapter; @@ -60,13 +62,10 @@ public void setup() taskLogs = new NoopTaskLogs(); Config config = new ConfigBuilder().build(); - config.setAdditionalProperty( - DruidKubernetesClient.ENABLE_INFORMERS_KEY, - kubernetesTaskRunnerConfig.isUseK8sSharedInformers() - ); druidKubernetesClient = new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), config); + druidKubernetesCachingClient = null; taskAdapter = new TestTaskAdapter(); } @@ -80,7 +79,8 @@ public void test_get_returnsSameKuberentesTaskRunner_asBuild() taskLogs, druidKubernetesClient, emitter, - taskAdapter + taskAdapter, + druidKubernetesCachingClient ); KubernetesTaskRunner expectedRunner = factory.build(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index a85d1a15f71b..f8e1084c3565 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -50,9 +50,9 @@ public class CachingKubernetesPeonClientTest private KubernetesClient client; private KubernetesMockServer server; - private TestKubernetesClient clientApi; private CachingKubernetesPeonClient peonClient; private StubServiceEmitter serviceEmitter; + private TestCachingKubernetesClient cachingClient; @BeforeEach public void setup() throws Exception @@ -60,17 +60,19 @@ public void setup() throws Exception serviceEmitter = new StubServiceEmitter("service", "host"); // Set up real informers with the mock client - clientApi = new TestKubernetesClient(client, NAMESPACE); - clientApi.start(); + TestKubernetesClient clientApi = new TestKubernetesClient(client, NAMESPACE); - peonClient = new CachingKubernetesPeonClient(clientApi, NAMESPACE, "", false, serviceEmitter); + cachingClient = new TestCachingKubernetesClient(clientApi, NAMESPACE); + cachingClient.start(); + + peonClient = new CachingKubernetesPeonClient(cachingClient, NAMESPACE, "", false, serviceEmitter); } @AfterEach public void teardown() { - if (clientApi != null) { - clientApi.stop(); + if (cachingClient != null) { + cachingClient.stop(); } } @@ -92,7 +94,7 @@ public void test_getPeonPod_withPodInCache_returnsPresentOptional() throws Excep client.pods().inNamespace(NAMESPACE).resource(pod).create(); // Wait for informer to sync - clientApi.waitForSync(); + cachingClient.waitForSync(); // Query from cache Optional result = peonClient.getPeonPod(JOB_NAME); @@ -105,7 +107,7 @@ public void test_getPeonPod_withPodInCache_returnsPresentOptional() throws Excep public void test_getPeonPod_withoutPodInCache_returnsAbsentOptional() throws Exception { // Wait for informer to sync (empty cache) - clientApi.waitForSync(); + cachingClient.waitForSync(); Optional result = peonClient.getPeonPod(JOB_NAME); @@ -134,7 +136,7 @@ public void test_getPeonPod_withMultiplePodsForSameJob_returnsFirstOne() throws client.pods().inNamespace(NAMESPACE).resource(pod1).create(); client.pods().inNamespace(NAMESPACE).resource(pod2).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); Optional result = peonClient.getPeonPod(JOB_NAME); @@ -157,7 +159,7 @@ public void test_getPeonJob_withJobInCache_returnsPresentOptional() throws Excep client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); Optional result = peonClient.getPeonJob(JOB_NAME); @@ -168,7 +170,7 @@ public void test_getPeonJob_withJobInCache_returnsPresentOptional() throws Excep @Test public void test_getPeonJob_withoutJobInCache_returnsAbsentOptional() throws Exception { - clientApi.waitForSync(); + cachingClient.waitForSync(); Optional result = peonClient.getPeonJob(JOB_NAME); @@ -197,7 +199,7 @@ public void test_getPeonJobs_withoutOverlordNamespace_returnsAllJobsFromCache() client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job1).create(); client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job2).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); List jobs = peonClient.getPeonJobs(); @@ -207,7 +209,7 @@ public void test_getPeonJobs_withoutOverlordNamespace_returnsAllJobsFromCache() @Test public void test_getPeonJobs_withOverlordNamespace_returnsFilteredJobs() throws Exception { - peonClient = new CachingKubernetesPeonClient(clientApi, NAMESPACE, OVERLORD_NAMESPACE, false, serviceEmitter); + peonClient = new CachingKubernetesPeonClient(cachingClient, NAMESPACE, OVERLORD_NAMESPACE, false, serviceEmitter); Job matchingJob = new JobBuilder() .withNewMetadata() @@ -230,7 +232,7 @@ public void test_getPeonJobs_withOverlordNamespace_returnsFilteredJobs() throws client.batch().v1().jobs().inNamespace(NAMESPACE).resource(matchingJob).create(); client.batch().v1().jobs().inNamespace(NAMESPACE).resource(nonMatchingJob).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); List jobs = peonClient.getPeonJobs(); @@ -241,7 +243,7 @@ public void test_getPeonJobs_withOverlordNamespace_returnsFilteredJobs() throws @Test public void test_getPeonJobs_whenCacheEmpty_returnsEmptyList() throws Exception { - clientApi.waitForSync(); + cachingClient.waitForSync(); List jobs = peonClient.getPeonJobs(); @@ -265,7 +267,7 @@ public void test_waitForPeonJobCompletion_jobSucceeds() throws Exception .build(); client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); // Start waiting in background @@ -313,7 +315,7 @@ public void test_waitUntilPeonPodCreatedAndReady_podBecomesReady() throws Except .build(); client.pods().inNamespace(NAMESPACE).resource(pod).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); // Start waiting for pod to be ready in background CompletableFuture futurePod = CompletableFuture.supplyAsync(() -> @@ -360,7 +362,7 @@ public void test_waitUntilPeonPodCreatedAndReady_timeoutWhenPodNotReady() throws .build(); client.pods().inNamespace(NAMESPACE).resource(pod).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); // Wait for pod to be ready with short timeout Pod result = peonClient.waitUntilPeonPodCreatedAndReady(JOB_NAME, 1, TimeUnit.SECONDS); @@ -372,7 +374,7 @@ public void test_waitUntilPeonPodCreatedAndReady_timeoutWhenPodNotReady() throws @Test public void test_waitUntilPeonPodCreatedAndReady_returnNullWhenPodNeverCreated() throws Exception { - clientApi.waitForSync(); + cachingClient.waitForSync(); Assertions.assertNull(peonClient.waitUntilPeonPodCreatedAndReady(JOB_NAME, 1, TimeUnit.SECONDS)); } @@ -394,7 +396,7 @@ public void test_waitForPeonJobCompletion_timeoutWhenJobNeverCompletes() throws .build(); client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); // Wait with short timeout - job never completes JobResponse response = peonClient.waitForPeonJobCompletion(taskId, 500, TimeUnit.MILLISECONDS); @@ -421,7 +423,7 @@ public void test_waitForPeonJobCompletion_jobFails() throws Exception .build(); client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); // Start waiting in background CompletableFuture futureResponse = CompletableFuture.supplyAsync(() -> @@ -470,7 +472,7 @@ public void test_waitForPeonJobCompletion_jobGetsDeleted() throws Exception .build(); client.batch().v1().jobs().inNamespace(NAMESPACE).resource(job).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); // Start waiting in background CompletableFuture futureResponse = CompletableFuture.supplyAsync(() -> @@ -513,7 +515,7 @@ public void test_waitForPeonJobCompletion_jobDeletedBeforeSeenInCache() throws E // Delete immediately before informer syncs client.batch().v1().jobs().inNamespace(NAMESPACE).withName(taskId.getK8sJobName()).delete(); - clientApi.waitForSync(); + cachingClient.waitForSync(); JobResponse response = peonClient.waitForPeonJobCompletion(taskId, 10, TimeUnit.SECONDS); @@ -534,7 +536,7 @@ void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() throws Interrup client.pods().inNamespace(NAMESPACE).resource(pod).create(); - clientApi.waitForSync(); + cachingClient.waitForSync(); server.expect().get() .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java new file mode 100644 index 000000000000..b670bbff5812 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; +import io.fabric8.kubernetes.client.informers.SharedIndexInformer; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +public class TestCachingKubernetesClient extends DruidKubernetesCachingClient +{ + private final KubernetesClient client; + private final SharedIndexInformer podInformer; + private final SharedIndexInformer jobInformer; + private final KubernetesResourceEventNotifier eventNotifier; + private final CountDownLatch syncLatch; + + public TestCachingKubernetesClient(KubernetesClientApi clientApi, String namespace) + { + super(clientApi, namespace, 1000L); + this.client = clientApi.getClient(); + this.eventNotifier = new KubernetesResourceEventNotifier(); + this.syncLatch = new CountDownLatch(2); // Wait for both informers + + // Set up pod informer with real event handlers + this.podInformer = client.pods() + .inNamespace(namespace) + .inform( + new ResourceEventHandler() + { + @Override + public void onAdd(Pod pod) + { + notifyPodChange(pod); + } + + @Override + public void onUpdate(Pod oldPod, Pod newPod) + { + notifyPodChange(newPod); + } + + @Override + public void onDelete(Pod pod, boolean deletedFinalStateUnknown) + { + notifyPodChange(pod); + } + }, 1000L + ); + + // Add pod indexer + Map>> podIndexers = new HashMap<>(); + podIndexers.put( + "byJobName", pod -> { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + return Collections.singletonList(jobName); + } + } + return Collections.emptyList(); + } + ); + podInformer.addIndexers(podIndexers); + + // Set up job informer with real event handlers + this.jobInformer = client.batch() + .v1() + .jobs() + .inNamespace(namespace) + .withLabel(DruidK8sConstants.LABEL_KEY) + .inform( + new ResourceEventHandler() + { + @Override + public void onAdd(Job job) + { + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + + @Override + public void onUpdate(Job oldJob, Job newJob) + { + eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); + } + + @Override + public void onDelete(Job job, boolean deletedFinalStateUnknown) + { + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + }, 1000L + ); + + // Add job indexers + Map>> jobIndexers = new HashMap<>(); + jobIndexers.put( + "byJobName", job -> { + if (job.getMetadata() != null && job.getMetadata().getName() != null) { + return Collections.singletonList(job.getMetadata().getName()); + } + return Collections.emptyList(); + } + ); + jobIndexers.put( + "byOverlordNamespace", job -> { + if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { + String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); + if (overlordNamespace != null) { + return Collections.singletonList(overlordNamespace); + } + } + return Collections.emptyList(); + } + ); + jobInformer.addIndexers(jobIndexers); + } + + public void start() + { + // Add ready callbacks to count down latch + podInformer.addEventHandlerWithResyncPeriod( + new ResourceEventHandler() + { + @Override + public void onAdd(Pod obj) + { + + } + + @Override + public void onUpdate(Pod oldObj, Pod newObj) + { + + } + + @Override + public void onDelete(Pod obj, boolean deletedFinalStateUnknown) + { + + } + }, 1000L + ); + + jobInformer.addEventHandlerWithResyncPeriod( + new ResourceEventHandler() + { + @Override + public void onAdd(Job obj) + { + + } + + @Override + public void onUpdate(Job oldObj, Job newObj) + { + + } + + @Override + public void onDelete(Job obj, boolean deletedFinalStateUnknown) + { + + } + }, 1000L + ); + + podInformer.run(); + jobInformer.run(); + + // Count down after starting + syncLatch.countDown(); + syncLatch.countDown(); + } + + public void stop() + { + if (podInformer != null) { + podInformer.stop(); + } + if (jobInformer != null) { + jobInformer.stop(); + } + if (eventNotifier != null) { + eventNotifier.cancelAll(); + } + } + + public void waitForSync() throws InterruptedException + { + syncLatch.await(5, TimeUnit.SECONDS); + // Give informers a bit more time to process + Thread.sleep(200); + } + + private void notifyPodChange(Pod pod) + { + if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { + String jobName = pod.getMetadata().getLabels().get("job-name"); + if (jobName != null) { + eventNotifier.notifyPodChange(jobName, pod); + } + } + } + + @Override + public T executePodCacheRequest(KubernetesInformerExecutor executor) + { + return executor.executeRequest(podInformer); + } + + @Override + public T executeJobCacheRequest(KubernetesInformerExecutor executor) + { + return executor.executeRequest(jobInformer); + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java index 5401b7ffed9c..1f9fdcd7b6c9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestKubernetesClient.java @@ -19,214 +19,16 @@ package org.apache.druid.k8s.overlord.common; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; -import io.fabric8.kubernetes.client.informers.ResourceEventHandler; -import io.fabric8.kubernetes.client.informers.SharedIndexInformer; - -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; public class TestKubernetesClient implements KubernetesClientApi { private final KubernetesClient client; - private final SharedIndexInformer podInformer; - private final SharedIndexInformer jobInformer; - private final KubernetesResourceEventNotifier eventNotifier; - private final CountDownLatch syncLatch; public TestKubernetesClient(KubernetesClient client, String namespace) { this.client = client; - this.eventNotifier = new KubernetesResourceEventNotifier(); - this.syncLatch = new CountDownLatch(2); // Wait for both informers - - // Set up pod informer with real event handlers - this.podInformer = client.pods() - .inNamespace(namespace) - .inform( - new ResourceEventHandler() - { - @Override - public void onAdd(Pod pod) - { - notifyPodChange(pod); - } - - @Override - public void onUpdate(Pod oldPod, Pod newPod) - { - notifyPodChange(newPod); - } - - @Override - public void onDelete(Pod pod, boolean deletedFinalStateUnknown) - { - notifyPodChange(pod); - } - }, 1000L - ); - - // Add pod indexer - Map>> podIndexers = new HashMap<>(); - podIndexers.put( - "byJobName", pod -> { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - return Collections.singletonList(jobName); - } - } - return Collections.emptyList(); - } - ); - podInformer.addIndexers(podIndexers); - - // Set up job informer with real event handlers - this.jobInformer = client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .inform( - new ResourceEventHandler() - { - @Override - public void onAdd(Job job) - { - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - - @Override - public void onUpdate(Job oldJob, Job newJob) - { - eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); - } - - @Override - public void onDelete(Job job, boolean deletedFinalStateUnknown) - { - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - }, 1000L - ); - - // Add job indexers - Map>> jobIndexers = new HashMap<>(); - jobIndexers.put( - "byJobName", job -> { - if (job.getMetadata() != null && job.getMetadata().getName() != null) { - return Collections.singletonList(job.getMetadata().getName()); - } - return Collections.emptyList(); - } - ); - jobIndexers.put( - "byOverlordNamespace", job -> { - if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { - String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); - if (overlordNamespace != null) { - return Collections.singletonList(overlordNamespace); - } - } - return Collections.emptyList(); - } - ); - jobInformer.addIndexers(jobIndexers); - } - - public void start() - { - // Add ready callbacks to count down latch - podInformer.addEventHandlerWithResyncPeriod( - new ResourceEventHandler() - { - @Override - public void onAdd(Pod obj) - { - - } - - @Override - public void onUpdate(Pod oldObj, Pod newObj) - { - - } - - @Override - public void onDelete(Pod obj, boolean deletedFinalStateUnknown) - { - - } - }, 1000L - ); - - jobInformer.addEventHandlerWithResyncPeriod( - new ResourceEventHandler() - { - @Override - public void onAdd(Job obj) - { - - } - - @Override - public void onUpdate(Job oldObj, Job newObj) - { - - } - - @Override - public void onDelete(Job obj, boolean deletedFinalStateUnknown) - { - - } - }, 1000L - ); - - podInformer.run(); - jobInformer.run(); - - // Count down after starting - syncLatch.countDown(); - syncLatch.countDown(); - } - - public void stop() - { - if (podInformer != null) { - podInformer.stop(); - } - if (jobInformer != null) { - jobInformer.stop(); - } - if (eventNotifier != null) { - eventNotifier.cancelAll(); - } - } - - public void waitForSync() throws InterruptedException - { - syncLatch.await(5, TimeUnit.SECONDS); - // Give informers a bit more time to process - Thread.sleep(200); - } - - private void notifyPodChange(Pod pod) - { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - eventNotifier.notifyPodChange(jobName, pod); - } - } } @Override @@ -235,33 +37,9 @@ public T executeRequest(KubernetesExecutor executor) throws KubernetesRes return executor.executeRequest(client); } - @Override - public T executePodCacheRequest(KubernetesInformerExecutor executor) - { - return executor.executeRequest(podInformer); - } - - @Override - public T executeJobCacheRequest(KubernetesInformerExecutor executor) - { - return executor.executeRequest(jobInformer); - } - @Override public KubernetesClient getClient() { return client; } - - @Override - public long getInformerResyncPeriodMillis() - { - return 1000L; - } - - @Override - public KubernetesResourceEventNotifier getEventNotifier() - { - return eventNotifier; - } } From f99673cb20a907823473ac7f53e961bd8b2bcc90 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 14:10:59 -0600 Subject: [PATCH 47/58] Make name for cache read methods more logical --- .../k8s/overlord/common/CachingKubernetesPeonClient.java | 8 ++++---- .../k8s/overlord/common/DruidKubernetesCachingClient.java | 4 ++-- .../k8s/overlord/common/TestCachingKubernetesClient.java | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index ec3625566f45..9f5db276c4fe 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -136,9 +136,9 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time public List getPeonJobs() { if (overlordNamespace.isEmpty()) { - return cachingClient.executeJobCacheRequest(informer -> informer.getIndexer().list()); + return cachingClient.readJobCache(informer -> informer.getIndexer().list()); } else { - return cachingClient.executeJobCacheRequest(informer -> + return cachingClient.readJobCache(informer -> informer.getIndexer() .byIndex(DruidKubernetesCachingClient.OVERLORD_NAMESPACE_INDEX, overlordNamespace)); } @@ -147,7 +147,7 @@ public List getPeonJobs() @Override public Optional getPeonPod(String jobName) { - return cachingClient.executePodCacheRequest(informer -> { + return cachingClient.readPodCache(informer -> { List pods = informer.getIndexer().byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); }); @@ -155,7 +155,7 @@ public Optional getPeonPod(String jobName) public Optional getPeonJob(String jobName) { - return cachingClient.executeJobCacheRequest(informer -> { + return cachingClient.readJobCache(informer -> { List jobs = informer.getIndexer().byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); return jobs.isEmpty() ? Optional.absent() : Optional.of(jobs.get(0)); }); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index ad7c11091bbe..385de742ae11 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -78,7 +78,7 @@ public KubernetesClient getClient() return baseClient.getClient(); } - public T executePodCacheRequest(KubernetesInformerExecutor executor) + public T readPodCache(KubernetesInformerExecutor executor) { if (podInformer == null) { throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); @@ -86,7 +86,7 @@ public T executePodCacheRequest(KubernetesInformerExecutor executor) return executor.executeRequest(podInformer); } - public T executeJobCacheRequest(KubernetesInformerExecutor executor) + public T readJobCache(KubernetesInformerExecutor executor) { if (jobInformer == null) { throw DruidException.defensive("Job informer is not initialized, caching is disabled"); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java index b670bbff5812..fe4ba186cec4 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java @@ -230,13 +230,13 @@ private void notifyPodChange(Pod pod) } @Override - public T executePodCacheRequest(KubernetesInformerExecutor executor) + public T readPodCache(KubernetesInformerExecutor executor) { return executor.executeRequest(podInformer); } @Override - public T executeJobCacheRequest(KubernetesInformerExecutor executor) + public T readJobCache(KubernetesInformerExecutor executor) { return executor.executeRequest(jobInformer); } From 8ec81cf5c4fd7f500e9481a6d0513d2938a8cdf3 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 14:43:35 -0600 Subject: [PATCH 48/58] Stop exposing the EventNotifier in DruidKubernetesCachingClient --- .../overlord/common/CachingKubernetesPeonClient.java | 8 ++++---- .../overlord/common/DruidKubernetesCachingClient.java | 10 ++++++++-- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 9f5db276c4fe..1f2c96cae333 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -70,7 +70,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time boolean jobSeenInCache = false; // Set up to watch for job changes - CompletableFuture jobFuture = cachingClient.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); + CompletableFuture jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); // We will loop until the full timeout is reached if the job is seen in cache. If the job does not show up in the cache we will exit earlier. // In this loop we first check the cache to see if our job is there and complete. This avoids missing notifications that happened before we set up the watch. @@ -99,7 +99,7 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time Job job = jobFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); // Immediately set up to watch for the next change in case we need to wait again - jobFuture = cachingClient.getEventNotifier().waitForJobChange(taskId.getK8sJobName()); + jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); log.debug("Received job[%s] change notification", taskId.getK8sJobName()); jobSeenInCache = true; @@ -170,7 +170,7 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time boolean podSeenInCache = false; // Set up to watch for pod changes - CompletableFuture podFuture = cachingClient.getEventNotifier().waitForPodChange(jobName); + CompletableFuture podFuture = cachingClient.waitForPodChange(jobName); // We will loop until the specified timeout is reached, or we see the pod become ready, whichever comes first. // We eagerly check the cache first to avoid missing notifications that happened before we set up the watch. @@ -199,7 +199,7 @@ protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, Time // We wake up every informer resync period to avoid event notifier misses. Pod pod = podFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); - podFuture = cachingClient.getEventNotifier().waitForPodChange(jobName); + podFuture = cachingClient.waitForPodChange(jobName); log.debug("Received pod[%s] change notification for job[%s]", podName, jobName); if (pod == null) { log.warn("Pod[%s] for job[%s] is null. This is unusual. Investigate Druid and k8s logs.", podName, jobName); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index 385de742ae11..705241d705b1 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -31,6 +31,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.function.Function; public class DruidKubernetesCachingClient @@ -230,9 +231,14 @@ private void notifyPodChange(Pod pod) } } - public KubernetesResourceEventNotifier getEventNotifier() + public CompletableFuture waitForJobChange(String jobName) { - return eventNotifier; + return eventNotifier.waitForJobChange(jobName); + } + + public CompletableFuture waitForPodChange(String jobName) + { + return eventNotifier.waitForPodChange(jobName); } public long getInformerResyncPeriodMillis() From 78e1c82295fabf641cfcf5248a5440207282d90b Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 14:56:25 -0600 Subject: [PATCH 49/58] Improve informer executor name per review --- .../k8s/overlord/common/DruidKubernetesCachingClient.java | 4 ++-- ...cutor.java => SharedInformerCacheReadRequestExecutor.java} | 2 +- .../k8s/overlord/common/TestCachingKubernetesClient.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) rename extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/{KubernetesInformerExecutor.java => SharedInformerCacheReadRequestExecutor.java} (94%) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index 705241d705b1..dfc5fac8c076 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -79,7 +79,7 @@ public KubernetesClient getClient() return baseClient.getClient(); } - public T readPodCache(KubernetesInformerExecutor executor) + public T readPodCache(SharedInformerCacheReadRequestExecutor executor) { if (podInformer == null) { throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); @@ -87,7 +87,7 @@ public T readPodCache(KubernetesInformerExecutor executor) return executor.executeRequest(podInformer); } - public T readJobCache(KubernetesInformerExecutor executor) + public T readJobCache(SharedInformerCacheReadRequestExecutor executor) { if (jobInformer == null) { throw DruidException.defensive("Job informer is not initialized, caching is disabled"); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java similarity index 94% rename from extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java rename to extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java index 53cd6c13575f..04291cd1046c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesInformerExecutor.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.client.informers.SharedIndexInformer; @FunctionalInterface -public interface KubernetesInformerExecutor +public interface SharedInformerCacheReadRequestExecutor { T executeRequest(SharedIndexInformer informer); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java index fe4ba186cec4..3b1df73f4235 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java @@ -230,13 +230,13 @@ private void notifyPodChange(Pod pod) } @Override - public T readPodCache(KubernetesInformerExecutor executor) + public T readPodCache(SharedInformerCacheReadRequestExecutor executor) { return executor.executeRequest(podInformer); } @Override - public T readJobCache(KubernetesInformerExecutor executor) + public T readJobCache(SharedInformerCacheReadRequestExecutor executor) { return executor.executeRequest(jobInformer); } From ed671f9c9cbd9f198154c96f06bfbd46052e285a Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 15:21:19 -0600 Subject: [PATCH 50/58] Simplify informer setup for caching client --- .../common/DruidKubernetesCachingClient.java | 102 ++++++++++-------- 1 file changed, 56 insertions(+), 46 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index dfc5fac8c076..bb6c57791d83 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -32,10 +32,54 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; import java.util.function.Function; public class DruidKubernetesCachingClient { + /** + * Event types for Kubernetes informer resource events. + */ + public enum InformerEventType + { + ADD, + UPDATE, + DELETE + } + + /** + * Impl of ResourceEventHandler that simplifies event handling + * by accepting a single lambda BiConsumer for all event types (add, update, delete). + * + * @param The Kubernetes resource type (e.g., Pod, Job) + */ + public static class InformerEventHandler implements ResourceEventHandler + { + private final BiConsumer eventConsumer; + + public InformerEventHandler(BiConsumer eventConsumer) + { + this.eventConsumer = eventConsumer; + } + + @Override + public void onAdd(T resource) + { + eventConsumer.accept(resource, InformerEventType.ADD); + } + + @Override + public void onUpdate(T oldResource, T newResource) + { + eventConsumer.accept(newResource, InformerEventType.UPDATE); + } + + @Override + public void onDelete(T resource, boolean deletedFinalStateUnknown) + { + eventConsumer.accept(resource, InformerEventType.DELETE); + } + } private static final EmittingLogger log = new EmittingLogger(DruidKubernetesCachingClient.class); public static final String JOB_NAME_INDEX = "byJobName"; @@ -109,29 +153,12 @@ private SharedIndexInformer setupPodInformer(String namespace) .inNamespace(namespace) .withLabel(DruidK8sConstants.LABEL_KEY) .inform( - new ResourceEventHandler<>() - { - @Override - public void onAdd(Pod pod) - { - log.debug("Pod[%s] got added", pod.getMetadata().getName()); - notifyPodChange(pod); - } - - @Override - public void onUpdate(Pod oldPod, Pod newPod) - { - log.debug("Pod[%s] got updated", oldPod.getMetadata().getName()); - notifyPodChange(newPod); - } - - @Override - public void onDelete(Pod pod, boolean deletedFinalStateUnknown) - { - log.debug("Pod[%s] got deleted", pod.getMetadata().getName()); - notifyPodChange(pod); - } - }, informerResyncPeriodMillis + new InformerEventHandler<>( + (pod, eventType) -> { + log.debug("Pod[%s] got %s", pod.getMetadata().getName(), eventType.name().toLowerCase()); + notifyPodChange(pod); + } + ), informerResyncPeriodMillis ); Function> jobNameIndexer = pod -> { @@ -167,29 +194,12 @@ private SharedIndexInformer setupJobInformer(String namespace) .inNamespace(namespace) .withLabel(DruidK8sConstants.LABEL_KEY) .inform( - new ResourceEventHandler<>() - { - @Override - public void onAdd(Job job) - { - log.debug("Job[%s] got added", job.getMetadata().getName()); - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - - @Override - public void onUpdate(Job oldJob, Job newJob) - { - log.debug("Job[%s] got updated", newJob.getMetadata().getName()); - eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); - } - - @Override - public void onDelete(Job job, boolean deletedFinalStateUnknown) - { - log.debug("Job[%s] got deleted", job.getMetadata().getName()); - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - }, informerResyncPeriodMillis + new InformerEventHandler<>( + (job, eventType) -> { + log.debug("Job[%s] got %s", job.getMetadata().getName(), eventType.name().toLowerCase()); + eventNotifier.notifyJobChange(job.getMetadata().getName(), job); + } + ), informerResyncPeriodMillis ); Function> overlordNamespaceIndexer = job -> { From 6b58a298ccaa8ef17cdf32a896d191b1ace42d9f Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 16:31:09 -0600 Subject: [PATCH 51/58] cleanup caching client tests and add a lifecycle stop to the informers --- .../overlord/KubernetesOverlordModule.java | 29 ++- .../common/DruidKubernetesCachingClient.java | 18 +- .../CachingKubernetesPeonClientTest.java | 16 +- .../common/TestCachingKubernetesClient.java | 207 +----------------- 4 files changed, 58 insertions(+), 212 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index 0c8832943696..508fde014803 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -205,7 +205,8 @@ public void stop() @Nullable public DruidKubernetesCachingClient makeCachingKubernetesClient( KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig, - DruidKubernetesClient baseClient + DruidKubernetesClient baseClient, + Lifecycle lifecycle ) { if (!kubernetesTaskRunnerConfig.isUseK8sSharedInformers()) { @@ -220,7 +221,31 @@ public DruidKubernetesCachingClient makeCachingKubernetesClient( .getMillis(); log.info("Creating Kubernetes caching client with informer resync period: %d ms", resyncPeriodMillis); - return new DruidKubernetesCachingClient(baseClient, namespace, resyncPeriodMillis); + final DruidKubernetesCachingClient cachingClient = new DruidKubernetesCachingClient( + baseClient, + namespace, + resyncPeriodMillis + ); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() + { + + } + + @Override + public void stop() + { + log.info("Stopping Kubernetes caching client"); + cachingClient.stop(); + } + } + ); + + return cachingClient; } /** diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index bb6c57791d83..6537c6c55361 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -87,10 +87,10 @@ public void onDelete(T resource, boolean deletedFinalStateUnknown) private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 300000L; // 5 minutes + protected final SharedIndexInformer podInformer; + protected final SharedIndexInformer jobInformer; + protected final KubernetesResourceEventNotifier eventNotifier; private final KubernetesClientApi baseClient; - private final SharedIndexInformer podInformer; - private final SharedIndexInformer jobInformer; - private final KubernetesResourceEventNotifier eventNotifier; private final long informerResyncPeriodMillis; public DruidKubernetesCachingClient( @@ -107,6 +107,18 @@ public DruidKubernetesCachingClient( this.jobInformer = setupJobInformer(namespace); } + public void stop() + { + if (podInformer != null) { + podInformer.stop(); + } + if (jobInformer != null) { + jobInformer.stop(); + } + // Cancel all pending futures in the event notifier + eventNotifier.cancelAll(); + } + public KubernetesClientApi getBaseClient() { return baseClient; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index f8e1084c3565..c13ac61c91c2 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -38,7 +38,9 @@ import java.net.HttpURLConnection; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; @EnableKubernetesMockClient(crud = true) public class CachingKubernetesPeonClientTest @@ -84,6 +86,7 @@ public void test_getPeonPod_withPodInCache_returnsPresentOptional() throws Excep .withNewMetadata() .withName(POD_NAME) .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .addToLabels("job-name", JOB_NAME) .endMetadata() .withNewStatus() @@ -121,6 +124,7 @@ public void test_getPeonPod_withMultiplePodsForSameJob_returnsFirstOne() throws .withNewMetadata() .withName("pod-1") .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .addToLabels("job-name", JOB_NAME) .endMetadata() .build(); @@ -129,6 +133,7 @@ public void test_getPeonPod_withMultiplePodsForSameJob_returnsFirstOne() throws .withNewMetadata() .withName("pod-2") .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .addToLabels("job-name", JOB_NAME) .endMetadata() .build(); @@ -308,6 +313,7 @@ public void test_waitUntilPeonPodCreatedAndReady_podBecomesReady() throws Except .withNewMetadata() .withName(POD_NAME) .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .addToLabels("job-name", JOB_NAME) .endMetadata() .withNewStatus() @@ -330,6 +336,7 @@ public void test_waitUntilPeonPodCreatedAndReady_podBecomesReady() throws Except .withNewMetadata() .withName(POD_NAME) .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .addToLabels("job-name", JOB_NAME) .endMetadata() .withNewStatus() @@ -355,6 +362,7 @@ public void test_waitUntilPeonPodCreatedAndReady_timeoutWhenPodNotReady() throws .withNewMetadata() .withName(POD_NAME) .withNamespace(NAMESPACE) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .addToLabels("job-name", JOB_NAME) .endMetadata() .withNewStatus() @@ -524,19 +532,21 @@ public void test_waitForPeonJobCompletion_jobDeletedBeforeSeenInCache() throws E } @Test - void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() throws InterruptedException + void test_getPeonLogsWatcher_withJob_returnsWatchLogInOptional() + throws InterruptedException, TimeoutException, ExecutionException { K8sTaskId taskId = new K8sTaskId("", "id"); Pod pod = new PodBuilder() .withNewMetadata() .withName(POD_NAME) + .addToLabels(DruidK8sConstants.LABEL_KEY, "true") .addToLabels("job-name", taskId.getK8sJobName()) .endMetadata() .build(); + CompletableFuture podFuture = cachingClient.waitForPodChange(taskId.getK8sJobName()); client.pods().inNamespace(NAMESPACE).resource(pod).create(); - - cachingClient.waitForSync(); + podFuture.get(5, TimeUnit.SECONDS); server.expect().get() .withPath("/api/v1/namespaces/namespace/pods/id/log?pretty=false&container=main") diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java index 3b1df73f4235..db60be29f1bb 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java @@ -19,225 +19,24 @@ package org.apache.druid.k8s.overlord.common; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.batch.v1.Job; -import io.fabric8.kubernetes.client.KubernetesClient; -import io.fabric8.kubernetes.client.informers.ResourceEventHandler; -import io.fabric8.kubernetes.client.informers.SharedIndexInformer; - -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; - public class TestCachingKubernetesClient extends DruidKubernetesCachingClient { - private final KubernetesClient client; - private final SharedIndexInformer podInformer; - private final SharedIndexInformer jobInformer; - private final KubernetesResourceEventNotifier eventNotifier; - private final CountDownLatch syncLatch; + private static final long TESTING_RESYNC_PERIOD_MS = 10L; public TestCachingKubernetesClient(KubernetesClientApi clientApi, String namespace) { - super(clientApi, namespace, 1000L); - this.client = clientApi.getClient(); - this.eventNotifier = new KubernetesResourceEventNotifier(); - this.syncLatch = new CountDownLatch(2); // Wait for both informers - - // Set up pod informer with real event handlers - this.podInformer = client.pods() - .inNamespace(namespace) - .inform( - new ResourceEventHandler() - { - @Override - public void onAdd(Pod pod) - { - notifyPodChange(pod); - } - - @Override - public void onUpdate(Pod oldPod, Pod newPod) - { - notifyPodChange(newPod); - } - - @Override - public void onDelete(Pod pod, boolean deletedFinalStateUnknown) - { - notifyPodChange(pod); - } - }, 1000L - ); - - // Add pod indexer - Map>> podIndexers = new HashMap<>(); - podIndexers.put( - "byJobName", pod -> { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - return Collections.singletonList(jobName); - } - } - return Collections.emptyList(); - } - ); - podInformer.addIndexers(podIndexers); - - // Set up job informer with real event handlers - this.jobInformer = client.batch() - .v1() - .jobs() - .inNamespace(namespace) - .withLabel(DruidK8sConstants.LABEL_KEY) - .inform( - new ResourceEventHandler() - { - @Override - public void onAdd(Job job) - { - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - - @Override - public void onUpdate(Job oldJob, Job newJob) - { - eventNotifier.notifyJobChange(newJob.getMetadata().getName(), newJob); - } - - @Override - public void onDelete(Job job, boolean deletedFinalStateUnknown) - { - eventNotifier.notifyJobChange(job.getMetadata().getName(), job); - } - }, 1000L - ); - - // Add job indexers - Map>> jobIndexers = new HashMap<>(); - jobIndexers.put( - "byJobName", job -> { - if (job.getMetadata() != null && job.getMetadata().getName() != null) { - return Collections.singletonList(job.getMetadata().getName()); - } - return Collections.emptyList(); - } - ); - jobIndexers.put( - "byOverlordNamespace", job -> { - if (job.getMetadata() != null && job.getMetadata().getLabels() != null) { - String overlordNamespace = job.getMetadata().getLabels().get(DruidK8sConstants.OVERLORD_NAMESPACE_KEY); - if (overlordNamespace != null) { - return Collections.singletonList(overlordNamespace); - } - } - return Collections.emptyList(); - } - ); - jobInformer.addIndexers(jobIndexers); + super(clientApi, namespace, TESTING_RESYNC_PERIOD_MS); } public void start() { - // Add ready callbacks to count down latch - podInformer.addEventHandlerWithResyncPeriod( - new ResourceEventHandler() - { - @Override - public void onAdd(Pod obj) - { - - } - - @Override - public void onUpdate(Pod oldObj, Pod newObj) - { - - } - - @Override - public void onDelete(Pod obj, boolean deletedFinalStateUnknown) - { - - } - }, 1000L - ); - - jobInformer.addEventHandlerWithResyncPeriod( - new ResourceEventHandler() - { - @Override - public void onAdd(Job obj) - { - - } - - @Override - public void onUpdate(Job oldObj, Job newObj) - { - - } - - @Override - public void onDelete(Job obj, boolean deletedFinalStateUnknown) - { - - } - }, 1000L - ); - podInformer.run(); jobInformer.run(); - - // Count down after starting - syncLatch.countDown(); - syncLatch.countDown(); - } - - public void stop() - { - if (podInformer != null) { - podInformer.stop(); - } - if (jobInformer != null) { - jobInformer.stop(); - } - if (eventNotifier != null) { - eventNotifier.cancelAll(); - } } public void waitForSync() throws InterruptedException { - syncLatch.await(5, TimeUnit.SECONDS); // Give informers a bit more time to process - Thread.sleep(200); - } - - private void notifyPodChange(Pod pod) - { - if (pod.getMetadata() != null && pod.getMetadata().getLabels() != null) { - String jobName = pod.getMetadata().getLabels().get("job-name"); - if (jobName != null) { - eventNotifier.notifyPodChange(jobName, pod); - } - } - } - - @Override - public T readPodCache(SharedInformerCacheReadRequestExecutor executor) - { - return executor.executeRequest(podInformer); - } - - @Override - public T readJobCache(SharedInformerCacheReadRequestExecutor executor) - { - return executor.executeRequest(jobInformer); + Thread.sleep(50L); } } From 62ddcc35da882a234e6e5ef7694a2e22077f0af8 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 5 Dec 2025 16:47:09 -0600 Subject: [PATCH 52/58] Improve thread safety of KubernetesResourceEventNotifier --- .../KubernetesResourceEventNotifier.java | 52 +++++++++---------- .../KubernetesResourceEventNotifierTest.java | 12 ++--- 2 files changed, 29 insertions(+), 35 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java index 58a704e5d4d6..13038b82dc72 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java @@ -23,10 +23,8 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import org.apache.druid.java.util.emitter.EmittingLogger; -import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; /** * Manages event notifications for Kubernetes resources (Jobs and Pods). @@ -39,8 +37,8 @@ public class KubernetesResourceEventNotifier { private static final EmittingLogger log = new EmittingLogger(KubernetesResourceEventNotifier.class); - private final ConcurrentHashMap>> jobWatchers = new ConcurrentHashMap<>(); - private final ConcurrentHashMap>> podWatchers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> jobWatchers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> podWatchers = new ConcurrentHashMap<>(); /** * Register to be notified when a job with the given name changes. @@ -51,10 +49,11 @@ public class KubernetesResourceEventNotifier */ public CompletableFuture waitForJobChange(String jobName) { - CompletableFuture future = new CompletableFuture<>(); - jobWatchers.computeIfAbsent(jobName, k -> new CopyOnWriteArrayList<>()).add(future); - log.debug("Registered watcher for job [%s]. Total watchers: %d", jobName, jobWatchers.get(jobName).size()); - return future; + return jobWatchers.computeIfAbsent(jobName, k -> { + log.debug("Creating new watcher for job [%s]", jobName); + return new CompletableFuture<>(); + }); + } /** @@ -66,42 +65,39 @@ public CompletableFuture waitForJobChange(String jobName) */ public CompletableFuture waitForPodChange(String jobName) { - CompletableFuture future = new CompletableFuture<>(); - podWatchers.computeIfAbsent(jobName, k -> new CopyOnWriteArrayList<>()).add(future); - log.debug("Registered watcher for pod with job-name [%s]. Total watchers: %d", jobName, podWatchers.get(jobName).size()); - return future; + return podWatchers.computeIfAbsent(jobName, k -> { + log.debug("Creating new watcher for pod with job-name [%s]", jobName); + return new CompletableFuture<>(); + }); } /** - * Notify all watchers that a job with the given name has changed. - * Completes all pending futures for this job and clears the watcher list. + * Notify all watchers that a job with the given name has changed and remove the watcher from the map. * * @param jobName The name of the job that changed */ public void notifyJobChange(String jobName, Job job) { - List> futures = jobWatchers.get(jobName); - if (futures != null && !futures.isEmpty()) { - log.debug("Notifying %d watchers of job [%s] change", futures.size(), jobName); - futures.forEach(f -> f.complete(job)); - futures.clear(); + CompletableFuture future = jobWatchers.remove(jobName); + if (future != null) { + log.debug("Notifying watchers of job [%s] change", jobName); + future.complete(job); } } /** - * Notify all watchers that a pod for the given job name has changed. - * Completes all pending futures for pods with this job-name label and clears the watcher list. + * Notify all watchers that a pod for the given job name has changed and remove the watcher from the map. * * @param jobName The job-name label value that changed */ public void notifyPodChange(String jobName, Pod pod) { - List> futures = podWatchers.get(jobName); - if (futures != null && !futures.isEmpty()) { - log.debug("Notifying %d watchers of pod change for job-name [%s]", futures.size(), jobName); - futures.forEach(f -> f.complete(pod)); - futures.clear(); + CompletableFuture future = podWatchers.remove(jobName); + if (future != null) { + log.debug("Notifying watchers of pod change for job-name [%s]", jobName); + future.complete(pod); } + } /** @@ -110,8 +106,8 @@ public void notifyPodChange(String jobName, Pod pod) public void cancelAll() { log.info("Cancelling all pending watchers"); - jobWatchers.values().forEach(futures -> futures.forEach(f -> f.cancel(true))); - podWatchers.values().forEach(futures -> futures.forEach(f -> f.cancel(true))); + jobWatchers.values().forEach(f -> f.cancel(true)); + podWatchers.values().forEach(f -> f.cancel(true)); jobWatchers.clear(); podWatchers.clear(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java index 09dec764e4ff..d4e8cb11370f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java @@ -30,8 +30,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -270,27 +270,25 @@ public void testJobAndPodWatchers_Independent() throws Exception } @Test - public void testWaitForJobChange_ReturnsNewFutureEachTime() + public void testWaitForJobChange_returnsSameFutureIfOneExists() { String jobName = "test-job"; CompletableFuture future1 = notifier.waitForJobChange(jobName); CompletableFuture future2 = notifier.waitForJobChange(jobName); - // Should be different future instances - assertNotEquals(future1, future2); + assertEquals(future1, future2); } @Test - public void testWaitForPodChange_ReturnsNewFutureEachTime() + public void testWaitForPodChange_returnsSameFutureIfOneExists() { String jobName = "test-job"; CompletableFuture future1 = notifier.waitForPodChange(jobName); CompletableFuture future2 = notifier.waitForPodChange(jobName); - // Should be different future instances - assertNotEquals(future1, future2); + assertEquals(future1, future2); } private Job createMockJob(String name) From e58be26f7b87901adb587790644deff442e5de6e Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 7 Dec 2025 21:02:00 -0600 Subject: [PATCH 53/58] Simply the peon waiting code for the caching client --- docs/development/extensions-core/k8s-jobs.md | 6 +- .../common/CachingKubernetesPeonClient.java | 155 ++++++++---------- .../common/DruidKubernetesCachingClient.java | 14 +- .../overlord/common/KubernetesPeonClient.java | 30 +++- .../KubernetesResourceEventNotifier.java | 80 +++++++-- .../KubernetesResourceEventNotifierTest.java | 62 +------ 6 files changed, 175 insertions(+), 172 deletions(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index f0a23e25bb7e..903ddfdfed6c 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -41,7 +41,7 @@ Task lifecycle code in Druid talks directly to the Kubernetes API server for all ### `SharedInformer` "Caching" *(Experimental)* -Enabled by setting `druid.indexer.runner.enableKubernetesClientSharedInformers=true`, this mode uses `Fabric8` `SharedInformer` objects for monitoring state changes in the remote K8s cluster, reducing the number of direct API calls to the Kubernetes API server. This can greatly reduce load on the API server, especially in environments with a high volume of tasks. +Enabled by setting `druid.indexer.runner.useK8sSharedInformers=true`, this mode uses `Fabric8` `SharedInformer` objects for monitoring state changes in the remote K8s cluster, reducing the number of direct API calls to the Kubernetes API server. This can greatly reduce load on the API server, especially in environments with a high volume of tasks. This mode is experimental and should be used with caution in production until it has been vetted more thoroughly by the community. @@ -818,8 +818,8 @@ Should you require the needed permissions for interacting across Kubernetes name | `druid.indexer.runner.capacity` | `Integer` | Number of concurrent jobs that can be sent to Kubernetes. | `2147483647` | No | | `druid.indexer.runner.cpuCoreInMicro` | `Integer` | Number of CPU micro core for the task. | `1000` | No | | `druid.indexer.runner.logSaveTimeout` | `Duration` | The peon executing the ingestion task makes a best effort to persist the pod logs from `k8s` to persistent task log storage. The timeout ensures that `k8s` connection issues do not cause the pod to hang indefinitely thereby blocking Overlord operations. If the timeout occurs before the logs are saved, those logs will not be available in Druid. | `PT300S` | NO | -| `druid.indexer.runner.enableKubernetesClientSharedInformers` | `boolean` | Whether to use shared informers to watch for pod/job changes. This is more efficient on the Kubernetes API server, but may use more memory in the Overlord. | `false` | No | -| `druid.indexer.runner.kubernetesClientInformerResyncPeriod` | `Duration` | When using shared informers, controls how frequently the informers resync with the Kubernetes API server. This prevents change events from being missed, keeping the informer cache clean and accurate. | `PT300S` | No | +| `druid.indexer.runner.useK8sSharedInformers` | `boolean` | Whether to use shared informers to watch for pod/job changes. This is more efficient on the Kubernetes API server, but may use more memory in the Overlord. | `false` | No | +| `druid.indexer.runner.k8sSharedInformerResyncPeriod` | `Duration` | When using shared informers, controls how frequently the informers resync with the Kubernetes API server. This prevents change events from being missed, keeping the informer cache clean and accurate. | `PT300S` | No | ### Metrics added diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 1f2c96cae333..07f50f980832 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -30,7 +30,9 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -69,16 +71,12 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time Stopwatch stopwatch = Stopwatch.createStarted(); boolean jobSeenInCache = false; - // Set up to watch for job changes - CompletableFuture jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); - - // We will loop until the full timeout is reached if the job is seen in cache. If the job does not show up in the cache we will exit earlier. - // In this loop we first check the cache to see if our job is there and complete. This avoids missing notifications that happened before we set up the watch. - // If the job is not complete we wait for a notification of a job change or a timeout. - // If it is a timeout, we loop back to check the cache again. - // If it is a job change notification, we check the job state and exit if complete, or loop again if still running. - do { - try { + try { + CompletableFuture jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); + while (stopwatch.hasNotElapsed(timeout) && (jobSeenInCache || stopwatch.hasNotElapsed(jobMustBeSeenWithin))) { + if (jobFuture.isDone()) { + jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); + } Optional maybeJob = getPeonJob(taskId.getK8sJobName()); if (maybeJob.isPresent()) { jobSeenInCache = true; @@ -93,40 +91,34 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time // Job was in cache before, but now it's gone - it was deleted and will never complete. log.warn("K8s Job[%s] was not found. It can happen if the task was canceled", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); + } else { + log.debug("K8s job[%s] not yet found in cache", taskId.getK8sJobName()); } - // We wake up every informer resync period to avoid event notifier misses. - Job job = jobFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); - - // Immediately set up to watch for the next change in case we need to wait again - jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); - log.debug("Received job[%s] change notification", taskId.getK8sJobName()); - jobSeenInCache = true; - - if (job == null) { - log.warn("K8s job[%s] was not found. It can happen if the task was canceled", taskId.getK8sJobName()); - return new JobResponse(null, PeonPhase.FAILED); + try { + jobFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); } - - JobResponse currentResponse = determineJobResponse(job); - if (currentResponse.getPhase() != PeonPhase.RUNNING) { - return currentResponse; - } else { - log.debug("K8s job[%s] is still running", taskId.getK8sJobName()); + catch (ExecutionException | CancellationException e) { + Throwable cause = e.getCause(); + if (cause instanceof CancellationException) { + log.noStackTrace().warn("Job change watch for job[%s] was cancelled", taskId.getK8sJobName()); + } else { + log.noStackTrace().warn(cause, "Exception while waiting for change notification of job[%s]", taskId.getK8sJobName()); + } + } + catch (TimeoutException e) { + // No job change event notified within the timeout time. If there is more time, it will loop back and check the cache again. + log.debug("Timeout waiting for change notification of job[%s].", taskId.getK8sJobName()); + } + catch (InterruptedException e) { + throw DruidException.defensive(e, "Interrupted waiting for job change notification for job[%s]", taskId.getK8sJobName()); } } - catch (TimeoutException e) { - // A timeout here is not a problem, it forces us to loop around and check the cache again. - // This prevents the case where we miss a notification and wait forever. - log.debug("Timeout waiting for change notification of job[%s]. Waiting until full job timeout.", taskId.getK8sJobName()); - } - catch (InterruptedException e) { - throw DruidException.defensive(e, "Interrupted waiting for job change notification for job[%s]", taskId.getK8sJobName()); - } - catch (Throwable e) { - log.noStackTrace().warn(e, "Exception while waiting for change notification of job[%s]", taskId.getK8sJobName()); - } - } while (stopwatch.hasNotElapsed(timeout) && (jobSeenInCache || stopwatch.hasNotElapsed(jobMustBeSeenWithin))); + } + finally { + // Clean up: remove from map and cancel if still pending + cachingClient.cancelJobWatcher(taskId.getK8sJobName()); + } log.warn("Timed out waiting for K8s job[%s] to complete", taskId.getK8sJobName()); return new JobResponse(null, PeonPhase.FAILED); @@ -161,78 +153,61 @@ public Optional getPeonJob(String jobName) }); } + @Override @Nullable protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) { Duration timeout = Duration.millis(timeUnit.toMillis(howLong)); Stopwatch stopwatch = Stopwatch.createStarted(); - String podName = "unknown"; - boolean podSeenInCache = false; - - // Set up to watch for pod changes - CompletableFuture podFuture = cachingClient.waitForPodChange(jobName); - // We will loop until the specified timeout is reached, or we see the pod become ready, whichever comes first. - // We eagerly check the cache first to avoid missing notifications that happened before we set up the watch. - // If the pod is not ready we wait for a notification of a pod change or a timeout. - // If it is a timeout, we loop back to check the cache again (if there is time) - // If it is a pod change notification, we check the pod state and exit if ready, or loop again if still not ready. - do { - try { - // First check to see if pod is already in cache and ready in case our completion future started after the update event fired + try { + CompletableFuture podFuture = cachingClient.waitForPodChange(jobName); + while (stopwatch.hasNotElapsed(timeout)) { + if (podFuture.isDone()) { + podFuture = cachingClient.waitForPodChange(jobName); + } Optional maybePod = getPeonPod(jobName); if (maybePod.isPresent()) { - podSeenInCache = true; Pod pod = maybePod.get(); - podName = pod.getMetadata().getName(); - + String podName = pod.getMetadata() != null && pod.getMetadata().getName() != null + ? pod.getMetadata().getName() + : "unknown"; if (isPodRunningOrComplete(pod)) { - log.info("Pod[%s] for job[%s] is running or complete", podName, jobName); + log.info("Pod[%s] for job[%s] is now in Running/Complete state", podName, jobName); return pod; } else { - log.debug("Pod[%s] for job[%s] exists but not ready yet", podName, jobName); + log.debug("Pod[%s] for job[%s] found in cache but not yet Running/Complete", podName, jobName); } } else { - log.info("Pod for job[%s] not created yet", jobName); + log.debug("Pod for job[%s] not yet found in cache", jobName); } - // We wake up every informer resync period to avoid event notifier misses. - Pod pod = podFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); - - podFuture = cachingClient.waitForPodChange(jobName); - log.debug("Received pod[%s] change notification for job[%s]", podName, jobName); - if (pod == null) { - log.warn("Pod[%s] for job[%s] is null. This is unusual. Investigate Druid and k8s logs.", podName, jobName); - return null; - } else { - podSeenInCache = true; - podName = pod.getMetadata().getName(); - if (isPodRunningOrComplete(pod)) { - log.info("Pod[%s] for job[%s] is running or complete", podName, jobName); - return pod; + try { + podFuture.get(cachingClient.getInformerResyncPeriodMillis(), TimeUnit.MILLISECONDS); + } + catch (ExecutionException | CancellationException e) { + // This is unusual. Log warning but try to continue + Throwable cause = e.getCause(); + if (cause instanceof CancellationException) { + log.noStackTrace().warn("Pod change watch for job[%s] was cancelled", jobName); } else { - log.debug("Pod[%s] for job[%s] exists but not ready yet", podName, jobName); + log.noStackTrace().warn(cause, "Unexpected exception while waiting for pod change notification for job[%s]", jobName); } } + catch (TimeoutException e) { + // No pod change event notified within the timeout time. If there is more time, it will loop back and check the cache again. + log.debug("Timeout waiting for change notification of pod for job[%s].", jobName); + } + catch (InterruptedException e) { + throw DruidException.defensive(e, "Interrupted waiting for pod change notification for job[%s]", jobName); + } } - catch (TimeoutException e) { - // A timeout here is not a problem, it forces us to loop around and check the cache again. - // This prevents the case where we miss a notification and wait forever. - log.debug("Timeout waiting for pod change notification for job[%s], If full timeout has not been reached, the pod startup wait will continue", jobName); - } - catch (InterruptedException e) { - throw DruidException.defensive(e, "Interrupted waiting for pod change notification for job[%s]", jobName); - } - catch (Throwable e) { - log.warn("Unexpected exception[%s] waiting for pod change notification for job [%s]. Error message[%s]", e.getClass().getName(), jobName, e.getMessage()); - } - } while (stopwatch.hasNotElapsed(timeout)); - - if (podSeenInCache) { - log.warn("Timeout waiting for pod[%s] for job[%s] to become ready after it was created", podName, jobName); - } else { - log.warn("Timeout waiting for pod for job[%s] to be created", jobName); } + finally { + // Clean up: remove from map and cancel if still pending + cachingClient.cancelPodWatcher(jobName); + } + log.warn("Timed out waiting for pod for job[%s] to be created and ready", jobName); return null; } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index 6537c6c55361..25e7749e3029 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -167,7 +167,7 @@ private SharedIndexInformer setupPodInformer(String namespace) .inform( new InformerEventHandler<>( (pod, eventType) -> { - log.debug("Pod[%s] got %s", pod.getMetadata().getName(), eventType.name().toLowerCase()); + log.debug("Pod[%s] got %s", pod.getMetadata().getName(), eventType.name()); notifyPodChange(pod); } ), informerResyncPeriodMillis @@ -208,7 +208,7 @@ private SharedIndexInformer setupJobInformer(String namespace) .inform( new InformerEventHandler<>( (job, eventType) -> { - log.debug("Job[%s] got %s", job.getMetadata().getName(), eventType.name().toLowerCase()); + log.debug("Job[%s] got %s", job.getMetadata().getName(), eventType.name()); eventNotifier.notifyJobChange(job.getMetadata().getName(), job); } ), informerResyncPeriodMillis @@ -263,6 +263,16 @@ public CompletableFuture waitForPodChange(String jobName) return eventNotifier.waitForPodChange(jobName); } + public void cancelJobWatcher(String jobName) + { + eventNotifier.cancelJobWatcher(jobName); + } + + public void cancelPodWatcher(String jobName) + { + eventNotifier.cancelPodWatcher(jobName); + } + public long getInformerResyncPeriodMillis() { return informerResyncPeriodMillis; diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 8ecbc44d794c..b57d4378386f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -90,12 +90,7 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn createK8sJobWithRetries(job); log.info("Submitted job[%s] for task[%s]. Waiting for POD to launch.", jobName, task.getId()); - // Wait for the pod to be available - Pod mainPod = getPeonPodWithRetries(jobName); - log.info("Pod for job[%s] launched for task[%s]. Waiting for pod to be in running state.", jobName, task.getId()); - - // Wait for the pod to be in state running, completed, or failed. - Pod result = waitForPodResultWithRetries(mainPod, howLong, timeUnit); + Pod result = waitUntilPeonPodCreatedAndReady(jobName, howLong, timeUnit); if (result == null) { throw new ISE("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", task.getId()); @@ -107,6 +102,29 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn }); } + /** + * Waits until a pod for the given job is created and ready to be monitored. + *

    + * A pod can appear and dissapear in some cases, such as the task being canceled. In this case, null is returned and + * the caller should handle accordingly. + *

    + * + * @param jobName the name of the job whose pod we're waiting for + * @param howLong the maximum time to wait + * @param timeUnit the time unit for the timeout + * @return the {@link Pod} which was waited for or null if the pod appeared and dissapeared + * @throws DruidException if the pod never appears within the timeout period + */ + protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) + { + // Wait for the pod to be available + Pod mainPod = getPeonPodWithRetries(jobName); + log.info("Pod for job[%s] launched. Waiting for pod to be in running state.", jobName); + + // Wait for the pod to be in state running, completed, or failed. + return waitForPodResultWithRetries(mainPod, howLong, timeUnit); + } + public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) { return clientApi.executeRequest(client -> { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java index 13038b82dc72..6252a6f4bd3e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java @@ -30,7 +30,11 @@ * Manages event notifications for Kubernetes resources (Jobs and Pods). *

    * Allows tasks to wait for specific resource changes without polling, improving efficiency and responsiveness. - * Crtical component of {@link CachingKubernetesPeonClient} functionality. + * Critical component of {@link CachingKubernetesPeonClient} functionality. + *

    + *

    + * This implementation assumes only one waiter per job/pod at a time. If a new waiter is registered for a job that + * already has one, the previous waiter will be cancelled. *

    */ public class KubernetesResourceEventNotifier @@ -42,62 +46,106 @@ public class KubernetesResourceEventNotifier /** * Register to be notified when a job with the given name changes. - * The returned future will complete when the job is added, updated, or deleted. + *

    + * IMPORTANT: Callers must call {@link #cancelJobWatcher(String)} when done waiting to avoid resource leaks. * * @param jobName The name of the job to watch * @return A future that completes when the job changes */ public CompletableFuture waitForJobChange(String jobName) { - return jobWatchers.computeIfAbsent(jobName, k -> { - log.debug("Creating new watcher for job [%s]", jobName); - return new CompletableFuture<>(); - }); + CompletableFuture future = new CompletableFuture<>(); + CompletableFuture previous = jobWatchers.put(jobName, future); + + if (previous != null && !previous.isDone()) { + log.warn("Replacing active watcher for job [%s] - multiple waiters detected", jobName); + previous.cancel(true); + } + log.debug("Registered watcher for job [%s]", jobName); + return future; } /** * Register to be notified when a pod for the given job name changes. - * The returned future will complete when a pod with the job-name label changes. + *

    + * IMPORTANT: Callers must call {@link #cancelPodWatcher(String)} when done waiting to avoid resource leaks. * * @param jobName The job-name label value to watch for * @return A future that completes when a matching pod changes */ public CompletableFuture waitForPodChange(String jobName) { - return podWatchers.computeIfAbsent(jobName, k -> { - log.debug("Creating new watcher for pod with job-name [%s]", jobName); - return new CompletableFuture<>(); - }); + CompletableFuture future = new CompletableFuture<>(); + CompletableFuture previous = podWatchers.put(jobName, future); + + if (previous != null && !previous.isDone()) { + log.warn("Replacing active watcher for pod with job-name [%s] - multiple waiters detected", jobName); + previous.cancel(true); + } + + log.debug("Registered watcher for pod with job-name [%s]", jobName); + return future; + } + + /** + * Cancel and remove a job watcher. Safe to call even if the future has already completed. + * + * @param jobName The name of the job to stop watching + */ + public void cancelJobWatcher(String jobName) + { + CompletableFuture future = jobWatchers.remove(jobName); + if (future != null && !future.isDone()) { + log.debug("Cancelling watcher for job [%s]", jobName); + future.cancel(true); + } + } + + /** + * Cancel and remove a pod watcher. Safe to call even if the future has already completed. + * + * @param jobName The job-name label value to stop watching + */ + public void cancelPodWatcher(String jobName) + { + CompletableFuture future = podWatchers.remove(jobName); + if (future != null && !future.isDone()) { + log.debug("Cancelling watcher for pod with job-name [%s]", jobName); + future.cancel(true); + } } /** - * Notify all watchers that a job with the given name has changed and remove the watcher from the map. + * Notify the waiter that a job with the given name has changed. + * Completes the future and removes it from the map. * * @param jobName The name of the job that changed + * @param job The job that changed */ public void notifyJobChange(String jobName, Job job) { CompletableFuture future = jobWatchers.remove(jobName); if (future != null) { - log.debug("Notifying watchers of job [%s] change", jobName); + log.debug("Notifying watcher of job [%s] change", jobName); future.complete(job); } } /** - * Notify all watchers that a pod for the given job name has changed and remove the watcher from the map. + * Notify the waiter that a pod for the given job name has changed. + * Completes the future and removes it from the map. * * @param jobName The job-name label value that changed + * @param pod The pod that changed */ public void notifyPodChange(String jobName, Pod pod) { CompletableFuture future = podWatchers.remove(jobName); if (future != null) { - log.debug("Notifying watchers of pod change for job-name [%s]", jobName); + log.debug("Notifying watcher of pod change for job-name [%s]", jobName); future.complete(pod); } - } /** diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java index d4e8cb11370f..46dea8b2b60e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifierTest.java @@ -30,8 +30,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -84,56 +84,6 @@ public void testWaitForPodChange_CompletesOnNotification() throws Exception assertTrue(future.isDone()); } - @Test - public void testMultipleWatchers_AllNotified() throws Exception - { - String jobName = "test-job"; - Job mockJob = createMockJob(jobName); - - CompletableFuture future1 = notifier.waitForJobChange(jobName); - CompletableFuture future2 = notifier.waitForJobChange(jobName); - CompletableFuture future3 = notifier.waitForJobChange(jobName); - - assertFalse(future1.isDone()); - assertFalse(future2.isDone()); - assertFalse(future3.isDone()); - - notifier.notifyJobChange(jobName, mockJob); - - Job result1 = future1.get(1, TimeUnit.SECONDS); - Job result2 = future2.get(1, TimeUnit.SECONDS); - Job result3 = future3.get(1, TimeUnit.SECONDS); - - assertSame(mockJob, result1); - assertSame(mockJob, result2); - assertSame(mockJob, result3); - } - - @Test - public void testMultiplePodWatchers_AllNotified() throws Exception - { - String jobName = "test-job"; - Pod mockPod = createMockPod(jobName); - - CompletableFuture future1 = notifier.waitForPodChange(jobName); - CompletableFuture future2 = notifier.waitForPodChange(jobName); - CompletableFuture future3 = notifier.waitForPodChange(jobName); - - assertFalse(future1.isDone()); - assertFalse(future2.isDone()); - assertFalse(future3.isDone()); - - notifier.notifyPodChange(jobName, mockPod); - - Pod result1 = future1.get(1, TimeUnit.SECONDS); - Pod result2 = future2.get(1, TimeUnit.SECONDS); - Pod result3 = future3.get(1, TimeUnit.SECONDS); - - assertSame(mockPod, result1); - assertSame(mockPod, result2); - assertSame(mockPod, result3); - } - @Test public void testNotifyWithoutWatchers_NoException() { @@ -270,25 +220,27 @@ public void testJobAndPodWatchers_Independent() throws Exception } @Test - public void testWaitForJobChange_returnsSameFutureIfOneExists() + public void test_waitForJobChange_multipleWaitsCancelsOldFutureAndCreatesNewOne() { String jobName = "test-job"; CompletableFuture future1 = notifier.waitForJobChange(jobName); CompletableFuture future2 = notifier.waitForJobChange(jobName); - assertEquals(future1, future2); + assertTrue(future1.isCancelled()); + assertNotEquals(future1, future2); } @Test - public void testWaitForPodChange_returnsSameFutureIfOneExists() + public void test_waitForPodChange_multipleWaitsCancelsOldFutureAndCreatesNewOne() { String jobName = "test-job"; CompletableFuture future1 = notifier.waitForPodChange(jobName); CompletableFuture future2 = notifier.waitForPodChange(jobName); - assertEquals(future1, future2); + assertTrue(future1.isCancelled()); + assertNotEquals(future1, future2); } private Job createMockJob(String name) From 4b7a40896266462b4e91d232df90577e9ca4f169 Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 7 Dec 2025 21:34:40 -0600 Subject: [PATCH 54/58] Fix the k8s overlord module for the caching client --- .../org/apache/druid/k8s/overlord/KubernetesOverlordModule.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index 66eb450ba4da..b6ba2fdd6773 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -214,7 +214,7 @@ public void stop() @LazySingleton @Nullable public DruidKubernetesCachingClient makeCachingKubernetesClient( - KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig, + KubernetesTaskRunnerStaticConfig kubernetesTaskRunnerConfig, DruidKubernetesClient baseClient, Lifecycle lifecycle ) From ef93c44b1ada17845e13fe253b801b1c104c2ba8 Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 7 Dec 2025 22:55:41 -0600 Subject: [PATCH 55/58] fix configs for docker embedded test --- .../embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java index 2677612f1bf4..75ae9a8e6159 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/k8s/BaseKubernetesTaskRunnerDockerTest.java @@ -50,8 +50,8 @@ protected EmbeddedDruidCluster addServers(EmbeddedDruidCluster cluster) .addProperty("druid.indexer.runner.type", "k8s") .addProperty("druid.indexer.runner.namespace", "druid") .addProperty("druid.indexer.runner.capacity", "4") - .addProperty("druid.indexer.runner.enableKubernetesClientSharedInformers", String.valueOf(useSharedInformers())) - .addProperty("druid.indexer.runner.kubernetesClientInformerResyncPeriod", "PT1s") + .addProperty("druid.indexer.runner.useK8sSharedInformers", String.valueOf(useSharedInformers())) + .addProperty("druid.indexer.runner.k8sSharedInformerResyncPeriod", "PT1s") .usingPort(30090); final K3sClusterResource k3sCluster = new K3sClusterWithOperatorResource() From 53d09763027cb22b2ddad524f5ef4152826c898d Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 8 Dec 2025 09:06:28 -0600 Subject: [PATCH 56/58] fix broken embedded tests --- .../apache/druid/testing/embedded/EmbeddedClusterApis.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java index 65bbd0ef155d..d93ae182f4bd 100644 --- a/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java +++ b/services/src/test/java/org/apache/druid/testing/embedded/EmbeddedClusterApis.java @@ -52,8 +52,8 @@ import java.io.Closeable; import java.util.ArrayList; -import java.util.Collection; import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -236,7 +236,7 @@ public TaskStatus waitForTaskToFinish(String taskId, LatchableEmitter emitter) */ public int getTaskCount(String status, String dataSource) { - return ImmutableList.copyOf((Collection) onLeaderOverlord(o -> o.taskStatuses(status, dataSource, 100))).size(); + return ImmutableList.copyOf((Iterator) onLeaderOverlord(o -> o.taskStatuses(status, dataSource, 100))).size(); } /** From b2982c32b6b99abeb06af7c98e1bd51232d0a66a Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 8 Dec 2025 11:06:12 -0600 Subject: [PATCH 57/58] use the indexer not informer for cache reads --- .../common/CachingKubernetesPeonClient.java | 17 +++++++++-------- .../common/DruidKubernetesCachingClient.java | 4 ++-- .../SharedInformerCacheReadRequestExecutor.java | 4 ++-- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 07f50f980832..6dd5d70b6386 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -22,6 +22,7 @@ import com.google.common.base.Optional; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.client.informers.cache.Store; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -128,27 +129,27 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time public List getPeonJobs() { if (overlordNamespace.isEmpty()) { - return cachingClient.readJobCache(informer -> informer.getIndexer().list()); + return cachingClient.readJobCache(Store::list); } else { - return cachingClient.readJobCache(informer -> - informer.getIndexer() - .byIndex(DruidKubernetesCachingClient.OVERLORD_NAMESPACE_INDEX, overlordNamespace)); + return cachingClient.readJobCache( + indexer -> + indexer.byIndex(DruidKubernetesCachingClient.OVERLORD_NAMESPACE_INDEX, overlordNamespace)); } } @Override public Optional getPeonPod(String jobName) { - return cachingClient.readPodCache(informer -> { - List pods = informer.getIndexer().byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); + return cachingClient.readPodCache(indexer -> { + List pods = indexer.byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); return pods.isEmpty() ? Optional.absent() : Optional.of(pods.get(0)); }); } public Optional getPeonJob(String jobName) { - return cachingClient.readJobCache(informer -> { - List jobs = informer.getIndexer().byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); + return cachingClient.readJobCache(indexer -> { + List jobs = indexer.byIndex(DruidKubernetesCachingClient.JOB_NAME_INDEX, jobName); return jobs.isEmpty() ? Optional.absent() : Optional.of(jobs.get(0)); }); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index 25e7749e3029..2a062ef0d8a9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -140,7 +140,7 @@ public T readPodCache(SharedInformerCacheReadRequestExecutor executo if (podInformer == null) { throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); } - return executor.executeRequest(podInformer); + return executor.executeRequest(podInformer.getIndexer()); } public T readJobCache(SharedInformerCacheReadRequestExecutor executor) @@ -148,7 +148,7 @@ public T readJobCache(SharedInformerCacheReadRequestExecutor executo if (jobInformer == null) { throw DruidException.defensive("Job informer is not initialized, caching is disabled"); } - return executor.executeRequest(jobInformer); + return executor.executeRequest(jobInformer.getIndexer()); } /** diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java index 04291cd1046c..21a9c6c84e98 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java @@ -19,10 +19,10 @@ package org.apache.druid.k8s.overlord.common; -import io.fabric8.kubernetes.client.informers.SharedIndexInformer; +import io.fabric8.kubernetes.client.informers.cache.Indexer; @FunctionalInterface public interface SharedInformerCacheReadRequestExecutor { - T executeRequest(SharedIndexInformer informer); + T executeRequest(Indexer indexer); } From 4f735a467be0ba656525f0833f0796d6a737eb59 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 8 Dec 2025 12:29:40 -0600 Subject: [PATCH 58/58] Cleanup after another review round --- .../overlord/KubernetesTaskRunnerConfig.java | 23 ++++- .../common/CachingKubernetesPeonClient.java | 20 +++-- .../common/DruidKubernetesCachingClient.java | 83 ++++--------------- .../overlord/common/InformerEventHandler.java | 58 +++++++++++++ .../overlord/common/InformerEventType.java | 30 +++++++ .../overlord/common/KubernetesPeonClient.java | 23 +++-- .../KubernetesResourceEventNotifier.java | 8 +- ...or.java => SharedInformerCacheReader.java} | 4 +- .../CachingKubernetesPeonClientTest.java | 1 - .../common/TestCachingKubernetesClient.java | 6 -- 10 files changed, 150 insertions(+), 106 deletions(-) create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventHandler.java create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventType.java rename extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/{SharedInformerCacheReadRequestExecutor.java => SharedInformerCacheReader.java} (90%) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index dfe7b66be179..0e96e6ab2c3f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -72,8 +72,23 @@ public interface KubernetesTaskRunnerConfig Integer getCapacity(); + /** + * Whether to use caching for Kubernetes resources tied to indexing tasks. + *

    + * Enabling shared informers can significantly reduce the number of API calls made to the Kubernetes API server, + * improving performance and reducing load on the server. However, it also increases memory usage as informers + * maintain local caches of resources. + *

    + */ boolean isUseK8sSharedInformers(); + /** + * The resync period for the Kubernetes shared informers, if enabled. + *

    + * Periodic resyncs ensure that the informer's local cache is kept up to date with the remote Kubernetes API server + * state. This helps handle missed events or transient errors. + *

    + */ Period getK8sSharedInformerResyncPeriod(); static Builder builder() @@ -238,15 +253,15 @@ public Builder withLogSaveTimeout(Period logSaveTimeout) return this; } - public Builder withEnablePeonClientCache(boolean enableKubernetesClientCaching) + public Builder withUseK8sSharedInformers(boolean useK8sSharedInformers) { - this.useK8sSharedInformers = enableKubernetesClientCaching; + this.useK8sSharedInformers = useK8sSharedInformers; return this; } - public Builder withKubernetesClientInformerResyncPeriod(Period kubernetesClientInformerResyncPeriod) + public Builder withK8sSharedInformerResyncPeriod(Period k8sSharedInformerResyncPeriod) { - this.k8sSharedInformerResyncPeriod = kubernetesClientInformerResyncPeriod; + this.k8sSharedInformerResyncPeriod = k8sSharedInformerResyncPeriod; return this; } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java index 6dd5d70b6386..e67cddb5e467 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClient.java @@ -67,15 +67,16 @@ public CachingKubernetesPeonClient( @Override public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, TimeUnit unit) { - Duration timeout = Duration.millis(unit.toMillis(howLong)); - Duration jobMustBeSeenWithin = Duration.millis(cachingClient.getInformerResyncPeriodMillis() * 2); - Stopwatch stopwatch = Stopwatch.createStarted(); + final Duration timeout = Duration.millis(unit.toMillis(howLong)); + final Duration jobMustBeSeenWithin = Duration.millis(cachingClient.getInformerResyncPeriodMillis() * 2); + final Stopwatch stopwatch = Stopwatch.createStarted(); boolean jobSeenInCache = false; try { - CompletableFuture jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); + CompletableFuture jobFuture = null; while (stopwatch.hasNotElapsed(timeout) && (jobSeenInCache || stopwatch.hasNotElapsed(jobMustBeSeenWithin))) { - if (jobFuture.isDone()) { + if (jobFuture == null || jobFuture.isDone()) { + // Register a future to watch the next change to this job jobFuture = cachingClient.waitForJobChange(taskId.getK8sJobName()); } Optional maybeJob = getPeonJob(taskId.getK8sJobName()); @@ -158,13 +159,14 @@ public Optional getPeonJob(String jobName) @Nullable protected Pod waitUntilPeonPodCreatedAndReady(String jobName, long howLong, TimeUnit timeUnit) { - Duration timeout = Duration.millis(timeUnit.toMillis(howLong)); - Stopwatch stopwatch = Stopwatch.createStarted(); + final Duration timeout = Duration.millis(timeUnit.toMillis(howLong)); + final Stopwatch stopwatch = Stopwatch.createStarted(); try { - CompletableFuture podFuture = cachingClient.waitForPodChange(jobName); + CompletableFuture podFuture = null; while (stopwatch.hasNotElapsed(timeout)) { - if (podFuture.isDone()) { + if (podFuture == null || podFuture.isDone()) { + // Register a future to watch the next change to this pod podFuture = cachingClient.waitForPodChange(jobName); } Optional maybePod = getPeonPod(jobName); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java index 2a062ef0d8a9..b6128177377b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidKubernetesCachingClient.java @@ -22,9 +22,7 @@ import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; -import io.fabric8.kubernetes.client.informers.ResourceEventHandler; import io.fabric8.kubernetes.client.informers.SharedIndexInformer; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.emitter.EmittingLogger; import java.util.Collections; @@ -32,64 +30,18 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.function.BiConsumer; import java.util.function.Function; public class DruidKubernetesCachingClient { - /** - * Event types for Kubernetes informer resource events. - */ - public enum InformerEventType - { - ADD, - UPDATE, - DELETE - } - - /** - * Impl of ResourceEventHandler that simplifies event handling - * by accepting a single lambda BiConsumer for all event types (add, update, delete). - * - * @param The Kubernetes resource type (e.g., Pod, Job) - */ - public static class InformerEventHandler implements ResourceEventHandler - { - private final BiConsumer eventConsumer; - - public InformerEventHandler(BiConsumer eventConsumer) - { - this.eventConsumer = eventConsumer; - } - - @Override - public void onAdd(T resource) - { - eventConsumer.accept(resource, InformerEventType.ADD); - } - - @Override - public void onUpdate(T oldResource, T newResource) - { - eventConsumer.accept(newResource, InformerEventType.UPDATE); - } - - @Override - public void onDelete(T resource, boolean deletedFinalStateUnknown) - { - eventConsumer.accept(resource, InformerEventType.DELETE); - } - } private static final EmittingLogger log = new EmittingLogger(DruidKubernetesCachingClient.class); public static final String JOB_NAME_INDEX = "byJobName"; public static final String OVERLORD_NAMESPACE_INDEX = "byOverlordNamespace"; - private static final long DEFAULT_INFORMER_RESYNC_PERIOD_MS = 300000L; // 5 minutes - - protected final SharedIndexInformer podInformer; - protected final SharedIndexInformer jobInformer; - protected final KubernetesResourceEventNotifier eventNotifier; + private final SharedIndexInformer podInformer; + private final SharedIndexInformer jobInformer; + private final KubernetesResourceEventNotifier eventNotifier; private final KubernetesClientApi baseClient; private final long informerResyncPeriodMillis; @@ -107,6 +59,9 @@ public DruidKubernetesCachingClient( this.jobInformer = setupJobInformer(namespace); } + /** + * Stops the fabric8 informers and cancels all pending futures in the event notifier. + */ public void stop() { if (podInformer != null) { @@ -124,31 +79,25 @@ public KubernetesClientApi getBaseClient() return baseClient; } - // Delegate write operations to base client - public T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException - { - return baseClient.executeRequest(executor); - } - public KubernetesClient getClient() { return baseClient.getClient(); } - public T readPodCache(SharedInformerCacheReadRequestExecutor executor) + /** + * Reads from thePod Informer's {@link io.fabric8.kubernetes.client.informers.cache.Indexer} using the provided executor. + */ + public T readPodCache(SharedInformerCacheReader executor) { - if (podInformer == null) { - throw DruidException.defensive("Pod informer is not initialized, caching is disabled"); - } - return executor.executeRequest(podInformer.getIndexer()); + return executor.readFromCache(podInformer.getIndexer()); } - public T readJobCache(SharedInformerCacheReadRequestExecutor executor) + /** + * Reads from the Job Informer's {@link io.fabric8.kubernetes.client.informers.cache.Indexer} using the provided executor. + */ + public T readJobCache(SharedInformerCacheReader executor) { - if (jobInformer == null) { - throw DruidException.defensive("Job informer is not initialized, caching is disabled"); - } - return executor.executeRequest(jobInformer.getIndexer()); + return executor.readFromCache(jobInformer.getIndexer()); } /** diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventHandler.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventHandler.java new file mode 100644 index 000000000000..0e553e27bd96 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventHandler.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +import io.fabric8.kubernetes.client.informers.ResourceEventHandler; + +import java.util.function.BiConsumer; + +/** + * Implementation of ResourceEventHandler that simplifies event handling + * by accepting a single lambda BiConsumer for all event types (add, update, delete). + * + * @param The Kubernetes resource type (e.g., Pod, Job) + */ +public class InformerEventHandler implements ResourceEventHandler +{ + private final BiConsumer eventConsumer; + + public InformerEventHandler(BiConsumer eventConsumer) + { + this.eventConsumer = eventConsumer; + } + + @Override + public void onAdd(T resource) + { + eventConsumer.accept(resource, InformerEventType.ADD); + } + + @Override + public void onUpdate(T oldResource, T newResource) + { + eventConsumer.accept(newResource, InformerEventType.UPDATE); + } + + @Override + public void onDelete(T resource, boolean deletedFinalStateUnknown) + { + eventConsumer.accept(resource, InformerEventType.DELETE); + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventType.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventType.java new file mode 100644 index 000000000000..001b534670d5 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/InformerEventType.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.common; + +/** + * Event types for Kubernetes informer resource events. + */ +public enum InformerEventType +{ + ADD, + UPDATE, + DELETE +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index b57d4378386f..6f30e384b244 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -93,9 +93,9 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn Pod result = waitUntilPeonPodCreatedAndReady(jobName, howLong, timeUnit); if (result == null) { - throw new ISE("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled", task.getId()); + throw new ISE("K8s pod for the task[%s] appeared and disappeared. It can happen if the task was canceled", task.getId()); } - log.info("Pod for job[%s] is in state [%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); + log.info("Pod for job[%s] is in state[%s] for task[%s].", jobName, result.getStatus().getPhase(), task.getId()); long duration = System.currentTimeMillis() - start; emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); return result; @@ -140,13 +140,13 @@ public JobResponse waitForPeonJobCompletion(K8sTaskId taskId, long howLong, Time unit ); if (job == null) { - log.info("K8s job for the task [%s] was not found. It can happen if the task was canceled", taskId); + log.info("K8s job for the task[%s] was not found. It can happen if the task was canceled", taskId); return new JobResponse(null, PeonPhase.FAILED); } if (job.getStatus().getSucceeded() != null) { return new JobResponse(job, PeonPhase.SUCCEEDED); } - log.warn("Task %s failed with status %s", taskId, job.getStatus()); + log.warn("Task[%s] failed with status[%s]", taskId, job.getStatus()); return new JobResponse(job, PeonPhase.FAILED); }); } @@ -161,13 +161,13 @@ public boolean deletePeonJob(K8sTaskId taskId) .withName(taskId.getK8sJobName()) .delete().isEmpty()); if (result) { - log.info("Cleaned up k8s job: %s", taskId); + log.info("Cleaned up k8s job[%s]", taskId); } else { - log.info("K8s job does not exist: %s", taskId); + log.info("K8s job[%s] does not exist", taskId); } return result; } else { - log.info("Not cleaning up job %s due to flag: debugJobs=true", taskId); + log.info("Not cleaning up job[%s] due to flag: debugJobs=true", taskId); return true; } } @@ -204,18 +204,15 @@ public Optional getPeonLogWatcher(K8sTaskId taskId) return Optional.of(logWatch); } catch (Exception e) { - log.error(e, "Error watching logs from task: %s, pod: %s", taskId, podName); + log.error(e, "Error watching logs from task[%s], pod[%s].", taskId, podName); return Optional.absent(); } } /** * Get an InputStream for the logs of the peon pod associated with the given taskId. - *

    - * Any issues creating the InputStream will be logged and an absent Optional will be returned. - *

    * - * @return an Optional containing the {@link InputStream} if the pod exists and logs could be streamed, or absent otherwise + * @return an Optional containing the {@link InputStream} for the logs of the pod, if it exists and logs could be streamed, or absent otherwise. */ public Optional getPeonLogs(K8sTaskId taskId) { @@ -290,7 +287,7 @@ public int deleteCompletedPeonJobsOlderThan(long howFarBack, TimeUnit timeUnit) .delete().isEmpty()) { numDeleted.incrementAndGet(); } else { - log.error("Failed to delete job %s", x.getMetadata().getName()); + log.error("Failed to delete job[%s]", x.getMetadata().getName()); } }); return numDeleted.get(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java index 6252a6f4bd3e..4d5f366d7826 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesResourceEventNotifier.java @@ -58,11 +58,11 @@ public CompletableFuture waitForJobChange(String jobName) CompletableFuture previous = jobWatchers.put(jobName, future); if (previous != null && !previous.isDone()) { - log.warn("Replacing active watcher for job [%s] - multiple waiters detected", jobName); + log.warn("Replacing active watcher for job[%s] - multiple waiters detected", jobName); previous.cancel(true); } - log.debug("Registered watcher for job [%s]", jobName); + log.debug("Registered watcher for job[%s]", jobName); return future; } @@ -97,7 +97,7 @@ public void cancelJobWatcher(String jobName) { CompletableFuture future = jobWatchers.remove(jobName); if (future != null && !future.isDone()) { - log.debug("Cancelling watcher for job [%s]", jobName); + log.debug("Cancelling watcher for job[%s]", jobName); future.cancel(true); } } @@ -143,7 +143,7 @@ public void notifyPodChange(String jobName, Pod pod) { CompletableFuture future = podWatchers.remove(jobName); if (future != null) { - log.debug("Notifying watcher of pod change for job-name [%s]", jobName); + log.debug("Notifying watcher of pod change for job-name[%s]", jobName); future.complete(pod); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReader.java similarity index 90% rename from extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java rename to extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReader.java index 21a9c6c84e98..50f5ee30a8e8 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReadRequestExecutor.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/SharedInformerCacheReader.java @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.client.informers.cache.Indexer; @FunctionalInterface -public interface SharedInformerCacheReadRequestExecutor +public interface SharedInformerCacheReader { - T executeRequest(Indexer indexer); + T readFromCache(Indexer indexer); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java index c13ac61c91c2..0f1bfb5dd2e4 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/CachingKubernetesPeonClientTest.java @@ -65,7 +65,6 @@ public void setup() throws Exception TestKubernetesClient clientApi = new TestKubernetesClient(client, NAMESPACE); cachingClient = new TestCachingKubernetesClient(clientApi, NAMESPACE); - cachingClient.start(); peonClient = new CachingKubernetesPeonClient(cachingClient, NAMESPACE, "", false, serviceEmitter); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java index db60be29f1bb..46f3b05df73a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/TestCachingKubernetesClient.java @@ -28,12 +28,6 @@ public TestCachingKubernetesClient(KubernetesClientApi clientApi, String namespa super(clientApi, namespace, TESTING_RESYNC_PERIOD_MS); } - public void start() - { - podInformer.run(); - jobInformer.run(); - } - public void waitForSync() throws InterruptedException { // Give informers a bit more time to process