From 7a8672639a7db9f449e13cd0f7877ec6c0cfba05 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 17 Aug 2023 11:17:37 -0400 Subject: [PATCH 01/27] Separate out task logs --- distribution/docker/peon.sh | 2 +- .../k8s/overlord/KubernetesPeonLifecycle.java | 24 +++++++- .../k8s/overlord/KubernetesTaskRunner.java | 3 +- .../overlord/KubernetesTaskRunnerConfig.java | 27 ++++++++- .../overlord/KubernetesTaskRunnerFactory.java | 2 + .../overlord/taskadapter/K8sTaskAdapter.java | 23 +++++-- .../taskadapter/PodTemplateTaskAdapter.java | 32 ++++++---- .../overlord/KubernetesPeonLifecycleTest.java | 8 +-- .../KubernetesTaskRunnerConfigTest.java | 4 ++ .../KubernetesTaskRunnerFactoryTest.java | 2 + .../overlord/KubernetesTaskRunnerTest.java | 6 +- .../taskadapter/K8sTaskAdapterTest.java | 60 +++++++++++++++++++ .../PodTemplateTaskAdapterTest.java | 35 +++++++++++ .../resources/expectedNoopJobNoTaskJson.yaml | 45 ++++++++++++++ .../apache/druid/storage/s3/S3TaskLogs.java | 16 ++++- .../worker/executor/ExecutorLifecycle.java | 46 +++++++++++++- .../executor/ExecutorLifecycleConfig.java | 15 +++++ .../org/apache/druid/tasklogs/TaskLogs.java | 1 + .../druid/tasklogs/TaskPayloadManager.java | 13 ++++ .../java/org/apache/druid/cli/CliPeon.java | 16 ++++- 20 files changed, 343 insertions(+), 37 deletions(-) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml create mode 100644 processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java diff --git a/distribution/docker/peon.sh b/distribution/docker/peon.sh index 66e34c997445..50ba9727c9d6 100755 --- a/distribution/docker/peon.sh +++ b/distribution/docker/peon.sh @@ -150,6 +150,6 @@ then fi # take the ${TASK_JSON} environment variable and base64 decode, unzip and throw it in ${TASK_DIR}/task.json -mkdir -p ${TASK_DIR}; echo ${TASK_JSON} | base64 -d | gzip -d > ${TASK_DIR}/task.json; +mkdir -p ${TASK_DIR}; [ -n "$TASK_JSON" ] && echo ${TASK_JSON} | base64 -d | gzip -d > ${TASK_DIR}/task.json; exec bin/run-java ${JAVA_OPTS} -cp $COMMON_CONF_DIR:$SERVICE_CONF_DIR:lib/*: org.apache.druid.cli.Main internal peon $@ diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index 447a8632bb90..f4bd11069615 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -42,6 +42,7 @@ import java.io.IOException; import java.io.InputStream; +import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; @@ -71,9 +72,9 @@ public interface TaskStateListener protected enum State { - /** Lifecycle's state before {@link #run(Job, long, long)} or {@link #join(long)} is called. */ + /** Lifecycle's state before {@link #run(Job, long, long, Task)} or {@link #join(long)} is called. */ NOT_STARTED, - /** Lifecycle's state since {@link #run(Job, long, long)} is called. */ + /** Lifecycle's state since {@link #run(Job, long, long, Task)} is called. */ PENDING, /** Lifecycle's state since {@link #join(long)} is called. */ RUNNING, @@ -117,13 +118,14 @@ protected KubernetesPeonLifecycle( * @return * @throws IllegalStateException */ - protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) throws IllegalStateException + protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, Task task) throws IllegalStateException { try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); // In case something bad happens and run is called twice on this KubernetesPeonLifecycle, reset taskLocation. taskLocation = null; + writeTaskPayload(task); kubernetesClient.launchPeonJobAndWaitForStart( job, launchTimeout, @@ -132,6 +134,10 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) return join(timeout); } + catch (IOException e) { + log.info("Failed to run task: %s", taskId.getOriginalTaskId()); + throw new RuntimeException(e); + } catch (Exception e) { log.info("Failed to run task: %s", taskId.getOriginalTaskId()); shutdown(); @@ -142,6 +148,18 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) } } + private void writeTaskPayload(Task task) throws IOException{ + Path file = Files.createTempFile(taskId.getOriginalTaskId(), "task.json"); + try { + FileUtils.writeStringToFile(file.toFile(), mapper.writeValueAsString(task), Charset.defaultCharset()); + taskLogs.pushTaskPayload(task.getId(), file.toFile()); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + Files.deleteIfExists(file); + } + } + /** * Join existing Kubernetes Job * diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 99095264c372..c16df80c46aa 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -196,7 +196,8 @@ protected TaskStatus doTask(Task task, boolean run) taskStatus = peonLifecycle.run( adapter.fromTask(task), config.getTaskLaunchTimeout().toStandardDuration().getMillis(), - config.getTaskTimeout().toStandardDuration().getMillis() + config.getTaskTimeout().toStandardDuration().getMillis(), + task ); } else { taskStatus = peonLifecycle.join( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index 0d67c55b30aa..256c6c3e9053 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -90,6 +90,11 @@ public class KubernetesTaskRunnerConfig // how long to wait for the peon k8s job to launch private Period k8sjobLaunchTimeout = new Period("PT1H"); + // Whether to pass the task.json payload to the peon K8s Jobs as a environment variable. + @JsonProperty + @NotNull + private Boolean taskPayloadAsEnvVariable = Boolean.TRUE; + @JsonProperty // ForkingTaskRunner inherits the monitors from the MM, in k8s mode // the peon inherits the monitors from the overlord, so if someone specifies @@ -135,7 +140,8 @@ private KubernetesTaskRunnerConfig( List javaOptsArray, Map labels, Map annotations, - Integer capacity + Integer capacity, + Boolean taskPayloadAsEnvVariable ) { this.namespace = namespace; @@ -196,6 +202,10 @@ private KubernetesTaskRunnerConfig( capacity, this.capacity ); + this.taskPayloadAsEnvVariable = ObjectUtils.defaultIfNull( + taskPayloadAsEnvVariable, + Boolean.TRUE + ); } public String getNamespace() @@ -279,6 +289,11 @@ public Integer getCapacity() return capacity; } + public Boolean isTaskPayloadAsEnvVariable() + { + return taskPayloadAsEnvVariable; + } + public static Builder builder() { return new Builder(); @@ -302,6 +317,7 @@ public static class Builder private Map labels; private Map annotations; private Integer capacity; + private Boolean taskPayloadAsEnvVariable; public Builder() { @@ -403,6 +419,12 @@ public Builder withCapacity(@Min(0) @Max(Integer.MAX_VALUE) Integer capacity) return this; } + public Builder withTaskPayloadAsEnvVariable(Boolean taskPayloadAsEnvVariable) + { + this.taskPayloadAsEnvVariable = taskPayloadAsEnvVariable; + return this; + } + public KubernetesTaskRunnerConfig build() { return new KubernetesTaskRunnerConfig( @@ -421,7 +443,8 @@ public KubernetesTaskRunnerConfig build() this.javaOptsArray, this.labels, this.annotations, - this.capacity + this.capacity, + this.taskPayloadAsEnvVariable ); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 76698ba8fe31..206dee6c7e3f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -23,6 +23,8 @@ import com.google.inject.Inject; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.JsonNonNull; import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.config.TaskConfig; diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index 712bc1a47e20..174dbeb67582 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -219,15 +219,11 @@ void addEnvironmentVariables(Container mainContainer, PeonCommandContext context .build()); } - mainContainer.getEnv().addAll(Lists.newArrayList( + List envVars = Lists.newArrayList( new EnvVarBuilder() .withName(DruidK8sConstants.TASK_DIR_ENV) .withValue(context.getTaskDir().getAbsolutePath()) .build(), - new EnvVarBuilder() - .withName(DruidK8sConstants.TASK_JSON_ENV) - .withValue(taskContents) - .build(), new EnvVarBuilder() .withName(DruidK8sConstants.JAVA_OPTS) .withValue(Joiner.on(" ").join(context.getJavaOpts())) @@ -244,7 +240,16 @@ void addEnvironmentVariables(Container mainContainer, PeonCommandContext context null, "metadata.name" )).build()).build() - )); + ); + if (taskRunnerConfig.isTaskPayloadAsEnvVariable()) { + envVars.add( + new EnvVarBuilder() + .withName(DruidK8sConstants.TASK_JSON_ENV) + .withValue(taskRunnerConfig.isTaskPayloadAsEnvVariable() ? taskContents : "") + .build() + ); + } + mainContainer.getEnv().addAll(envVars); } protected Container setupMainContainer( @@ -403,6 +408,12 @@ private List generateCommand(Task task) command.add("--loadBroadcastSegments"); command.add("true"); } + + // If we are not passing the TASK_JSON, pass the taskId flag to pull the task payload. + if (!taskRunnerConfig.isTaskPayloadAsEnvVariable()) { + command.add("--taskId"); + command.add(task.getId()); + } log.info( "Peon Command for K8s job: %s", ForkingTaskRunner.getMaskedCommand(startupLoggingConfig.getMaskProperties(), command) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index a3d10f7dcd1c..76a7b3c52268 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -24,8 +24,8 @@ import com.fasterxml.jackson.databind.introspect.AnnotatedClass; import com.fasterxml.jackson.databind.introspect.AnnotatedClassResolver; import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import io.fabric8.kubernetes.api.model.EnvVar; import io.fabric8.kubernetes.api.model.EnvVarBuilder; import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder; @@ -55,6 +55,7 @@ import java.nio.file.Files; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; @@ -210,7 +211,7 @@ private Optional loadPodTemplate(String key, Properties properties) private Collection getEnv(Task task) { - return ImmutableList.of( + List envVars = Lists.newArrayList( new EnvVarBuilder() .withName(DruidK8sConstants.TASK_DIR_ENV) .withValue(taskConfig.getBaseDir()) @@ -219,17 +220,21 @@ private Collection getEnv(Task task) .withName(DruidK8sConstants.TASK_ID_ENV) .withValue(task.getId()) .build(), - new EnvVarBuilder() - .withName(DruidK8sConstants.TASK_JSON_ENV) - .withValueFrom(new EnvVarSourceBuilder().withFieldRef(new ObjectFieldSelector( - null, - StringUtils.format("metadata.annotations['%s']", DruidK8sConstants.TASK) - )).build()).build(), new EnvVarBuilder() .withName(DruidK8sConstants.LOAD_BROADCAST_SEGMENTS_ENV) .withValue(Boolean.toString(task.supportsQueries())) .build() ); + if (taskRunnerConfig.isTaskPayloadAsEnvVariable()) { + envVars.add(new EnvVarBuilder() + .withName(DruidK8sConstants.TASK_JSON_ENV) + .withValueFrom(new EnvVarSourceBuilder().withFieldRef(new ObjectFieldSelector( + null, + StringUtils.format("metadata.annotations['%s']", DruidK8sConstants.TASK) + )).build()).build() + ); + } + return envVars; } private Map getPodLabels(KubernetesTaskRunnerConfig config, Task task) @@ -239,14 +244,17 @@ private Map getPodLabels(KubernetesTaskRunnerConfig config, Task private Map getPodTemplateAnnotations(Task task) throws IOException { - return ImmutableMap.builder() - .put(DruidK8sConstants.TASK, Base64Compression.compressBase64(mapper.writeValueAsString(task))) + ImmutableMap.Builder podTemplateAnnotationBuilder = ImmutableMap.builder() .put(DruidK8sConstants.TLS_ENABLED, String.valueOf(node.isEnableTlsPort())) .put(DruidK8sConstants.TASK_ID, task.getId()) .put(DruidK8sConstants.TASK_TYPE, task.getType()) .put(DruidK8sConstants.TASK_GROUP_ID, task.getGroupId()) - .put(DruidK8sConstants.TASK_DATASOURCE, task.getDataSource()) - .build(); + .put(DruidK8sConstants.TASK_DATASOURCE, task.getDataSource()); + if (taskRunnerConfig.isTaskPayloadAsEnvVariable()) { + podTemplateAnnotationBuilder + .put(DruidK8sConstants.TASK, Base64Compression.compressBase64(mapper.writeValueAsString(task))); + } + return podTemplateAnnotationBuilder.build(); } private Map getJobLabels(KubernetesTaskRunnerConfig config, Task task) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 980a425a8554..7ab9a1c2d6dd 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -113,7 +113,7 @@ protected synchronized TaskStatus join(long timeout) replayAll(); - TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L); + TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L, task); verifyAll(); @@ -157,12 +157,12 @@ protected synchronized TaskStatus join(long timeout) replayAll(); - peonLifecycle.run(job, 0L, 0L); + peonLifecycle.run(job, 0L, 0L, task); Assert.assertThrows( "Task [id] failed to run: invalid peon lifecycle state transition [STOPPED]->[PENDING]", IllegalStateException.class, - () -> peonLifecycle.run(job, 0L, 0L) + () -> peonLifecycle.run(job, 0L, 0L, task) ); verifyAll(); @@ -208,7 +208,7 @@ protected synchronized TaskStatus join(long timeout) Assert.assertThrows( Exception.class, - () -> peonLifecycle.run(job, 0L, 0L) + () -> peonLifecycle.run(job, 0L, 0L, task) ); verifyAll(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java index 1f4a7281f649..a1e0bd14f8ed 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java @@ -55,6 +55,7 @@ public void test_deserializable() throws IOException Assert.assertEquals(ImmutableMap.of(), config.getLabels()); Assert.assertEquals(ImmutableMap.of(), config.getAnnotations()); Assert.assertEquals(Integer.valueOf(Integer.MAX_VALUE), config.getCapacity()); + Assert.assertEquals(Boolean.TRUE, config.isTaskPayloadAsEnvVariable()); } @Test @@ -80,6 +81,7 @@ public void test_builder_preservesDefaults() Assert.assertEquals(ImmutableMap.of(), config.getLabels()); Assert.assertEquals(ImmutableMap.of(), config.getAnnotations()); Assert.assertEquals(Integer.valueOf(Integer.MAX_VALUE), config.getCapacity()); + Assert.assertEquals(Boolean.TRUE, config.isTaskPayloadAsEnvVariable()); } @Test @@ -102,6 +104,7 @@ public void test_builder() .withLabels(ImmutableMap.of("key", "value")) .withAnnotations(ImmutableMap.of("key", "value")) .withCapacity(1) + .withTaskPayloadAsEnvVariable(Boolean.FALSE) .build(); Assert.assertEquals("namespace", config.getNamespace()); @@ -119,5 +122,6 @@ public void test_builder() Assert.assertEquals(ImmutableMap.of("key", "value"), config.getLabels()); Assert.assertEquals(ImmutableMap.of("key", "value"), config.getAnnotations()); Assert.assertEquals(Integer.valueOf(1), config.getCapacity()); + Assert.assertEquals(Boolean.FALSE, config.isTaskPayloadAsEnvVariable()); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index ba9d2accf170..56b9a5b5e5f7 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -44,6 +44,7 @@ public class KubernetesTaskRunnerFactoryTest { private ObjectMapper objectMapper; + private ObjectMapper jsonMapper; private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; private StartupLoggingConfig startupLoggingConfig; private TaskLogs taskLogs; @@ -58,6 +59,7 @@ public class KubernetesTaskRunnerFactoryTest public void setup() { objectMapper = new TestUtils().getTestObjectMapper(); + jsonMapper = new TestUtils().getTestObjectMapper(); kubernetesTaskRunnerConfig = KubernetesTaskRunnerConfig.builder() .withCapacity(1) .build(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 0359488802eb..68d1e9ceb520 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -142,7 +142,8 @@ public void test_run_withoutExistingTask() throws IOException, ExecutionExceptio EasyMock.expect(kubernetesPeonLifecycle.run( EasyMock.eq(job), EasyMock.anyLong(), - EasyMock.anyLong() + EasyMock.anyLong(), + EasyMock.anyObject() )).andReturn(taskStatus); replayAll(); @@ -179,7 +180,8 @@ public void test_run_whenExceptionThrown_throwsRuntimeException() throws IOExcep EasyMock.expect(kubernetesPeonLifecycle.run( EasyMock.eq(job), EasyMock.anyLong(), - EasyMock.anyLong() + EasyMock.anyLong(), + EasyMock.anyObject() )).andThrow(new IllegalStateException()); replayAll(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 519e7177cbe5..37adcbd1cbf1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -65,9 +65,11 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -189,6 +191,64 @@ public void serializingAndDeserializingATask() throws IOException assertEquals(task, taskFromJob); } + @Test + public void toTask_dontSetTaskJSON() throws IOException + { + final PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); + TestKubernetesClient testClient = new TestKubernetesClient(client) + { + @SuppressWarnings("unchecked") + @Override + public T executeRequest(KubernetesExecutor executor) throws KubernetesResourceNotFoundException + { + return (T) new Pod() + { + @Override + public PodSpec getSpec() + { + return podSpec; + } + }; + } + }; + + KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() + .withNamespace("test") + .withTaskPayloadAsEnvVariable(false) + .build(); + K8sTaskAdapter adapter = new SingleContainerTaskAdapter( + testClient, + config, + taskConfig, + startupLoggingConfig, + node, + jsonMapper + ); + Task task = K8sTestUtils.getTask(); + Job job = adapter.fromTask(task); + // TASK_JSON should not be set in env variables + Assertions.assertFalse( + job.getSpec() + .getTemplate() + .getSpec() + .getContainers() + .get(0).getEnv() + .stream().anyMatch(env -> env.getName().equals(DruidK8sConstants.TASK_JSON_ENV)) + ); + + // --taskId should be passed to the peon command args + Assertions.assertTrue( + Arrays.stream(job.getSpec() + .getTemplate() + .getSpec() + .getContainers() + .get(0) + .getArgs() + .get(0).split(" ")).collect(Collectors.toSet()) + .containsAll(ImmutableList.of("--taskId", task.getId())) + ); + } + @Test void testGrabbingTheLastXmxValueFromACommand() { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 992a071e22a8..e06713a415d4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -243,6 +243,41 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites() throws IOExce assertJobSpecsEqual(actual, expected); } + @Test + public void test_fromTask_withNoopPodTemplateInRuntimeProperites_dontSetTaskJSON() throws IOException + { + Path templatePath = Files.createFile(tempDir.resolve("noop.yaml")); + mapper.writeValue(templatePath.toFile(), podTemplateSpec); + + Properties props = new Properties(); + props.setProperty("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString()); + props.setProperty("druid.indexer.runner.k8s.podTemplate.noop", templatePath.toString()); + + PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( + KubernetesTaskRunnerConfig.builder().withTaskPayloadAsEnvVariable(false).build(), + taskConfig, + node, + mapper, + props + ); + + Task task = new NoopTask( + "id", + "id", + "datasource", + 0, + 0, + null, + null, + null + ); + + Job actual = adapter.fromTask(task); + Job expected = K8sTestUtils.fileToResource("expectedNoopJobNoTaskJson.yaml", Job.class); + + Assertions.assertEquals(actual, expected); + } + @Test public void test_fromTask_withoutAnnotations_throwsIOE() throws IOException { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml new file mode 100644 index 000000000000..5dece8cad443 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml @@ -0,0 +1,45 @@ +apiVersion: batch/v1 +kind: Job +metadata: + name: "id-3e70afe5cd823dfc7dd308eea616426b" + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + annotations: + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" +spec: + activeDeadlineSeconds: 14400 + backoffLimit: 0 + ttlSecondsAfterFinished: 172800 + template: + metadata: + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + annotations: + tls.enabled: "false" + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" + spec: + containers: + - command: + - sleep + - "3600" + env: + - name: "TASK_DIR" + value: "/tmp" + - name: "TASK_ID" + value: "id" + image: one + name: primary diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java index 53112d0808db..1f5ec3c109ea 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.tasklogs.TaskLogs; +import org.apache.druid.tasklogs.TaskPayloadManager; import java.io.File; import java.io.IOException; @@ -39,7 +40,7 @@ /** * Provides task logs archived on S3. */ -public class S3TaskLogs implements TaskLogs +public class S3TaskLogs implements TaskLogs, TaskPayloadManager { private static final Logger log = new Logger(S3TaskLogs.class); @@ -83,6 +84,19 @@ public Optional streamTaskStatus(String taskid) throws IOException return streamTaskFileWithRetry(0, taskKey); } + @Override + public void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { + final String taskKey = getTaskLogKey(taskid, "task.json"); + log.info("Pushing task payload %s to: %s", taskPayloadFile, taskKey); + pushTaskFile(taskPayloadFile, taskKey); + } + + @Override + public Optional streamTaskPayload(String taskid) throws IOException { + final String taskKey = getTaskLogKey(taskid, "task.json"); + return streamTaskFileWithRetry(0, taskKey); + } + /** * Using the retry conditions defined in {@link S3Utils#S3RETRY}. */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java index ad46ee7e666d..768373518fbb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -25,20 +25,35 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; +import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionHolder; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.metadata.PasswordProviderRedactionMixIn; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceClientFactoryImpl; +import org.apache.druid.rpc.ServiceLocator; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.jboss.netty.handler.codec.http.HttpMethod; import java.io.File; import java.io.IOException; @@ -46,6 +61,8 @@ import java.nio.channels.FileChannel; import java.nio.channels.FileLock; import java.nio.file.StandardOpenOption; +import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; /** @@ -64,6 +81,8 @@ public class ExecutorLifecycle private final ExecutorService parentMonitorExec = Execs.singleThreaded("parent-monitor-%d"); + private final ServiceClient serviceClient; + private volatile Task task = null; private volatile ListenableFuture statusFuture = null; private volatile FileChannel taskLockChannel; @@ -75,7 +94,9 @@ public ExecutorLifecycle( TaskConfig taskConfig, TaskActionClientFactory taskActionClientFactory, TaskRunner taskRunner, - ObjectMapper jsonMapper + ObjectMapper jsonMapper, + @EscalatedGlobal final ServiceClientFactory clientFactory, + @IndexingService final ServiceLocator serviceLocator ) { this.taskExecutorConfig = taskExecutorConfig; @@ -83,6 +104,10 @@ public ExecutorLifecycle( this.taskActionClientFactory = taskActionClientFactory; this.taskRunner = taskRunner; this.jsonMapper = jsonMapper.copy().addMixIn(PasswordProvider.class, PasswordProviderRedactionMixIn.class); + this.serviceClient = clientFactory.makeClient("peon", serviceLocator, StandardRetryPolicy.builder() + .maxAttempts(5) + .retryNotAvailable(false) + .build()); } @LifecycleStart @@ -93,14 +118,29 @@ public void start() throws InterruptedException final InputStream parentStream = Preconditions.checkNotNull(taskExecutorConfig.getParentStream(), "parentStream"); try { - task = jsonMapper.readValue(taskFile, Task.class); + if (!taskExecutorConfig.getTaskId().equals("")) { + final String path = StringUtils.format("/druid/indexer/v1/task/%s", StringUtils.urlEncode(taskExecutorConfig.getTaskId())); + final Map response = jsonMapper.readValue( + serviceClient.request( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ).getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + task = jsonMapper.convertValue( + response.get("payload"), + Task.class + ); + } else { + task = jsonMapper.readValue(taskFile, Task.class); + } log.info( "Running with task: %s", jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task) ); } - catch (IOException e) { + catch (ExecutionException | IOException e) { throw new RuntimeException(e); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java index 0c48fe299a82..8862c3b387ea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java @@ -35,6 +35,10 @@ public class ExecutorLifecycleConfig @NotNull private File taskFile = null; + @JsonProperty + @NotNull + private String taskId = ""; + @JsonProperty @NotNull private File statusFile = null; @@ -69,6 +73,17 @@ public File getStatusFile() return statusFile; } + public ExecutorLifecycleConfig setTaskId(String taskId) + { + this.taskId = taskId; + return this; + } + + public String getTaskId() + { + return taskId; + } + public ExecutorLifecycleConfig setStatusFile(File statusFile) { this.statusFile = statusFile; diff --git a/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java b/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java index ee50217c9571..2029c74aa501 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java @@ -21,6 +21,7 @@ import org.apache.druid.guice.annotations.ExtensionPoint; + @ExtensionPoint public interface TaskLogs extends TaskLogStreamer, TaskLogPusher, TaskLogKiller { diff --git a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java new file mode 100644 index 000000000000..4e1ac066f993 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java @@ -0,0 +1,13 @@ +package org.apache.druid.tasklogs; + +import com.google.common.base.Optional; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + +public interface TaskPayloadManager { + void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException; + + Optional streamTaskPayload(String taskid) throws IOException; +} diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 8b8e1a426e8f..d8a02ab86674 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -40,6 +40,7 @@ import com.google.inject.name.Named; import com.google.inject.name.Names; import io.netty.util.SuppressForbidden; +import org.apache.commons.io.IOUtils; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; @@ -66,6 +67,8 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.tasklogs.TaskLogs; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; @@ -133,6 +136,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.Charset; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -176,6 +180,9 @@ public class CliPeon extends GuiceRunnable @Option(name = "--loadBroadcastSegments", title = "loadBroadcastSegments", description = "Enable loading of broadcast segments") public String loadBroadcastSegments = "false"; + @Option(name = "--taskId", title = "taskId", description = "TaskId for fetching task.json remotely") + public String taskId = ""; + private static final Logger log = new Logger(CliPeon.class); private Properties properties; @@ -229,7 +236,8 @@ public void configure(Binder binder) LifecycleModule.register(binder, ExecutorLifecycle.class); ExecutorLifecycleConfig executorLifecycleConfig = new ExecutorLifecycleConfig() .setTaskFile(Paths.get(taskDirPath, "task.json").toFile()) - .setStatusFile(Paths.get(taskDirPath, "attempt", attemptId, "status.json").toFile()); + .setStatusFile(Paths.get(taskDirPath, "attempt", attemptId, "status.json").toFile()) + .setTaskId(taskId); if ("k8s".equals(properties.getProperty("druid.indexer.runner.type", null))) { log.info("Running peon in k8s mode"); @@ -282,9 +290,13 @@ public Supplier> heartbeatDimensions(Task task) @Provides @LazySingleton - public Task readTask(@Json ObjectMapper mapper, ExecutorLifecycleConfig config) + public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskLogs taskLogs) { try { + if (!config.getTaskId().equals("")) { + String task = IOUtils.toString(taskLogs.streamTaskPayload(config.getTaskId()).get(), Charset.defaultCharset()); + return smileMapper.readValue(task, Task.class); + } return mapper.readValue(config.getTaskFile(), Task.class); } catch (IOException e) { From 151b5c0ef9ee5f0e2a9980d13700f2c382ef382a Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 17 Aug 2023 16:38:17 -0400 Subject: [PATCH 02/27] working with cleaner configs --- .../k8s/overlord/KubernetesPeonLifecycle.java | 13 ++- .../KubernetesPeonLifecycleFactory.java | 9 +- .../overlord/KubernetesTaskRunnerFactory.java | 9 +- .../overlord/KubernetesPeonLifecycleTest.java | 83 +++++++++++++------ .../KubernetesTaskRunnerFactoryTest.java | 30 ++++--- .../k8s/overlord/KubernetesWorkItemTest.java | 8 ++ .../storage/s3/S3StorageDruidModule.java | 3 + .../apache/druid/storage/s3/S3TaskLogs.java | 2 +- .../guice/IndexingServiceTaskLogsModule.java | 3 + .../worker/executor/ExecutorLifecycle.java | 31 +------ .../executor/ExecutorLifecycleConfig.java | 10 +-- .../apache/druid/tasklogs/NoopTaskLogs.java | 11 +++ .../tasklogs/NoopTaskPayloadManager.java | 21 +++++ .../org/apache/druid/tasklogs/TaskLogs.java | 2 +- .../druid/tasklogs/TaskPayloadManager.java | 9 +- .../java/org/apache/druid/cli/CliPeon.java | 13 +-- 16 files changed, 169 insertions(+), 88 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index f4bd11069615..c80c788510a5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -89,6 +89,7 @@ protected enum State private final ObjectMapper mapper; private final TaskStateListener stateListener; + private final KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; @MonotonicNonNull private LogWatch logWatch; @@ -99,7 +100,8 @@ protected KubernetesPeonLifecycle( KubernetesPeonClient kubernetesClient, TaskLogs taskLogs, ObjectMapper mapper, - TaskStateListener stateListener + TaskStateListener stateListener, + KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig ) { this.taskId = new K8sTaskId(task); @@ -107,6 +109,7 @@ protected KubernetesPeonLifecycle( this.taskLogs = taskLogs; this.mapper = mapper; this.stateListener = stateListener; + this.kubernetesTaskRunnerConfig = kubernetesTaskRunnerConfig; } /** @@ -123,9 +126,13 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); + // Need to use the taskPayloadManager if we are not passing the task via env variable. + if (!kubernetesTaskRunnerConfig.isTaskPayloadAsEnvVariable()) { + writeTaskPayload(task); + } + // In case something bad happens and run is called twice on this KubernetesPeonLifecycle, reset taskLocation. taskLocation = null; - writeTaskPayload(task); kubernetesClient.launchPeonJobAndWaitForStart( job, launchTimeout, @@ -148,7 +155,7 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, } } - private void writeTaskPayload(Task task) throws IOException{ + private void writeTaskPayload(Task task) throws IOException { Path file = Files.createTempFile(taskId.getOriginalTaskId(), "task.json"); try { FileUtils.writeStringToFile(file.toFile(), mapper.writeValueAsString(task), Charset.defaultCharset()); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java index bf4e3a712577..2f8dd1d49a9b 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java @@ -23,22 +23,26 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; +import org.apache.druid.tasklogs.TaskPayloadManager; public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory { private final KubernetesPeonClient client; private final TaskLogs taskLogs; private final ObjectMapper mapper; + private final KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; public KubernetesPeonLifecycleFactory( KubernetesPeonClient client, TaskLogs taskLogs, - ObjectMapper mapper + ObjectMapper mapper, + KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig ) { this.client = client; this.taskLogs = taskLogs; this.mapper = mapper; + this.kubernetesTaskRunnerConfig = kubernetesTaskRunnerConfig; } @Override @@ -49,7 +53,8 @@ public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStat client, taskLogs, mapper, - stateListener + stateListener, + kubernetesTaskRunnerConfig ); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 206dee6c7e3f..1af6444c6c5a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -41,6 +41,7 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.tasklogs.TaskLogs; +import org.apache.druid.tasklogs.TaskPayloadManager; import java.util.Locale; import java.util.Properties; @@ -59,7 +60,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory response = jsonMapper.readValue( - serviceClient.request( - new RequestBuilder(HttpMethod.GET, path), - new BytesFullResponseHandler() - ).getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - task = jsonMapper.convertValue( - response.get("payload"), - Task.class - ); - } else { - task = jsonMapper.readValue(taskFile, Task.class); - } - + task = jsonMapper.readValue(taskFile, Task.class); log.info( "Running with task: %s", jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task) ); } - catch (ExecutionException | IOException e) { + catch (IOException e) { throw new RuntimeException(e); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java index 8862c3b387ea..241f86937957 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java @@ -37,7 +37,7 @@ public class ExecutorLifecycleConfig @JsonProperty @NotNull - private String taskId = ""; + private boolean useTaskPayloadManager = false; @JsonProperty @NotNull @@ -73,15 +73,15 @@ public File getStatusFile() return statusFile; } - public ExecutorLifecycleConfig setTaskId(String taskId) + public ExecutorLifecycleConfig setUseTaskPayloadManager(boolean useTaskPayloadManager) { - this.taskId = taskId; + this.useTaskPayloadManager = useTaskPayloadManager; return this; } - public String getTaskId() + public boolean isUseTaskPayloadManager() { - return taskId; + return useTaskPayloadManager; } public ExecutorLifecycleConfig setStatusFile(File statusFile) diff --git a/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java b/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java index 287b2f6fcc34..e46871d05d3d 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.common.logger.Logger; import java.io.File; +import java.io.IOException; import java.io.InputStream; public class NoopTaskLogs implements TaskLogs @@ -64,4 +65,14 @@ public void killOlderThan(long timestamp) { log.info("Noop: No task logs are deleted."); } + + @Override + public void pushTaskPayload(String taskid, File taskPayloadFile) { + log.info("Not pushing payload for task: %s", taskid); + } + + @Override + public Optional streamTaskPayload(String taskid) { + return Optional.absent(); + } } diff --git a/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java b/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java new file mode 100644 index 000000000000..c366900bd90e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java @@ -0,0 +1,21 @@ +package org.apache.druid.tasklogs; + +import com.google.common.base.Optional; +import org.apache.druid.java.util.common.logger.Logger; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + +public class NoopTaskPayloadManager implements TaskPayloadManager { + private final Logger log = new Logger(TaskPayloadManager.class); + @Override + public void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { + log.info("Not pushing task payload for task: %s", taskid); + } + + @Override + public Optional streamTaskPayload(String taskid) throws IOException { + return Optional.absent(); + } +} diff --git a/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java b/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java index 2029c74aa501..2756911f6a34 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/TaskLogs.java @@ -23,6 +23,6 @@ @ExtensionPoint -public interface TaskLogs extends TaskLogStreamer, TaskLogPusher, TaskLogKiller +public interface TaskLogs extends TaskLogStreamer, TaskLogPusher, TaskLogKiller, TaskPayloadManager { } diff --git a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java index 4e1ac066f993..978cdc3384b3 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java @@ -1,13 +1,18 @@ package org.apache.druid.tasklogs; import com.google.common.base.Optional; +import org.apache.commons.lang.NotImplementedException; import java.io.File; import java.io.IOException; import java.io.InputStream; public interface TaskPayloadManager { - void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException; + default void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { + throw new NotImplementedException("Managing task payloads is not implemented for this druid.indexer.logs.type"); + } - Optional streamTaskPayload(String taskid) throws IOException; + default Optional streamTaskPayload(String taskid) throws IOException { + throw new NotImplementedException("Managing task payloads is not implemented for this druid.indexer.task.logs"); + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index d8a02ab86674..d143456b39fa 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -40,7 +40,9 @@ import com.google.inject.name.Named; import com.google.inject.name.Names; import io.netty.util.SuppressForbidden; +import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; @@ -68,7 +70,6 @@ import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.tasklogs.TaskLogs; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; @@ -132,6 +133,7 @@ import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.server.metrics.ServiceStatusMonitor; +import org.apache.druid.tasklogs.TaskPayloadManager; import org.eclipse.jetty.server.Server; import java.io.File; @@ -237,7 +239,7 @@ public void configure(Binder binder) ExecutorLifecycleConfig executorLifecycleConfig = new ExecutorLifecycleConfig() .setTaskFile(Paths.get(taskDirPath, "task.json").toFile()) .setStatusFile(Paths.get(taskDirPath, "attempt", attemptId, "status.json").toFile()) - .setTaskId(taskId); + .setUseTaskPayloadManager(StringUtils.isNotEmpty(taskId)); if ("k8s".equals(properties.getProperty("druid.indexer.runner.type", null))) { log.info("Running peon in k8s mode"); @@ -290,11 +292,12 @@ public Supplier> heartbeatDimensions(Task task) @Provides @LazySingleton - public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskLogs taskLogs) + public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager) { try { - if (!config.getTaskId().equals("")) { - String task = IOUtils.toString(taskLogs.streamTaskPayload(config.getTaskId()).get(), Charset.defaultCharset()); + if (config.isUseTaskPayloadManager()) { + String task = IOUtils.toString(taskPayloadManager.streamTaskPayload(taskId).get(), Charset.defaultCharset()); + FileUtils.write(config.getTaskFile(), task, Charset.defaultCharset()); return smileMapper.readValue(task, Task.class); } return mapper.readValue(config.getTaskFile(), Task.class); From 455dfcfc7f383ad0f39ec07dae36a034269b1df5 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 17 Aug 2023 16:52:31 -0400 Subject: [PATCH 03/27] Remove unneeded changes --- .../overlord/KubernetesTaskRunnerFactory.java | 8 +------ .../KubernetesTaskRunnerFactoryTest.java | 24 +++++++------------ .../storage/s3/S3StorageDruidModule.java | 3 --- .../worker/executor/ExecutorLifecycle.java | 11 ++------- .../java/org/apache/druid/cli/CliPeon.java | 1 + 5 files changed, 12 insertions(+), 35 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 1af6444c6c5a..4a8224b15e02 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -23,8 +23,6 @@ import com.google.inject.Inject; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.annotations.EscalatedGlobal; -import org.apache.druid.guice.annotations.Json; -import org.apache.druid.guice.annotations.JsonNonNull; import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.config.TaskConfig; @@ -41,7 +39,6 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.tasklogs.TaskLogs; -import org.apache.druid.tasklogs.TaskPayloadManager; import java.util.Locale; import java.util.Properties; @@ -60,7 +57,6 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory statusFuture = null; private volatile FileChannel taskLockChannel; @@ -83,9 +81,7 @@ public ExecutorLifecycle( TaskConfig taskConfig, TaskActionClientFactory taskActionClientFactory, TaskRunner taskRunner, - ObjectMapper jsonMapper, - @EscalatedGlobal final ServiceClientFactory clientFactory, - @IndexingService final ServiceLocator serviceLocator + ObjectMapper jsonMapper ) { this.taskExecutorConfig = taskExecutorConfig; @@ -93,10 +89,6 @@ public ExecutorLifecycle( this.taskActionClientFactory = taskActionClientFactory; this.taskRunner = taskRunner; this.jsonMapper = jsonMapper.copy().addMixIn(PasswordProvider.class, PasswordProviderRedactionMixIn.class); - this.serviceClient = clientFactory.makeClient("peon", serviceLocator, StandardRetryPolicy.builder() - .maxAttempts(5) - .retryNotAvailable(false) - .build()); } @LifecycleStart @@ -108,6 +100,7 @@ public void start() throws InterruptedException try { task = jsonMapper.readValue(taskFile, Task.class); + log.info( "Running with task: %s", jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task) diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index d143456b39fa..8338379a569b 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -297,6 +297,7 @@ public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, try { if (config.isUseTaskPayloadManager()) { String task = IOUtils.toString(taskPayloadManager.streamTaskPayload(taskId).get(), Charset.defaultCharset()); + // write the remote task.json to task file location for ExecutorLifecycle to pickup FileUtils.write(config.getTaskFile(), task, Charset.defaultCharset()); return smileMapper.readValue(task, Task.class); } From 6dd571369f2e389b6874ffd1023948879ef7cfef Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 18 Aug 2023 13:45:25 -0400 Subject: [PATCH 04/27] Working with new configs --- .../k8s/overlord/KubernetesPeonLifecycle.java | 12 ++-- .../KubernetesPeonLifecycleFactory.java | 10 ++-- .../overlord/KubernetesTaskRunnerFactory.java | 11 ++-- .../overlord/taskadapter/K8sTaskAdapter.java | 38 ++++++++++-- .../MultiContainerTaskAdapter.java | 6 +- .../taskadapter/PodTemplateTaskAdapter.java | 30 +++++++++- .../SingleContainerTaskAdapter.java | 6 +- .../k8s/overlord/taskadapter/TaskAdapter.java | 2 + .../overlord/KubernetesPeonLifecycleTest.java | 58 ++++++++++--------- .../DruidPeonClientIntegrationTest.java | 3 +- .../taskadapter/K8sTaskAdapterTest.java | 39 ++++++++----- .../MultiContainerTaskAdapterTest.java | 24 +++++--- .../PodTemplateTaskAdapterTest.java | 39 +++++++++---- .../SingleContainerTaskAdapterTest.java | 7 ++- .../indexing/common/config/TaskConfig.java | 22 +++++-- .../executor/ExecutorLifecycleConfig.java | 15 ----- .../common/config/TaskConfigBuilder.java | 9 ++- .../java/org/apache/druid/cli/CliPeon.java | 7 +-- 18 files changed, 225 insertions(+), 113 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index c80c788510a5..72e7d7c15480 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -30,6 +30,7 @@ import org.apache.commons.io.IOUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -88,8 +89,7 @@ protected enum State private final KubernetesPeonClient kubernetesClient; private final ObjectMapper mapper; private final TaskStateListener stateListener; - - private final KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; + private final TaskConfig taskConfig; @MonotonicNonNull private LogWatch logWatch; @@ -101,7 +101,7 @@ protected KubernetesPeonLifecycle( TaskLogs taskLogs, ObjectMapper mapper, TaskStateListener stateListener, - KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig + TaskConfig taskConfig ) { this.taskId = new K8sTaskId(task); @@ -109,7 +109,7 @@ protected KubernetesPeonLifecycle( this.taskLogs = taskLogs; this.mapper = mapper; this.stateListener = stateListener; - this.kubernetesTaskRunnerConfig = kubernetesTaskRunnerConfig; + this.taskConfig = taskConfig; } /** @@ -127,7 +127,7 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, updateState(new State[]{State.NOT_STARTED}, State.PENDING); // Need to use the taskPayloadManager if we are not passing the task via env variable. - if (!kubernetesTaskRunnerConfig.isTaskPayloadAsEnvVariable()) { + if (taskConfig.isEnableTaskPayloadManagerPerTask()) { writeTaskPayload(task); } @@ -142,7 +142,7 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, return join(timeout); } catch (IOException e) { - log.info("Failed to run task: %s", taskId.getOriginalTaskId()); + log.info("Failed to write task payload for: %s", taskId.getOriginalTaskId()); throw new RuntimeException(e); } catch (Exception e) { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java index 2f8dd1d49a9b..0ab40c4698a0 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java @@ -20,29 +20,29 @@ package org.apache.druid.k8s.overlord; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; -import org.apache.druid.tasklogs.TaskPayloadManager; public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory { private final KubernetesPeonClient client; private final TaskLogs taskLogs; private final ObjectMapper mapper; - private final KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; + private final TaskConfig taskConfig; public KubernetesPeonLifecycleFactory( KubernetesPeonClient client, TaskLogs taskLogs, ObjectMapper mapper, - KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig + TaskConfig taskConfig ) { this.client = client; this.taskLogs = taskLogs; this.mapper = mapper; - this.kubernetesTaskRunnerConfig = kubernetesTaskRunnerConfig; + this.taskConfig = taskConfig; } @Override @@ -54,7 +54,7 @@ public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStat taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 4a8224b15e02..1324c558071f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -99,7 +99,7 @@ public KubernetesTaskRunner build() kubernetesTaskRunnerConfig, peonClient, httpClient, - new KubernetesPeonLifecycleFactory(peonClient, taskLogs, smileMapper, kubernetesTaskRunnerConfig), + new KubernetesPeonLifecycleFactory(peonClient, taskLogs, smileMapper, taskConfig), emitter ); return runner; @@ -135,7 +135,8 @@ private TaskAdapter buildTaskAdapter(DruidKubernetesClient client) taskConfig, startupLoggingConfig, druidNode, - smileMapper + smileMapper, + taskLogs ); } else if (PodTemplateTaskAdapter.TYPE.equals(adapter)) { return new PodTemplateTaskAdapter( @@ -143,7 +144,8 @@ private TaskAdapter buildTaskAdapter(DruidKubernetesClient client) taskConfig, druidNode, smileMapper, - properties + properties, + taskLogs ); } else { return new SingleContainerTaskAdapter( @@ -152,7 +154,8 @@ private TaskAdapter buildTaskAdapter(DruidKubernetesClient client) taskConfig, startupLoggingConfig, druidNode, - smileMapper + smileMapper, + taskLogs ); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index 174dbeb67582..678a1602fb8d 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -41,6 +41,7 @@ import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; +import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; @@ -48,6 +49,7 @@ import org.apache.druid.indexing.overlord.QuotableWhiteSpaceSplitter; import org.apache.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.Base64Compression; @@ -57,8 +59,11 @@ import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogs; import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -89,6 +94,7 @@ public abstract class K8sTaskAdapter implements TaskAdapter protected final StartupLoggingConfig startupLoggingConfig; protected final DruidNode node; protected final ObjectMapper mapper; + protected final TaskLogs taskLogs; public K8sTaskAdapter( KubernetesClientApi client, @@ -96,7 +102,8 @@ public K8sTaskAdapter( TaskConfig taskConfig, StartupLoggingConfig startupLoggingConfig, DruidNode node, - ObjectMapper mapper + ObjectMapper mapper, + TaskLogs taskLogs ) { this.client = client; @@ -105,6 +112,7 @@ public K8sTaskAdapter( this.startupLoggingConfig = startupLoggingConfig; this.node = node; this.mapper = mapper; + this.taskLogs = taskLogs; } @Override @@ -126,6 +134,14 @@ public Job fromTask(Task task) throws IOException @Override public Task toTask(Job from) throws IOException { + if (taskConfig.isEnableTaskPayloadManagerPerTask()) { + com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); + if (!taskBody.isPresent()) { + throw new IOE("Could not find task payload in task logs for job [%s]", from.getMetadata().getName()); + } + String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); + return mapper.readValue(task, Task.class); + } PodSpec podSpec = from.getSpec().getTemplate().getSpec(); massageSpec(podSpec, "main"); List envVars = podSpec.getContainers().get(0).getEnv(); @@ -137,6 +153,19 @@ public Task toTask(Job from) throws IOException return mapper.readValue(Base64Compression.decompressBase64(contents), Task.class); } + @Override + public K8sTaskId getTaskId(Job from) throws IOException { + Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); + if (annotations == null) { + throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); + } + String taskId = annotations.get(DruidK8sConstants.TASK_ID); + if (taskId == null) { + throw new IOE("No task_id annotation found on pod spec for job [%s]", from.getMetadata().getName()); + } + return new K8sTaskId(taskId); + } + @VisibleForTesting abstract Job createJobFromPodSpec(PodSpec podSpec, Task task, PeonCommandContext context) throws IOException; @@ -409,11 +438,8 @@ private List generateCommand(Task task) command.add("true"); } - // If we are not passing the TASK_JSON, pass the taskId flag to pull the task payload. - if (!taskRunnerConfig.isTaskPayloadAsEnvVariable()) { - command.add("--taskId"); - command.add(task.getId()); - } + command.add("--taskId"); + command.add(task.getId()); log.info( "Peon Command for K8s job: %s", ForkingTaskRunner.getMaskedCommand(startupLoggingConfig.getMaskProperties(), command) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapter.java index 9cda8f864882..a81154a3bcba 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapter.java @@ -43,6 +43,7 @@ import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogs; import java.io.IOException; import java.util.Collections; @@ -59,10 +60,11 @@ public MultiContainerTaskAdapter( TaskConfig taskConfig, StartupLoggingConfig startupLoggingConfig, DruidNode druidNode, - ObjectMapper mapper + ObjectMapper mapper, + TaskLogs taskLogs ) { - super(client, taskRunnerConfig, taskConfig, startupLoggingConfig, druidNode, mapper); + super(client, taskRunnerConfig, taskConfig, startupLoggingConfig, druidNode, mapper, taskLogs); } @Override diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 76a7b3c52268..c0dcee6e5674 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -35,6 +35,7 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import io.fabric8.kubernetes.client.utils.Serialization; +import org.apache.commons.io.IOUtils; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; @@ -49,9 +50,12 @@ import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.KubernetesOverlordUtils; import org.apache.druid.server.DruidNode; +import org.apache.druid.tasklogs.TaskLogs; import java.io.File; import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; import java.nio.file.Files; import java.util.Collection; import java.util.HashMap; @@ -86,13 +90,15 @@ public class PodTemplateTaskAdapter implements TaskAdapter private final DruidNode node; private final ObjectMapper mapper; private final HashMap templates; + private final TaskLogs taskLogs; public PodTemplateTaskAdapter( KubernetesTaskRunnerConfig taskRunnerConfig, TaskConfig taskConfig, DruidNode node, ObjectMapper mapper, - Properties properties + Properties properties, + TaskLogs taskLogs ) { this.taskRunnerConfig = taskRunnerConfig; @@ -100,6 +106,7 @@ public PodTemplateTaskAdapter( this.node = node; this.mapper = mapper; this.templates = initializePodTemplates(properties); + this.taskLogs = taskLogs; } /** @@ -162,6 +169,14 @@ public Job fromTask(Task task) throws IOException @Override public Task toTask(Job from) throws IOException { + if (taskConfig.isEnableTaskPayloadManagerPerTask()) { + com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); + if (!taskBody.isPresent()) { + throw new IOE("Could not find task payload in task logs for job [%s]", from.getMetadata().getName()); + } + String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); + return mapper.readValue(task, Task.class); + } Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); if (annotations == null) { throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); @@ -173,6 +188,19 @@ public Task toTask(Job from) throws IOException return mapper.readValue(Base64Compression.decompressBase64(task), Task.class); } + @Override + public K8sTaskId getTaskId(Job from) throws IOException { + Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); + if (annotations == null) { + throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); + } + String taskId = annotations.get(DruidK8sConstants.TASK_ID); + if (taskId == null) { + throw new IOE("No task_id annotation found on pod spec for job [%s]", from.getMetadata().getName()); + } + return new K8sTaskId(taskId); + } + private HashMap initializePodTemplates(Properties properties) { HashMap podTemplateMap = new HashMap<>(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapter.java index c64acd153106..be0588c35b15 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapter.java @@ -32,6 +32,7 @@ import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogs; import java.io.IOException; import java.util.Collections; @@ -47,10 +48,11 @@ public SingleContainerTaskAdapter( TaskConfig taskConfig, StartupLoggingConfig startupLoggingConfig, DruidNode druidNode, - ObjectMapper mapper + ObjectMapper mapper, + TaskLogs taskLogs ) { - super(client, taskRunnerConfig, taskConfig, startupLoggingConfig, druidNode, mapper); + super(client, taskRunnerConfig, taskConfig, startupLoggingConfig, druidNode, mapper, taskLogs); } @Override diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java index 05933604f2ba..718f70f2f82a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java @@ -21,6 +21,7 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.common.K8sTaskId; import java.io.IOException; @@ -31,4 +32,5 @@ public interface TaskAdapter Task toTask(Job from) throws IOException; + K8sTaskId getTaskId(Job from) throws IOException; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 14e1c89f2336..704481377953 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -30,6 +30,8 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; 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.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.k8s.overlord.common.JobResponse; @@ -70,14 +72,14 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport private ObjectMapper mapper; private Task task; private K8sTaskId k8sTaskId; - private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; + private TaskConfig taskConfig; @Before public void setup() { mapper = new TestUtils().getTestObjectMapper(); task = NoopTask.create(ID, 0); k8sTaskId = new K8sTaskId(task); - kubernetesTaskRunnerConfig = new KubernetesTaskRunnerConfig(); + taskConfig = new TaskConfigBuilder().build(); EasyMock.expect(logWatch.getOutput()).andReturn(IOUtils.toInputStream("", StandardCharsets.UTF_8)).anyTimes(); } @@ -90,7 +92,7 @@ public void test_run() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ) { @Override @@ -135,7 +137,7 @@ public void test_run_whenCalledMultipleTimes_raisesIllegalStateException() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ) { @Override @@ -184,7 +186,7 @@ public void test_run_whenExceptionRaised_setsRunnerTaskStateToNone() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ) { @Override @@ -231,7 +233,7 @@ public void test_join_withoutJob_returnsFailedTaskStatus() throws IOException taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( @@ -272,7 +274,7 @@ public void test_join() throws IOException taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); Job job = new JobBuilder() @@ -326,7 +328,7 @@ public void test_join_whenCalledMultipleTimes_raisesIllegalStateException() thro taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); Job job = new JobBuilder() @@ -388,7 +390,7 @@ public void test_join_withoutTaskStatus_returnsFailedTaskStatus() throws IOExcep taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); Job job = new JobBuilder() @@ -440,7 +442,7 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskStatus_returnsFaile taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); Job job = new JobBuilder() @@ -492,7 +494,7 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskLogs_isIgnored() th taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); Job job = new JobBuilder() @@ -544,7 +546,7 @@ public void test_join_whenRuntimeExceptionThrownWhileWaitingForKubernetesJob_thr taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( @@ -586,7 +588,7 @@ public void test_shutdown_withNotStartedTaskState() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); peonLifecycle.shutdown(); } @@ -600,7 +602,7 @@ public void test_shutdown_withPendingTaskState() throws NoSuchFieldException, Il taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -622,7 +624,7 @@ public void test_shutdown_withRunningTaskState() throws NoSuchFieldException, Il taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -644,7 +646,7 @@ public void test_shutdown_withStoppedTaskState() throws NoSuchFieldException, Il taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); @@ -660,7 +662,7 @@ public void test_streamLogs_withNotStartedTaskState() throws NoSuchFieldExceptio taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); @@ -676,7 +678,7 @@ public void test_streamLogs_withPendingTaskState() throws NoSuchFieldException, taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -692,7 +694,7 @@ public void test_streamLogs_withRunningTaskState() throws NoSuchFieldException, taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -716,7 +718,7 @@ public void test_streamLogs_withStoppedTaskState() throws NoSuchFieldException, taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); @@ -733,7 +735,7 @@ public void test_getTaskLocation_withNotStartedTaskState_returnsUnknown() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); @@ -750,7 +752,7 @@ public void test_getTaskLocation_withPendingTaskState_returnsUnknown() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -767,7 +769,7 @@ public void test_getTaskLocation_withRunningTaskState_withoutPeonPod_returnsUnkn taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -790,7 +792,7 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithoutStatus_r taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -819,7 +821,7 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatus_retu taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -856,7 +858,7 @@ public void test_getTaskLocation_saveTaskLocation() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -893,7 +895,7 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatusWithT taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -931,7 +933,7 @@ public void test_getTaskLocation_withStoppedTaskState_returnsUnknown() taskLogs, mapper, stateListener, - kubernetesTaskRunnerConfig + taskConfig ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 327e4276d1cd..57c0f515ddc1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -120,7 +120,8 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception taskConfig, startupLoggingConfig, druidNode, - jsonMapper + jsonMapper, + null ); String taskBasePath = "/home/taskDir"; PeonCommandContext context = new PeonCommandContext(Collections.singletonList( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 37adcbd1cbf1..e9bc01d12114 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -59,6 +59,8 @@ import org.apache.druid.k8s.overlord.common.TestKubernetesClient; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogs; +import org.easymock.Mock; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -84,6 +86,8 @@ class K8sTaskAdapterTest private final TaskConfig taskConfig; private final DruidNode node; private final ObjectMapper jsonMapper; + @Mock private TaskLogs taskLogs; + public K8sTaskAdapterTest() { @@ -141,7 +145,9 @@ public PodSpec getSpec() taskConfig, startupLoggingConfig, node, - jsonMapper + jsonMapper, + taskLogs + ); Task task = K8sTestUtils.getTask(); Job jobFromSpec = adapter.fromTask(task); @@ -168,7 +174,8 @@ public void serializingAndDeserializingATask() throws IOException taskConfig, startupLoggingConfig, node, - jsonMapper + jsonMapper, + taskLogs ); Task task = K8sTestUtils.getTask(); Job jobFromSpec = adapter.createJobFromPodSpec( @@ -222,7 +229,8 @@ public PodSpec getSpec() taskConfig, startupLoggingConfig, node, - jsonMapper + jsonMapper, + taskLogs ); Task task = K8sTestUtils.getTask(); Job job = adapter.fromTask(task); @@ -342,7 +350,8 @@ void testAddingMonitors() throws IOException taskConfig, startupLoggingConfig, node, - jsonMapper + jsonMapper, + taskLogs ); Task task = K8sTestUtils.getTask(); // no monitor in overlord, no monitor override @@ -365,7 +374,8 @@ void testAddingMonitors() throws IOException taskConfig, startupLoggingConfig, node, - jsonMapper + jsonMapper, + taskLogs ); adapter.addEnvironmentVariables(container, context, task.toString()); EnvVar env = container.getEnv() @@ -382,7 +392,8 @@ void testAddingMonitors() throws IOException taskConfig, startupLoggingConfig, node, - jsonMapper + jsonMapper, + taskLogs ); container.getEnv().add(new EnvVarBuilder() .withName("druid_monitoring_monitors") @@ -407,13 +418,15 @@ void testEphemeralStorageIsRespected() throws IOException KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); - SingleContainerTaskAdapter adapter = - new SingleContainerTaskAdapter(testClient, - config, taskConfig, - startupLoggingConfig, - node, - jsonMapper - ); + SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter( + testClient, + config, + taskConfig, + startupLoggingConfig, + node, + jsonMapper, + taskLogs + ); NoopTask task = NoopTask.create("id", 1); Job actual = adapter.createJobFromPodSpec( pod.getSpec(), diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java index aa93856ded86..c0831a703d47 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java @@ -41,6 +41,8 @@ import org.apache.druid.k8s.overlord.common.TestKubernetesClient; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogs; +import org.easymock.Mock; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -58,6 +60,7 @@ class MultiContainerTaskAdapterTest private TaskConfig taskConfig; private DruidNode druidNode; private ObjectMapper jsonMapper; + @Mock private TaskLogs taskLogs; @BeforeEach public void setup() @@ -98,7 +101,8 @@ public void testMultiContainerSupport() throws IOException taskConfig, startupLoggingConfig, druidNode, - jsonMapper + jsonMapper, + taskLogs ); NoopTask task = NoopTask.create("id", 1); Job actual = adapter.createJobFromPodSpec( @@ -146,7 +150,8 @@ public void testMultiContainerSupportWithNamedContainer() throws IOException taskConfig, startupLoggingConfig, druidNode, - jsonMapper + jsonMapper, + taskLogs ); NoopTask task = NoopTask.create("id", 1); PodSpec spec = pod.getSpec(); @@ -191,12 +196,15 @@ public void testOverridingPeonMonitors() throws IOException .withPrimaryContainerName("primary") .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor")) .build(); - MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter(testClient, - config, - taskConfig, - startupLoggingConfig, - druidNode, - jsonMapper); + MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter( + testClient, + config, + taskConfig, + startupLoggingConfig, + druidNode, + jsonMapper, + taskLogs + ); NoopTask task = NoopTask.create("id", 1); PodSpec spec = pod.getSpec(); K8sTaskAdapter.massageSpec(spec, config.getPrimaryContainerName()); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index e06713a415d4..b800a407fcc7 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -36,7 +36,9 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.server.DruidNode; +import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; +import org.easymock.Mock; import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -59,6 +61,7 @@ public class PodTemplateTaskAdapterTest private TaskConfig taskConfig; private DruidNode node; private ObjectMapper mapper; + @Mock private TaskLogs taskLogs; @BeforeEach public void setup() @@ -89,7 +92,8 @@ public void test_fromTask_withoutBasePodTemplateInRuntimeProperites_raisesIAE() taskConfig, node, mapper, - new Properties() + new Properties(), + taskLogs )); } @@ -109,7 +113,8 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_r taskConfig, node, mapper, - props + props, + taskLogs )); } @@ -127,7 +132,8 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites() throws IOExce taskConfig, node, mapper, - props + props, + taskLogs ); Task task = new NoopTask( @@ -168,7 +174,8 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_andTlsEnabled() true ), mapper, - props + props, + taskLogs ); Task task = new NoopTask( @@ -204,7 +211,8 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperties_withEmptyFile_r taskConfig, node, mapper, - props + props, + taskLogs )); } @@ -223,7 +231,8 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites() throws IOExce taskConfig, node, mapper, - props + props, + taskLogs ); Task task = new NoopTask( @@ -258,7 +267,8 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites_dontSetTaskJSON taskConfig, node, mapper, - props + props, + taskLogs ); Task task = new NoopTask( @@ -292,7 +302,8 @@ public void test_fromTask_withoutAnnotations_throwsIOE() throws IOException taskConfig, node, mapper, - props + props, + taskLogs ); Job job = K8sTestUtils.fileToResource("baseJobWithoutAnnotations.yaml", Job.class); @@ -315,7 +326,8 @@ public void test_fromTask_withoutTaskAnnotation_throwsIOE() throws IOException taskConfig, node, mapper, - props + props, + taskLogs ); Job baseJob = K8sTestUtils.fileToResource("baseJobWithoutAnnotations.yaml", Job.class); @@ -346,7 +358,8 @@ public void test_fromTask() throws IOException taskConfig, node, mapper, - props + props, + taskLogs ); Job job = K8sTestUtils.fileToResource("baseJob.yaml", Job.class); @@ -371,7 +384,8 @@ public void test_fromTask_withRealIds() throws IOException taskConfig, node, mapper, - props + props, + taskLogs ); Task task = new NoopTask( @@ -406,7 +420,8 @@ public void test_fromTask_taskSupportsQueries() throws IOException taskConfig, node, mapper, - props + props, + taskLogs ); Task task = EasyMock.mock(Task.class); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java index 3e51c5a7acbc..372b7cfc2929 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java @@ -39,6 +39,8 @@ import org.apache.druid.k8s.overlord.common.TestKubernetesClient; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.TaskLogs; +import org.easymock.Mock; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -57,6 +59,8 @@ class SingleContainerTaskAdapterTest private DruidNode druidNode; private ObjectMapper jsonMapper; + @Mock private TaskLogs taskLogs; + @BeforeEach public void setup() { @@ -96,7 +100,8 @@ public void testSingleContainerSupport() throws IOException taskConfig, startupLoggingConfig, druidNode, - jsonMapper + jsonMapper, + taskLogs ); NoopTask task = NoopTask.create("id", 1); Job actual = adapter.createJobFromPodSpec( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index db48d6f07f7d..022cd3a9d69f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -125,6 +125,9 @@ public enum BatchProcessingMode @JsonProperty private final long tmpStorageBytesPerTask; + @JsonProperty + private final boolean enableTaskPayloadManagerPerTask; + @JsonCreator public TaskConfig( @JsonProperty("baseDir") String baseDir, @@ -142,7 +145,8 @@ public TaskConfig( @JsonProperty("batchProcessingMode") String batchProcessingMode, @JsonProperty("storeEmptyColumns") @Nullable Boolean storeEmptyColumns, @JsonProperty("encapsulatedTask") boolean enableTaskLevelLogPush, - @JsonProperty("tmpStorageBytesPerTask") @Nullable Long tmpStorageBytesPerTask + @JsonProperty("tmpStorageBytesPerTask") @Nullable Long tmpStorageBytesPerTask, + @JsonProperty("enableTaskPayloadManagerPerTask") @Nullable boolean enableTaskPayloadManagerPerTask ) { this.baseDir = Configs.valueOrDefault(baseDir, System.getProperty("java.io.tmpdir")); @@ -193,6 +197,7 @@ public TaskConfig( this.storeEmptyColumns = Configs.valueOrDefault(storeEmptyColumns, DEFAULT_STORE_EMPTY_COLUMNS); this.tmpStorageBytesPerTask = Configs.valueOrDefault(tmpStorageBytesPerTask, DEFAULT_TMP_STORAGE_BYTES_PER_TASK); + this.enableTaskPayloadManagerPerTask = enableTaskPayloadManagerPerTask; } private TaskConfig( @@ -210,7 +215,8 @@ private TaskConfig( BatchProcessingMode batchProcessingMode, boolean storeEmptyColumns, boolean encapsulatedTask, - long tmpStorageBytesPerTask + long tmpStorageBytesPerTask, + boolean enableTaskPayloadManagerPerTask ) { this.baseDir = baseDir; @@ -228,6 +234,7 @@ private TaskConfig( this.storeEmptyColumns = storeEmptyColumns; this.encapsulatedTask = encapsulatedTask; this.tmpStorageBytesPerTask = tmpStorageBytesPerTask; + this.enableTaskPayloadManagerPerTask = enableTaskPayloadManagerPerTask; } @JsonProperty @@ -338,6 +345,11 @@ public boolean isEncapsulatedTask() return encapsulatedTask; } + @JsonProperty + public boolean isEnableTaskPayloadManagerPerTask() { + return enableTaskPayloadManagerPerTask; + } + @JsonProperty public long getTmpStorageBytesPerTask() { @@ -370,7 +382,8 @@ public TaskConfig withBaseTaskDir(File baseTaskDir) batchProcessingMode, storeEmptyColumns, encapsulatedTask, - tmpStorageBytesPerTask + tmpStorageBytesPerTask, + enableTaskPayloadManagerPerTask ); } @@ -391,7 +404,8 @@ public TaskConfig withTmpStorageBytesPerTask(long tmpStorageBytesPerTask) batchProcessingMode, storeEmptyColumns, encapsulatedTask, - tmpStorageBytesPerTask + tmpStorageBytesPerTask, + enableTaskPayloadManagerPerTask ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java index 241f86937957..0c48fe299a82 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycleConfig.java @@ -35,10 +35,6 @@ public class ExecutorLifecycleConfig @NotNull private File taskFile = null; - @JsonProperty - @NotNull - private boolean useTaskPayloadManager = false; - @JsonProperty @NotNull private File statusFile = null; @@ -73,17 +69,6 @@ public File getStatusFile() return statusFile; } - public ExecutorLifecycleConfig setUseTaskPayloadManager(boolean useTaskPayloadManager) - { - this.useTaskPayloadManager = useTaskPayloadManager; - return this; - } - - public boolean isUseTaskPayloadManager() - { - return useTaskPayloadManager; - } - public ExecutorLifecycleConfig setStatusFile(File statusFile) { this.statusFile = statusFile; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java index af920ebbeb73..9600d185ac63 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java @@ -41,6 +41,7 @@ public class TaskConfigBuilder private Boolean storeEmptyColumns; private boolean enableTaskLevelLogPush; private Long tmpStorageBytesPerTask; + private boolean enableTaskPayloadManagerPerTask; public TaskConfigBuilder setBaseDir(String baseDir) { @@ -132,6 +133,11 @@ public TaskConfigBuilder setTmpStorageBytesPerTask(Long tmpStorageBytesPerTask) return this; } + public TaskConfigBuilder setEnableTaskPayloadManagerPerTask(boolean enableTaskPayloadManagerPerTask) + { + this.enableTaskPayloadManagerPerTask = enableTaskPayloadManagerPerTask; + return this; + } public TaskConfig build() { return new TaskConfig( @@ -149,7 +155,8 @@ public TaskConfig build() batchProcessingMode, storeEmptyColumns, enableTaskLevelLogPush, - tmpStorageBytesPerTask + tmpStorageBytesPerTask, + enableTaskPayloadManagerPerTask ); } } diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 8338379a569b..3948a3dab0e7 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -238,8 +238,7 @@ public void configure(Binder binder) LifecycleModule.register(binder, ExecutorLifecycle.class); ExecutorLifecycleConfig executorLifecycleConfig = new ExecutorLifecycleConfig() .setTaskFile(Paths.get(taskDirPath, "task.json").toFile()) - .setStatusFile(Paths.get(taskDirPath, "attempt", attemptId, "status.json").toFile()) - .setUseTaskPayloadManager(StringUtils.isNotEmpty(taskId)); + .setStatusFile(Paths.get(taskDirPath, "attempt", attemptId, "status.json").toFile()); if ("k8s".equals(properties.getProperty("druid.indexer.runner.type", null))) { log.info("Running peon in k8s mode"); @@ -292,10 +291,10 @@ public Supplier> heartbeatDimensions(Task task) @Provides @LazySingleton - public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager) + public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager, TaskConfig taskConfig) { try { - if (config.isUseTaskPayloadManager()) { + if (taskConfig.isEnableTaskPayloadManagerPerTask()) { String task = IOUtils.toString(taskPayloadManager.streamTaskPayload(taskId).get(), Charset.defaultCharset()); // write the remote task.json to task file location for ExecutorLifecycle to pickup FileUtils.write(config.getTaskFile(), task, Charset.defaultCharset()); From 14650c955c520fd0717a23981481df81de5958b9 Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 18 Aug 2023 13:53:36 -0400 Subject: [PATCH 05/27] Pulling remote changes in --- .../druid/k8s/overlord/KubernetesPeonLifecycle.java | 6 +++--- .../druid/k8s/overlord/KubernetesTaskRunner.java | 3 +-- .../k8s/overlord/KubernetesPeonLifecycleTest.java | 10 ++++------ .../druid/k8s/overlord/KubernetesTaskRunnerTest.java | 6 ++---- 4 files changed, 10 insertions(+), 15 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index 852d97618149..0f26d7254c08 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -73,9 +73,9 @@ public interface TaskStateListener protected enum State { - /** Lifecycle's state before {@link #run(Job, long, long, Task)} or {@link #join(long)} is called. */ + /** Lifecycle's state before {@link #run(Job, long, long)} or {@link #join(long)} is called. */ NOT_STARTED, - /** Lifecycle's state since {@link #run(Job, long, long, Task)} is called. */ + /** Lifecycle's state since {@link #run(Job, long, long)} is called. */ PENDING, /** Lifecycle's state since {@link #join(long)} is called. */ RUNNING, @@ -123,7 +123,7 @@ protected KubernetesPeonLifecycle( * @return * @throws IllegalStateException */ - protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, Task task) throws IllegalStateException + protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) throws IllegalStateException { try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 2b395868ddca..8fc6a5624ddb 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -197,8 +197,7 @@ protected TaskStatus doTask(Task task, boolean run) taskStatus = peonLifecycle.run( adapter.fromTask(task), config.getTaskLaunchTimeout().toStandardDuration().getMillis(), - config.getTaskTimeout().toStandardDuration().getMillis(), - task + config.getTaskTimeout().toStandardDuration().getMillis() ); } else { taskStatus = peonLifecycle.join( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 7d8271488bb9..56e02089dc86 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -38,9 +38,7 @@ import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonPhase; -import org.apache.druid.tasklogs.NoopTaskPayloadManager; import org.apache.druid.tasklogs.TaskLogs; -import org.apache.druid.tasklogs.TaskPayloadManager; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; import org.easymock.EasyMockSupport; @@ -120,7 +118,7 @@ protected synchronized TaskStatus join(long timeout) replayAll(); - TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L, task); + TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L); verifyAll(); @@ -166,12 +164,12 @@ protected synchronized TaskStatus join(long timeout) replayAll(); - peonLifecycle.run(job, 0L, 0L, task); + peonLifecycle.run(job, 0L, 0L); Assert.assertThrows( "Task [id] failed to run: invalid peon lifecycle state transition [STOPPED]->[PENDING]", IllegalStateException.class, - () -> peonLifecycle.run(job, 0L, 0L, task) + () -> peonLifecycle.run(job, 0L, 0L) ); verifyAll(); @@ -219,7 +217,7 @@ protected synchronized TaskStatus join(long timeout) Assert.assertThrows( Exception.class, - () -> peonLifecycle.run(job, 0L, 0L, task) + () -> peonLifecycle.run(job, 0L, 0L) ); verifyAll(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 7bd2efddce23..613e3b1031e1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -142,8 +142,7 @@ public void test_run_withoutExistingTask() throws IOException, ExecutionExceptio EasyMock.expect(kubernetesPeonLifecycle.run( EasyMock.eq(job), EasyMock.anyLong(), - EasyMock.anyLong(), - EasyMock.anyObject() + EasyMock.anyLong() )).andReturn(taskStatus); replayAll(); @@ -180,8 +179,7 @@ public void test_run_whenExceptionThrown_throwsRuntimeException() throws IOExcep EasyMock.expect(kubernetesPeonLifecycle.run( EasyMock.eq(job), EasyMock.anyLong(), - EasyMock.anyLong(), - EasyMock.anyObject() + EasyMock.anyLong() )).andThrow(new IllegalStateException()); replayAll(); From 378a472fd8b83549cfde3c890c7f8ac43fffff7b Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 18 Aug 2023 16:59:10 -0400 Subject: [PATCH 06/27] Fixing checkstyle --- .../k8s/overlord/KubernetesPeonLifecycle.java | 29 +++++++++++-------- .../overlord/taskadapter/K8sTaskAdapter.java | 3 +- .../taskadapter/PodTemplateTaskAdapter.java | 3 +- .../overlord/KubernetesPeonLifecycleTest.java | 1 + .../KubernetesTaskRunnerFactoryTest.java | 4 --- .../apache/druid/storage/s3/S3TaskLogs.java | 7 +++-- .../guice/IndexingServiceTaskLogsModule.java | 1 - .../indexing/common/config/TaskConfig.java | 3 +- .../worker/executor/ExecutorLifecycle.java | 6 ---- .../apache/druid/tasklogs/NoopTaskLogs.java | 7 +++-- .../tasklogs/NoopTaskPayloadManager.java | 21 -------------- .../druid/tasklogs/TaskPayloadManager.java | 28 ++++++++++++++++-- .../java/org/apache/druid/cli/CliPeon.java | 1 - 13 files changed, 57 insertions(+), 57 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index 0f26d7254c08..abcfa4c81d48 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -128,7 +128,6 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); - // Need to use the taskPayloadManager if we are not passing the task via env variable. if (taskConfig.isEnableTaskPayloadManagerPerTask()) { writeTaskPayload(task); } @@ -144,10 +143,6 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) return join(timeout); } - catch (IOException e) { - log.info("Failed to write task payload for: %s", taskId.getOriginalTaskId()); - throw new RuntimeException(e); - } catch (Exception e) { log.info("Failed to run task: %s", taskId.getOriginalTaskId()); shutdown(); @@ -158,15 +153,25 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) } } - private void writeTaskPayload(Task task) throws IOException { - Path file = Files.createTempFile(taskId.getOriginalTaskId(), "task.json"); + private void writeTaskPayload(Task task) + { try { - FileUtils.writeStringToFile(file.toFile(), mapper.writeValueAsString(task), Charset.defaultCharset()); - taskLogs.pushTaskPayload(task.getId(), file.toFile()); - } catch (Exception e) { + Path file = Files.createTempFile(taskId.getOriginalTaskId(), "task.json"); + try { + FileUtils.writeStringToFile(file.toFile(), mapper.writeValueAsString(task), Charset.defaultCharset()); + taskLogs.pushTaskPayload(task.getId(), file.toFile()); + } + catch (Exception e) { + log.error("Failed to write task payload for task: %s", taskId.getOriginalTaskId()); + throw new RuntimeException(e); + } + finally { + Files.deleteIfExists(file); + } + } + catch (IOException e) { + log.error("Failed to write task payload for task: %s", taskId.getOriginalTaskId()); throw new RuntimeException(e); - } finally { - Files.deleteIfExists(file); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index 678a1602fb8d..93b794fd0023 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -154,7 +154,8 @@ public Task toTask(Job from) throws IOException } @Override - public K8sTaskId getTaskId(Job from) throws IOException { + public K8sTaskId getTaskId(Job from) throws IOException + { Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); if (annotations == null) { throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index c0dcee6e5674..1a69ea3b18b5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -189,7 +189,8 @@ public Task toTask(Job from) throws IOException } @Override - public K8sTaskId getTaskId(Job from) throws IOException { + public K8sTaskId getTaskId(Job from) throws IOException + { Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); if (annotations == null) { throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 56e02089dc86..00fe5ae66251 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -71,6 +71,7 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport private Task task; private K8sTaskId k8sTaskId; private TaskConfig taskConfig; + @Before public void setup() { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 9efe8c319fe3..ba9d2accf170 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -32,9 +32,7 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.tasklogs.NoopTaskLogs; -import org.apache.druid.tasklogs.NoopTaskPayloadManager; import org.apache.druid.tasklogs.TaskLogs; -import org.apache.druid.tasklogs.TaskPayloadManager; import org.easymock.Mock; import org.junit.Assert; import org.junit.Before; @@ -55,7 +53,6 @@ public class KubernetesTaskRunnerFactoryTest private DruidKubernetesClient druidKubernetesClient; @Mock private ServiceEmitter emitter; - private TaskPayloadManager taskPayloadManager; @Before public void setup() @@ -78,7 +75,6 @@ public void setup() taskConfig = new TaskConfigBuilder().setBaseDir("/tmp").build(); properties = new Properties(); druidKubernetesClient = new DruidKubernetesClient(); - taskPayloadManager = new NoopTaskPayloadManager(); } @Test diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java index 5eaa8da02482..4032a380e520 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.tasklogs.TaskLogs; -import org.apache.druid.tasklogs.TaskPayloadManager; import java.io.File; import java.io.IOException; @@ -85,14 +84,16 @@ public Optional streamTaskStatus(String taskid) throws IOException } @Override - public void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { + public void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException + { final String taskKey = getTaskLogKey(taskid, "task.json"); log.info("Pushing task payload %s to: %s", taskPayloadFile, taskKey); pushTaskFile(taskPayloadFile, taskKey); } @Override - public Optional streamTaskPayload(String taskid) throws IOException { + public Optional streamTaskPayload(String taskid) throws IOException + { final String taskKey = getTaskLogKey(taskid, "task.json"); return streamTaskFileWithRetry(0, taskKey); } diff --git a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTaskLogsModule.java b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTaskLogsModule.java index e91b3f3bf2e3..6e7df12f8d20 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTaskLogsModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceTaskLogsModule.java @@ -28,7 +28,6 @@ import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.tasklogs.TaskLogKiller; import org.apache.druid.tasklogs.TaskLogPusher; -import org.apache.druid.tasklogs.TaskLogStreamer; import org.apache.druid.tasklogs.TaskLogs; import org.apache.druid.tasklogs.TaskPayloadManager; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index 022cd3a9d69f..f24611a67f0f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -346,7 +346,8 @@ public boolean isEncapsulatedTask() } @JsonProperty - public boolean isEnableTaskPayloadManagerPerTask() { + public boolean isEnableTaskPayloadManagerPerTask() + { return enableTaskPayloadManagerPerTask; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java index f59ba87cc4d7..ad46ee7e666d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -25,8 +25,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; -import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; @@ -41,10 +39,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.metadata.PasswordProviderRedactionMixIn; -import org.apache.druid.rpc.ServiceClient; -import org.apache.druid.rpc.ServiceClientFactory; -import org.apache.druid.rpc.ServiceLocator; -import org.apache.druid.rpc.StandardRetryPolicy; import java.io.File; import java.io.IOException; diff --git a/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java b/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java index e46871d05d3d..5568a5160fd8 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskLogs.java @@ -23,7 +23,6 @@ import org.apache.druid.java.util.common.logger.Logger; import java.io.File; -import java.io.IOException; import java.io.InputStream; public class NoopTaskLogs implements TaskLogs @@ -67,12 +66,14 @@ public void killOlderThan(long timestamp) } @Override - public void pushTaskPayload(String taskid, File taskPayloadFile) { + public void pushTaskPayload(String taskid, File taskPayloadFile) + { log.info("Not pushing payload for task: %s", taskid); } @Override - public Optional streamTaskPayload(String taskid) { + public Optional streamTaskPayload(String taskid) + { return Optional.absent(); } } diff --git a/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java b/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java deleted file mode 100644 index c366900bd90e..000000000000 --- a/processing/src/main/java/org/apache/druid/tasklogs/NoopTaskPayloadManager.java +++ /dev/null @@ -1,21 +0,0 @@ -package org.apache.druid.tasklogs; - -import com.google.common.base.Optional; -import org.apache.druid.java.util.common.logger.Logger; - -import java.io.File; -import java.io.IOException; -import java.io.InputStream; - -public class NoopTaskPayloadManager implements TaskPayloadManager { - private final Logger log = new Logger(TaskPayloadManager.class); - @Override - public void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { - log.info("Not pushing task payload for task: %s", taskid); - } - - @Override - public Optional streamTaskPayload(String taskid) throws IOException { - return Optional.absent(); - } -} diff --git a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java index 978cdc3384b3..881139828ca1 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.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.tasklogs; import com.google.common.base.Optional; @@ -7,12 +26,15 @@ import java.io.IOException; import java.io.InputStream; -public interface TaskPayloadManager { - default void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { +public interface TaskPayloadManager +{ + default void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException + { throw new NotImplementedException("Managing task payloads is not implemented for this druid.indexer.logs.type"); } - default Optional streamTaskPayload(String taskid) throws IOException { + default Optional streamTaskPayload(String taskid) throws IOException + { throw new NotImplementedException("Managing task payloads is not implemented for this druid.indexer.task.logs"); } } diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 3948a3dab0e7..32c748383c92 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -42,7 +42,6 @@ import io.netty.util.SuppressForbidden; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; From 12374bedd89deec24f04aa954efe81e00e19ec11 Mon Sep 17 00:00:00 2001 From: George Wu Date: Sun, 20 Aug 2023 17:23:36 -0400 Subject: [PATCH 07/27] Cleanup unit tests --- .../overlord/KubernetesPeonLifecycleTest.java | 46 +++++++ .../taskadapter/K8sTaskAdapterTest.java | 114 +++++++++++++++++- .../PodTemplateTaskAdapterTest.java | 114 +++++++++++++++++- .../src/test/resources/expectedNoopJob.yaml | 4 +- .../resources/expectedNoopJobLongIds.yaml | 4 +- .../resources/expectedNoopJobNoTaskJson.yaml | 2 + .../resources/expectedNoopJobTlsEnabled.yaml | 4 +- 7 files changed, 277 insertions(+), 11 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 00fe5ae66251..a6a6bcffefe2 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -47,6 +47,7 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.Mockito; import java.io.File; import java.io.IOException; @@ -128,6 +129,51 @@ protected synchronized TaskStatus join(long timeout) Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState()); } + @Test + public void test_run_useTaskManager() throws IOException { + TaskConfig taskConfigTaskManager = new TaskConfigBuilder().setEnableTaskPayloadManagerPerTask(true).build(); + KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle( + task, + kubernetesClient, + taskLogs, + mapper, + stateListener, + taskConfigTaskManager + ) + { + @Override + protected synchronized TaskStatus join(long timeout) + { + return TaskStatus.success(ID); + } + }; + + Job job = new JobBuilder().withNewMetadata().withName(ID).endMetadata().build(); + + EasyMock.expect(kubernetesClient.launchPeonJobAndWaitForStart( + EasyMock.eq(job), + EasyMock.eq(task), + EasyMock.anyLong(), + EasyMock.eq(TimeUnit.MILLISECONDS) + )).andReturn(null); + Assert.assertEquals(KubernetesPeonLifecycle.State.NOT_STARTED, peonLifecycle.getState()); + + stateListener.stateChanged(KubernetesPeonLifecycle.State.PENDING, ID); + EasyMock.expectLastCall().once(); + stateListener.stateChanged(KubernetesPeonLifecycle.State.STOPPED, ID); + EasyMock.expectLastCall().once(); + + taskLogs.pushTaskPayload(EasyMock.anyString(), EasyMock.anyObject()); + replayAll(); + + TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L); + + verifyAll(); + Assert.assertTrue(taskStatus.isSuccess()); + Assert.assertEquals(ID, taskStatus.getId()); + Assert.assertEquals(KubernetesPeonLifecycle.State.STOPPED, peonLifecycle.getState()); + } + @Test public void test_run_whenCalledMultipleTimes_raisesIllegalStateException() { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index e9bc01d12114..285c7bae7d28 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -37,6 +37,7 @@ import io.fabric8.kubernetes.api.model.ResourceRequirements; import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder; import io.fabric8.kubernetes.api.model.batch.v1.Job; +import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import io.fabric8.kubernetes.api.model.batch.v1.JobList; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; @@ -50,8 +51,10 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.IOE; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; +import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.KubernetesExecutor; import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException; @@ -59,13 +62,20 @@ import org.apache.druid.k8s.overlord.common.TestKubernetesClient; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; +import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.tasklogs.TaskLogs; -import org.easymock.Mock; +import org.easymock.EasyMockRunner; +import org.easymock.EasyMockSupport; +import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import java.io.ByteArrayInputStream; import java.io.File; import java.io.IOException; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -86,7 +96,7 @@ class K8sTaskAdapterTest private final TaskConfig taskConfig; private final DruidNode node; private final ObjectMapper jsonMapper; - @Mock private TaskLogs taskLogs; + private final TaskLogs taskLogs; public K8sTaskAdapterTest() @@ -111,6 +121,7 @@ public K8sTaskAdapterTest() ); startupLoggingConfig = new StartupLoggingConfig(); taskConfig = new TaskConfigBuilder().setBaseDir("src/test/resources").build(); + taskLogs = new NoopTaskLogs(); } @Test @@ -199,7 +210,7 @@ public void serializingAndDeserializingATask() throws IOException } @Test - public void toTask_dontSetTaskJSON() throws IOException + public void fromTask_dontSetTaskJSON() throws IOException { final PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); TestKubernetesClient testClient = new TestKubernetesClient(client) @@ -257,6 +268,103 @@ public PodSpec getSpec() ); } + @Test + public void toTask_useTaskPayloadManager() throws IOException + { + TestKubernetesClient testClient = new TestKubernetesClient(client); + KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() + .withNamespace("test") + .build(); + Task taskInTaskPayloadManager = K8sTestUtils.getTask(); + TaskLogs mockTestLogs = Mockito.mock(TaskLogs.class); + Mockito.when(mockTestLogs.streamTaskPayload("ID")).thenReturn(com.google.common.base.Optional.of( + new ByteArrayInputStream(jsonMapper.writeValueAsString(taskInTaskPayloadManager).getBytes(Charset.defaultCharset())) + )); + K8sTaskAdapter adapter = new SingleContainerTaskAdapter( + testClient, + config, + new TaskConfigBuilder().setEnableTaskPayloadManagerPerTask(true).build(), + startupLoggingConfig, + node, + jsonMapper, + mockTestLogs + ); + + Job job = new JobBuilder() + .editSpec().editTemplate().editMetadata() + .addToAnnotations(DruidK8sConstants.TASK_ID,"ID") + .endMetadata().endTemplate().endSpec().build(); + + Task taskFromJob = adapter.toTask(job); + assertEquals(taskInTaskPayloadManager, taskFromJob); + } + + @Test + public void getTaskId() throws IOException + { + TestKubernetesClient testClient = new TestKubernetesClient(client); + KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); + K8sTaskAdapter adapter = new SingleContainerTaskAdapter( + testClient, + config, + taskConfig, + startupLoggingConfig, + node, + jsonMapper, + taskLogs + ); + Job job = new JobBuilder() + .editSpec().editTemplate().editMetadata() + .addToAnnotations(DruidK8sConstants.TASK_ID,"ID") + .endMetadata().endTemplate().endSpec().build(); + + assertEquals(new K8sTaskId("ID"), adapter.getTaskId(job)); + } + + @Test + public void getTaskId_noAnnotations() + { + TestKubernetesClient testClient = new TestKubernetesClient(client); + KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); + K8sTaskAdapter adapter = new SingleContainerTaskAdapter( + testClient, + config, + taskConfig, + startupLoggingConfig, + node, + jsonMapper, + taskLogs + ); + Job job = new JobBuilder() + .editSpec().editTemplate().editMetadata() + .endMetadata().endTemplate().endSpec() + .editMetadata().withName("job").endMetadata().build(); + + Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + } + + @Test + public void getTaskId_missingTaskIdAnnotation() + { + TestKubernetesClient testClient = new TestKubernetesClient(client); + KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); + K8sTaskAdapter adapter = new SingleContainerTaskAdapter( + testClient, + config, + taskConfig, + startupLoggingConfig, + node, + jsonMapper, + taskLogs + ); + Job job = new JobBuilder() + .editSpec().editTemplate().editMetadata() + .addToAnnotations(DruidK8sConstants.TASK_GROUP_ID,"ID") + .endMetadata().endTemplate().endSpec() + .editMetadata().withName("job").endMetadata().build(); + + Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + } @Test void testGrabbingTheLastXmxValueFromACommand() { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index b800a407fcc7..787a2a8a22f7 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -20,6 +20,7 @@ package org.apache.druid.k8s.overlord.taskadapter; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; import io.fabric8.kubernetes.api.model.PodTemplate; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; @@ -34,7 +35,9 @@ import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.Base64Compression; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; +import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; +import org.apache.druid.k8s.overlord.common.TestKubernetesClient; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; @@ -44,8 +47,11 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mockito; +import java.io.ByteArrayInputStream; import java.io.IOException; +import java.nio.charset.Charset; import java.nio.file.Files; import java.nio.file.Path; import java.util.Collections; @@ -53,6 +59,8 @@ import java.util.Properties; import java.util.stream.Collectors; +import static org.junit.jupiter.api.Assertions.assertEquals; + public class PodTemplateTaskAdapterTest { @TempDir private Path tempDir; @@ -313,7 +321,80 @@ public void test_fromTask_withoutAnnotations_throwsIOE() throws IOException } @Test - public void test_fromTask_withoutTaskAnnotation_throwsIOE() throws IOException + public void test_getTaskId() throws IOException + { + Path templatePath = Files.createFile(tempDir.resolve("base.yaml")); + mapper.writeValue(templatePath.toFile(), podTemplateSpec); + + Properties props = new Properties(); + props.setProperty("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString()); + PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( + taskRunnerConfig, + taskConfig, + node, + mapper, + props, + taskLogs + ); + Job job = new JobBuilder() + .editSpec().editTemplate().editMetadata() + .addToAnnotations(DruidK8sConstants.TASK_ID,"ID") + .endMetadata().endTemplate().endSpec().build(); + + assertEquals(new K8sTaskId("ID"), adapter.getTaskId(job)); + } + + @Test + public void test_getTaskId_noAnnotations() throws IOException + { + Path templatePath = Files.createFile(tempDir.resolve("base.yaml")); + mapper.writeValue(templatePath.toFile(), podTemplateSpec); + + Properties props = new Properties(); + props.setProperty("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString()); + PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( + taskRunnerConfig, + taskConfig, + node, + mapper, + props, + taskLogs + ); + Job job = new JobBuilder() + .editSpec().editTemplate().editMetadata() + .endMetadata().endTemplate().endSpec() + .editMetadata().withName("job").endMetadata().build(); + + Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + } + + @Test + public void test_getTaskId_missingTaskIdAnnotation() throws IOException + { + Path templatePath = Files.createFile(tempDir.resolve("base.yaml")); + mapper.writeValue(templatePath.toFile(), podTemplateSpec); + + Properties props = new Properties(); + props.setProperty("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString()); + PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( + taskRunnerConfig, + taskConfig, + node, + mapper, + props, + taskLogs + ); + Job job = new JobBuilder() + .editSpec().editTemplate().editMetadata() + .addToAnnotations(DruidK8sConstants.TASK_GROUP_ID,"ID") + .endMetadata().endTemplate().endSpec() + .editMetadata().withName("job").endMetadata().build(); + + Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + } + + @Test + public void test_toTask_withoutTaskAnnotation_throwsIOE() throws IOException { Path templatePath = Files.createFile(tempDir.resolve("base.yaml")); mapper.writeValue(templatePath.toFile(), podTemplateSpec); @@ -345,7 +426,7 @@ public void test_fromTask_withoutTaskAnnotation_throwsIOE() throws IOException } @Test - public void test_fromTask() throws IOException + public void test_toTask() throws IOException { Path templatePath = Files.createFile(tempDir.resolve("base.yaml")); mapper.writeValue(templatePath.toFile(), podTemplateSpec); @@ -369,6 +450,35 @@ public void test_fromTask() throws IOException Assertions.assertEquals(expected, actual); } + @Test + public void test_toTask_useTaskPayloadManager() throws IOException + { + Path templatePath = Files.createFile(tempDir.resolve("base.yaml")); + mapper.writeValue(templatePath.toFile(), podTemplateSpec); + + Properties props = new Properties(); + props.put("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString()); + + Task expected = NoopTask.create("id", 1); + TaskLogs mockTestLogs = Mockito.mock(TaskLogs.class); + Mockito.when(mockTestLogs.streamTaskPayload("id")).thenReturn(com.google.common.base.Optional.of( + new ByteArrayInputStream(mapper.writeValueAsString(expected).getBytes(Charset.defaultCharset())) + )); + + PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( + taskRunnerConfig, + new TaskConfigBuilder().setEnableTaskPayloadManagerPerTask(true).build(), + node, + mapper, + props, + mockTestLogs + ); + + Job job = K8sTestUtils.fileToResource("expectedNoopJob.yaml", Job.class); + Task actual = adapter.toTask(job); + Assertions.assertEquals(expected, actual); + } + @Test public void test_fromTask_withRealIds() throws IOException { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml index 7c048ea6df48..1c1a5e2a47a8 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml @@ -42,11 +42,11 @@ spec: value: "/tmp" - name: "TASK_ID" value: "id" + - name: "LOAD_BROADCAST_SEGMENTS" + value: "false" - name: "TASK_JSON" valueFrom: fieldRef: fieldPath: "metadata.annotations['task']" - - name: "LOAD_BROADCAST_SEGMENTS" - value: "false" image: one name: primary diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml index b781070892c9..167c2ed68a4f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml @@ -42,11 +42,11 @@ spec: value: "/tmp" - name: "TASK_ID" value: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z" + - name: "LOAD_BROADCAST_SEGMENTS" + value: "false" - name: "TASK_JSON" valueFrom: fieldRef: fieldPath: "metadata.annotations['task']" - - name: "LOAD_BROADCAST_SEGMENTS" - value: "false" image: one name: primary diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml index 5dece8cad443..d72d0ef37b03 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml @@ -41,5 +41,7 @@ spec: value: "/tmp" - name: "TASK_ID" value: "id" + - name: "LOAD_BROADCAST_SEGMENTS" + value: "false" image: one name: primary diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml index 0eb8f8b02f08..57202eea3da6 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml @@ -42,11 +42,11 @@ spec: value: "/tmp" - name: "TASK_ID" value: "id" + - name: "LOAD_BROADCAST_SEGMENTS" + value: "false" - name: "TASK_JSON" valueFrom: fieldRef: fieldPath: "metadata.annotations['task']" - - name: "LOAD_BROADCAST_SEGMENTS" - value: "false" image: one name: primary From ea8a37b5428d60d080a7fc3342bbee4d5129f523 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 21 Aug 2023 10:36:47 -0400 Subject: [PATCH 08/27] fix checkstyle --- .../druid/k8s/overlord/KubernetesPeonLifecycleTest.java | 4 ++-- .../k8s/overlord/taskadapter/K8sTaskAdapterTest.java | 9 +++------ .../overlord/taskadapter/PodTemplateTaskAdapterTest.java | 6 ++---- 3 files changed, 7 insertions(+), 12 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index a6a6bcffefe2..36a77f1a740e 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -47,7 +47,6 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Mockito; import java.io.File; import java.io.IOException; @@ -130,7 +129,8 @@ protected synchronized TaskStatus join(long timeout) } @Test - public void test_run_useTaskManager() throws IOException { + public void test_run_useTaskManager() throws IOException + { TaskConfig taskConfigTaskManager = new TaskConfigBuilder().setEnableTaskPayloadManagerPerTask(true).build(); KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle( task, diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 285c7bae7d28..ed570f987aee 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -64,12 +64,9 @@ import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.tasklogs.TaskLogs; -import org.easymock.EasyMockRunner; -import org.easymock.EasyMockSupport; import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.runner.RunWith; import org.mockito.Mockito; import java.io.ByteArrayInputStream; @@ -292,7 +289,7 @@ public void toTask_useTaskPayloadManager() throws IOException Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() - .addToAnnotations(DruidK8sConstants.TASK_ID,"ID") + .addToAnnotations(DruidK8sConstants.TASK_ID, "ID") .endMetadata().endTemplate().endSpec().build(); Task taskFromJob = adapter.toTask(job); @@ -315,7 +312,7 @@ public void getTaskId() throws IOException ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() - .addToAnnotations(DruidK8sConstants.TASK_ID,"ID") + .addToAnnotations(DruidK8sConstants.TASK_ID, "ID") .endMetadata().endTemplate().endSpec().build(); assertEquals(new K8sTaskId("ID"), adapter.getTaskId(job)); @@ -359,7 +356,7 @@ public void getTaskId_missingTaskIdAnnotation() ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() - .addToAnnotations(DruidK8sConstants.TASK_GROUP_ID,"ID") + .addToAnnotations(DruidK8sConstants.TASK_GROUP_ID, "ID") .endMetadata().endTemplate().endSpec() .editMetadata().withName("job").endMetadata().build(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 787a2a8a22f7..39695223a04b 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -20,7 +20,6 @@ package org.apache.druid.k8s.overlord.taskadapter; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Optional; import io.fabric8.kubernetes.api.model.PodTemplate; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; @@ -37,7 +36,6 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; -import org.apache.druid.k8s.overlord.common.TestKubernetesClient; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; @@ -338,7 +336,7 @@ public void test_getTaskId() throws IOException ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() - .addToAnnotations(DruidK8sConstants.TASK_ID,"ID") + .addToAnnotations(DruidK8sConstants.TASK_ID, "ID") .endMetadata().endTemplate().endSpec().build(); assertEquals(new K8sTaskId("ID"), adapter.getTaskId(job)); @@ -386,7 +384,7 @@ public void test_getTaskId_missingTaskIdAnnotation() throws IOException ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() - .addToAnnotations(DruidK8sConstants.TASK_GROUP_ID,"ID") + .addToAnnotations(DruidK8sConstants.TASK_GROUP_ID, "ID") .endMetadata().endTemplate().endSpec() .editMetadata().withName("job").endMetadata().build(); From 39c62ac382aaa09a8ea7369d12871a26dff25880 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 21 Aug 2023 11:10:44 -0400 Subject: [PATCH 09/27] Add more unit tests --- .../druid/storage/s3/S3TaskLogsTest.java | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java index 7428081aa3ed..011dc4888456 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java @@ -35,8 +35,11 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.commons.io.IOUtils; import org.apache.druid.common.utils.CurrentTimeMillisSupplier; import org.apache.druid.java.util.common.StringUtils; +import org.easymock.Capture; +import org.easymock.CaptureType; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; import org.easymock.EasyMockSupport; @@ -55,6 +58,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.net.URI; +import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.stream.Collectors; @@ -143,6 +147,78 @@ public void test_pushTaskStatus() throws IOException EasyMock.verify(s3Client); } + @Test + public void test_pushTaskPayload() throws IOException + { + Capture putObjectRequestCapture = Capture.newInstance(CaptureType.FIRST); + EasyMock.expect(s3Client.putObject(EasyMock.capture(putObjectRequestCapture))) + .andReturn(new PutObjectResult()) + .once(); + + EasyMock.replay(s3Client); + + S3TaskLogsConfig config = new S3TaskLogsConfig(); + config.setS3Bucket(TEST_BUCKET); + config.setS3Prefix("prefix"); + config.setDisableAcl(true); + + CurrentTimeMillisSupplier timeSupplier = new CurrentTimeMillisSupplier(); + S3InputDataConfig inputDataConfig = new S3InputDataConfig(); + S3TaskLogs s3TaskLogs = new S3TaskLogs(s3Client, config, inputDataConfig, timeSupplier); + + File payloadFile = tempFolder.newFile("task.json"); + String taskId = "index_test-datasource_2019-06-18T13:30:28.887Z"; + s3TaskLogs.pushTaskPayload(taskId, payloadFile); + + PutObjectRequest putObjectRequest = putObjectRequestCapture.getValue(); + Assert.assertEquals(TEST_BUCKET, putObjectRequest.getBucketName()); + Assert.assertEquals("prefix/" + taskId + "/task.json", putObjectRequest.getKey()); + Assert.assertEquals(payloadFile, putObjectRequest.getFile()); + EasyMock.verify(s3Client); + } + + @Test + public void test_streamTaskPayload() throws IOException + { + String taskPayloadString = "task payload"; + + ObjectMetadata objectMetadata = new ObjectMetadata(); + objectMetadata.setContentLength(taskPayloadString.length()); + EasyMock.expect(s3Client.getObjectMetadata(EasyMock.anyObject(), EasyMock.anyObject())) + .andReturn(objectMetadata) + .once(); + + InputStream taskPayload = new ByteArrayInputStream(taskPayloadString.getBytes(Charset.defaultCharset())); + S3Object s3Object = new S3Object(); + s3Object.setObjectContent(taskPayload); + Capture getObjectRequestCapture = Capture.newInstance(CaptureType.FIRST); + EasyMock.expect(s3Client.getObject(EasyMock.capture(getObjectRequestCapture))) + .andReturn(s3Object) + .once(); + + EasyMock.replay(s3Client); + + S3TaskLogsConfig config = new S3TaskLogsConfig(); + config.setS3Bucket(TEST_BUCKET); + config.setS3Prefix("prefix"); + config.setDisableAcl(true); + + CurrentTimeMillisSupplier timeSupplier = new CurrentTimeMillisSupplier(); + S3InputDataConfig inputDataConfig = new S3InputDataConfig(); + S3TaskLogs s3TaskLogs = new S3TaskLogs(s3Client, config, inputDataConfig, timeSupplier); + + String taskId = "index_test-datasource_2019-06-18T13:30:28.887Z"; + Optional payloadResponse = s3TaskLogs.streamTaskPayload(taskId); + + GetObjectRequest getObjectRequest = getObjectRequestCapture.getValue(); + Assert.assertEquals(TEST_BUCKET, getObjectRequest.getBucketName()); + Assert.assertEquals("prefix/" + taskId + "/task.json", getObjectRequest.getKey()); + Assert.assertTrue(payloadResponse.isPresent()); + + Assert.assertEquals(taskPayloadString, IOUtils.toString(payloadResponse.get(), Charset.defaultCharset())); + EasyMock.verify(s3Client); + } + @Test public void test_killAll_noException_deletesAllTaskLogs() throws IOException { From ca17d62e006b0fedfc14178dad4ebed1a6b96f6f Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 21 Aug 2023 15:21:44 -0400 Subject: [PATCH 10/27] Clean up check failures --- .../overlord/taskadapter/PodTemplateTaskAdapterTest.java | 3 ++- .../java/org/apache/druid/tasklogs/NoopTaskLogsTest.java | 7 +++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 39695223a04b..a9d2ea1fa7f5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -20,6 +20,7 @@ package org.apache.druid.k8s.overlord.taskadapter; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; import io.fabric8.kubernetes.api.model.PodTemplate; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; @@ -459,7 +460,7 @@ public void test_toTask_useTaskPayloadManager() throws IOException Task expected = NoopTask.create("id", 1); TaskLogs mockTestLogs = Mockito.mock(TaskLogs.class); - Mockito.when(mockTestLogs.streamTaskPayload("id")).thenReturn(com.google.common.base.Optional.of( + Mockito.when(mockTestLogs.streamTaskPayload("id")).thenReturn(Optional.of( new ByteArrayInputStream(mapper.writeValueAsString(expected).getBytes(Charset.defaultCharset())) )); diff --git a/processing/src/test/java/org/apache/druid/tasklogs/NoopTaskLogsTest.java b/processing/src/test/java/org/apache/druid/tasklogs/NoopTaskLogsTest.java index 30192932a9e0..2da98dab3038 100644 --- a/processing/src/test/java/org/apache/druid/tasklogs/NoopTaskLogsTest.java +++ b/processing/src/test/java/org/apache/druid/tasklogs/NoopTaskLogsTest.java @@ -32,4 +32,11 @@ public void test_streamTaskStatus() throws IOException TaskLogs taskLogs = new NoopTaskLogs(); Assert.assertFalse(taskLogs.streamTaskStatus("id").isPresent()); } + + @Test + public void test_streamTaskPayload() throws IOException + { + TaskLogs taskLogs = new NoopTaskLogs(); + Assert.assertFalse(taskLogs.streamTaskPayload("id").isPresent()); + } } From bf218545bb3aa00521b0ab587135084003b1a88d Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 23 Aug 2023 13:54:44 -0400 Subject: [PATCH 11/27] PR changes --- .../extensions-contrib/k8s-jobs.md | 36 ++++++++-------- .../druid/tasklogs/TaskPayloadManager.java | 20 ++++++++- .../tasklogs/TaskPayloadManagerTest.java | 43 +++++++++++++++++++ 3 files changed, 80 insertions(+), 19 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/tasklogs/TaskPayloadManagerTest.java diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 6917fbd87f6b..31d5eedd19aa 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -216,23 +216,25 @@ data: ``` ### Properties -|Property| Possible Values | Description |Default|required| -|--------|-----------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------|--------| -|`druid.indexer.runner.debugJobs`| `boolean` | Clean up K8s jobs after tasks complete. |False|No| -|`druid.indexer.runner.sidecarSupport`| `boolean` | Deprecated, specify adapter type as runtime property `druid.indexer.runner.k8s.adapter.type: overlordMultiContainer` instead. If your overlord pod has sidecars, this will attempt to start the task with the same sidecars as the overlord pod. |False|No| -|`druid.indexer.runner.primaryContainerName`| `String` | If running with sidecars, the `primaryContainerName` should be that of your druid container like `druid-overlord`. |First container in `podSpec` list|No| -|`druid.indexer.runner.kubexitImage`| `String` | Used kubexit project to help shutdown sidecars when the main pod completes. Otherwise jobs with sidecars never terminate. |karlkfi/kubexit:latest|No| -|`druid.indexer.runner.disableClientProxy`| `boolean` | Use this if you have a global http(s) proxy and you wish to bypass it. |false|No| -|`druid.indexer.runner.maxTaskDuration`| `Duration` | Max time a task is allowed to run for before getting killed |`PT4H`|No| -|`druid.indexer.runner.taskCleanupDelay`| `Duration` | How long do jobs stay around before getting reaped from K8s |`P2D`|No| -|`druid.indexer.runner.taskCleanupInterval`| `Duration` | How often to check for jobs to be reaped |`PT10M`|No| -|`druid.indexer.runner.K8sjobLaunchTimeout`| `Duration` | How long to wait to launch a K8s task before marking it as failed, on a resource constrained cluster it may take some time. |`PT1H`|No| -|`druid.indexer.runner.javaOptsArray`| `JsonArray` | java opts for the task. |`-Xmx1g`|No| -|`druid.indexer.runner.labels`| `JsonObject` | Additional labels you want to add to peon pod |`{}`|No| -|`druid.indexer.runner.annotations`| `JsonObject` | Additional annotations you want to add to peon pod |`{}`|No| -|`druid.indexer.runner.peonMonitors`| `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. |`[]`|No| -|`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. |`PT30S` (K8s default)|No| -|`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. |`2147483647`|No| +|Property| Possible Values | Description | Default |required| +|--------|-----------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------------------------|--------| +|`druid.indexer.runner.debugJobs`| `boolean` | Clean up K8s jobs after tasks complete. | False |No| +|`druid.indexer.runner.sidecarSupport`| `boolean` | Deprecated, specify adapter type as runtime property `druid.indexer.runner.k8s.adapter.type: overlordMultiContainer` instead. If your overlord pod has sidecars, this will attempt to start the task with the same sidecars as the overlord pod. | False |No| +|`druid.indexer.runner.primaryContainerName`| `String` | If running with sidecars, the `primaryContainerName` should be that of your druid container like `druid-overlord`. | First container in `podSpec` list |No| +|`druid.indexer.runner.kubexitImage`| `String` | Used kubexit project to help shutdown sidecars when the main pod completes. Otherwise jobs with sidecars never terminate. | karlkfi/kubexit:latest |No| +|`druid.indexer.runner.disableClientProxy`| `boolean` | Use this if you have a global http(s) proxy and you wish to bypass it. | false |No| +|`druid.indexer.runner.maxTaskDuration`| `Duration` | Max time a task is allowed to run for before getting killed | `PT4H` |No| +|`druid.indexer.runner.taskCleanupDelay`| `Duration` | How long do jobs stay around before getting reaped from K8s | `P2D` |No| +|`druid.indexer.runner.taskCleanupInterval`| `Duration` | How often to check for jobs to be reaped | `PT10M` |No| +|`druid.indexer.runner.K8sjobLaunchTimeout`| `Duration` | How long to wait to launch a K8s task before marking it as failed, on a resource constrained cluster it may take some time. | `PT1H` |No| +|`druid.indexer.runner.javaOptsArray`| `JsonArray` | java opts for the task. | `-Xmx1g` |No| +|`druid.indexer.runner.labels`| `JsonObject` | Additional labels you want to add to peon pod | `{}` |No| +|`druid.indexer.runner.annotations`| `JsonObject` | Additional annotations you want to add to peon pod | `{}` |No| +|`druid.indexer.runner.peonMonitors`| `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. | `[]` |No| +|`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. | `PT30S` (K8s default) |No| +|`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. | `2147483647` |No| +|`druid.indexer.runner.taskPayloadAsEnvVariable`| `boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task paylods > ~1MB | `true` |No| +|`druid.indexer.task.enableTaskPayloadManagerPerTask `| `boolean` | Whether druid.indexer.logs.type should be used as a passthrough for task payloads. Set this to true if you plan on using task paylods > ~1MB | `false` |No| ### Metrics added diff --git a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java index 881139828ca1..66ebb70a38bd 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java @@ -21,20 +21,36 @@ import com.google.common.base.Optional; import org.apache.commons.lang.NotImplementedException; +import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.java.util.common.StringUtils; import java.io.File; import java.io.IOException; import java.io.InputStream; +/** + * Something that knows how to manage task json paylods. + */ +@ExtensionPoint public interface TaskPayloadManager { + /** + * Save payload so it can be retrieved later. + * + * @return inputStream for this taskPayload, if available + */ default void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { - throw new NotImplementedException("Managing task payloads is not implemented for this druid.indexer.logs.type"); + throw new NotImplementedException(StringUtils.format("this druid.indexer.logs.type [%s] does not support managing task payloads yet. You will have to switch to using environment variables", getClass())); } + /** + * Stream payload for a task. + * + * @return inputStream for this taskPayload, if available + */ default Optional streamTaskPayload(String taskid) throws IOException { - throw new NotImplementedException("Managing task payloads is not implemented for this druid.indexer.task.logs"); + throw new NotImplementedException(StringUtils.format("this druid.indexer.logs.type [%s] does not support managing task payloads yet. You will have to switch to using environment variables", getClass())); } } diff --git a/processing/src/test/java/org/apache/druid/tasklogs/TaskPayloadManagerTest.java b/processing/src/test/java/org/apache/druid/tasklogs/TaskPayloadManagerTest.java new file mode 100644 index 000000000000..4c53061a86f2 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/tasklogs/TaskPayloadManagerTest.java @@ -0,0 +1,43 @@ +/* + * 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.tasklogs; + +import org.apache.commons.lang.NotImplementedException; +import org.junit.Assert; +import org.junit.Test; + +public class TaskPayloadManagerTest implements TaskPayloadManager +{ + @Test + public void test_streamTaskPayload() + { + Assert.assertThrows(NotImplementedException.class, + () -> this.streamTaskPayload("id") + ); + } + + @Test + public void test_pushTaskPayload() + { + Assert.assertThrows(NotImplementedException.class, + () -> this.pushTaskPayload("id", null) + ); + } +} From be50e4510b03547a44b24a04d2dd3a92ae1ea096 Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 23 Aug 2023 14:49:55 -0400 Subject: [PATCH 12/27] Fix spellign errors --- docs/development/extensions-contrib/k8s-jobs.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 31d5eedd19aa..8fd2af453a79 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -233,8 +233,8 @@ data: |`druid.indexer.runner.peonMonitors`| `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. | `[]` |No| |`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. | `PT30S` (K8s default) |No| |`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. | `2147483647` |No| -|`druid.indexer.runner.taskPayloadAsEnvVariable`| `boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task paylods > ~1MB | `true` |No| -|`druid.indexer.task.enableTaskPayloadManagerPerTask `| `boolean` | Whether druid.indexer.logs.type should be used as a passthrough for task payloads. Set this to true if you plan on using task paylods > ~1MB | `false` |No| +|`druid.indexer.runner.taskPayloadAsEnvVariable`| `boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task payloads > 1 MB | `true` |No| +|`druid.indexer.task.enableTaskPayloadManagerPerTask `| `boolean` | Whether druid.indexer.logs.type should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB | `false` |No| ### Metrics added From 3a59f3f91b99d362a47bd71af4a1c6f9b4a9a092 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 28 Aug 2023 16:11:44 -0700 Subject: [PATCH 13/27] Fix spacing in docs --- .../extensions-contrib/k8s-jobs.md | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 8fd2af453a79..bcda82721714 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -216,23 +216,23 @@ data: ``` ### Properties -|Property| Possible Values | Description | Default |required| -|--------|-----------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------------------------|--------| -|`druid.indexer.runner.debugJobs`| `boolean` | Clean up K8s jobs after tasks complete. | False |No| -|`druid.indexer.runner.sidecarSupport`| `boolean` | Deprecated, specify adapter type as runtime property `druid.indexer.runner.k8s.adapter.type: overlordMultiContainer` instead. If your overlord pod has sidecars, this will attempt to start the task with the same sidecars as the overlord pod. | False |No| -|`druid.indexer.runner.primaryContainerName`| `String` | If running with sidecars, the `primaryContainerName` should be that of your druid container like `druid-overlord`. | First container in `podSpec` list |No| -|`druid.indexer.runner.kubexitImage`| `String` | Used kubexit project to help shutdown sidecars when the main pod completes. Otherwise jobs with sidecars never terminate. | karlkfi/kubexit:latest |No| -|`druid.indexer.runner.disableClientProxy`| `boolean` | Use this if you have a global http(s) proxy and you wish to bypass it. | false |No| -|`druid.indexer.runner.maxTaskDuration`| `Duration` | Max time a task is allowed to run for before getting killed | `PT4H` |No| -|`druid.indexer.runner.taskCleanupDelay`| `Duration` | How long do jobs stay around before getting reaped from K8s | `P2D` |No| -|`druid.indexer.runner.taskCleanupInterval`| `Duration` | How often to check for jobs to be reaped | `PT10M` |No| -|`druid.indexer.runner.K8sjobLaunchTimeout`| `Duration` | How long to wait to launch a K8s task before marking it as failed, on a resource constrained cluster it may take some time. | `PT1H` |No| -|`druid.indexer.runner.javaOptsArray`| `JsonArray` | java opts for the task. | `-Xmx1g` |No| -|`druid.indexer.runner.labels`| `JsonObject` | Additional labels you want to add to peon pod | `{}` |No| -|`druid.indexer.runner.annotations`| `JsonObject` | Additional annotations you want to add to peon pod | `{}` |No| -|`druid.indexer.runner.peonMonitors`| `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. | `[]` |No| -|`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. | `PT30S` (K8s default) |No| -|`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. | `2147483647` |No| +|Property| Possible Values | Description |Default|required| +|--------|-----------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------|--------| +|`druid.indexer.runner.debugJobs`| `boolean` | Clean up K8s jobs after tasks complete. |False|No| +|`druid.indexer.runner.sidecarSupport`| `boolean` | Deprecated, specify adapter type as runtime property `druid.indexer.runner.k8s.adapter.type: overlordMultiContainer` instead. If your overlord pod has sidecars, this will attempt to start the task with the same sidecars as the overlord pod. |False|No| +|`druid.indexer.runner.primaryContainerName`| `String` | If running with sidecars, the `primaryContainerName` should be that of your druid container like `druid-overlord`. |First container in `podSpec` list|No| +|`druid.indexer.runner.kubexitImage`| `String` | Used kubexit project to help shutdown sidecars when the main pod completes. Otherwise jobs with sidecars never terminate. |karlkfi/kubexit:latest|No| +|`druid.indexer.runner.disableClientProxy`| `boolean` | Use this if you have a global http(s) proxy and you wish to bypass it. |false|No| +|`druid.indexer.runner.maxTaskDuration`| `Duration` | Max time a task is allowed to run for before getting killed |`PT4H`|No| +|`druid.indexer.runner.taskCleanupDelay`| `Duration` | How long do jobs stay around before getting reaped from K8s |`P2D`|No| +|`druid.indexer.runner.taskCleanupInterval`| `Duration` | How often to check for jobs to be reaped |`PT10M`|No| +|`druid.indexer.runner.K8sjobLaunchTimeout`| `Duration` | How long to wait to launch a K8s task before marking it as failed, on a resource constrained cluster it may take some time. |`PT1H`|No| +|`druid.indexer.runner.javaOptsArray`| `JsonArray` | java opts for the task. |`-Xmx1g`|No| +|`druid.indexer.runner.labels`| `JsonObject` | Additional labels you want to add to peon pod |`{}`|No| +|`druid.indexer.runner.annotations`| `JsonObject` | Additional annotations you want to add to peon pod |`{}`|No| +|`druid.indexer.runner.peonMonitors`| `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. |`[]`|No| +|`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. |`PT30S` (K8s default)|No| +|`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. |`2147483647`|No| |`druid.indexer.runner.taskPayloadAsEnvVariable`| `boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task payloads > 1 MB | `true` |No| |`druid.indexer.task.enableTaskPayloadManagerPerTask `| `boolean` | Whether druid.indexer.logs.type should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB | `false` |No| From 3bc19de1e0cf4a302383536d5fe14103a336d9fe Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 28 Aug 2023 16:17:53 -0700 Subject: [PATCH 14/27] Don't overwrite table format --- docs/development/extensions-contrib/k8s-jobs.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index bcda82721714..2d5201fb7281 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -233,9 +233,8 @@ data: |`druid.indexer.runner.peonMonitors`| `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. |`[]`|No| |`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. |`PT30S` (K8s default)|No| |`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. |`2147483647`|No| -|`druid.indexer.runner.taskPayloadAsEnvVariable`| `boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task payloads > 1 MB | `true` |No| -|`druid.indexer.task.enableTaskPayloadManagerPerTask `| `boolean` | Whether druid.indexer.logs.type should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB | `false` |No| - +|`druid.indexer.runner.taskPayloadAsEnvVariable`| `Boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task payloads > 1 MB. |`true`|No| +|`druid.indexer.task.enableTaskPayloadManagerPerTask`| `Boolean` | Whether druid.indexer.logs.type should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB MB. |`false`|No| ### Metrics added |Metric|Description|Dimensions|Normal value| From 4b05f39703dc6968445b0a3486c729e6bd754dc5 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 28 Aug 2023 16:25:56 -0700 Subject: [PATCH 15/27] don't fix table format --- docs/development/extensions-contrib/k8s-jobs.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 2d5201fb7281..f7fcf4682a44 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -235,6 +235,7 @@ data: |`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. |`2147483647`|No| |`druid.indexer.runner.taskPayloadAsEnvVariable`| `Boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task payloads > 1 MB. |`true`|No| |`druid.indexer.task.enableTaskPayloadManagerPerTask`| `Boolean` | Whether druid.indexer.logs.type should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB MB. |`false`|No| + ### Metrics added |Metric|Description|Dimensions|Normal value| From ffcf0a76ae9cbc4c61657064181f64f8462b2430 Mon Sep 17 00:00:00 2001 From: George Wu Date: Wed, 13 Sep 2023 13:20:52 -0400 Subject: [PATCH 16/27] Rename config --- docs/development/extensions-contrib/k8s-jobs.md | 4 ++-- .../apache/druid/k8s/overlord/KubernetesPeonLifecycle.java | 2 +- .../apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java | 4 ++-- .../k8s/overlord/taskadapter/PodTemplateTaskAdapter.java | 4 ++-- services/src/main/java/org/apache/druid/cli/CliPeon.java | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index f7fcf4682a44..03c9c3dbafc1 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -234,7 +234,7 @@ data: |`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. |`PT30S` (K8s default)|No| |`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. |`2147483647`|No| |`druid.indexer.runner.taskPayloadAsEnvVariable`| `Boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task payloads > 1 MB. |`true`|No| -|`druid.indexer.task.enableTaskPayloadManagerPerTask`| `Boolean` | Whether druid.indexer.logs.type should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB MB. |`false`|No| +|`druid.indexer.task.useDeepStorageForTaskPayload`| `Boolean` | Whether druid.indexer.logs.type(Task deep storage) should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB. |`false`|No| ### Metrics added @@ -274,4 +274,4 @@ roleRef: kind: Role name: druid-k8s-task-scheduler apiGroup: rbac.authorization.k8s.io -``` \ No newline at end of file +``` diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index abcfa4c81d48..353ce7ad4df2 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -128,7 +128,7 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); - if (taskConfig.isEnableTaskPayloadManagerPerTask()) { + if (taskConfig.isUseDeepStorageForTaskPayload()) { writeTaskPayload(task); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index 93b794fd0023..d37cfc355b83 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -134,10 +134,10 @@ public Job fromTask(Task task) throws IOException @Override public Task toTask(Job from) throws IOException { - if (taskConfig.isEnableTaskPayloadManagerPerTask()) { + if (taskConfig.isUseDeepStorageForTaskPayload()) { com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); if (!taskBody.isPresent()) { - throw new IOE("Could not find task payload in task logs for job [%s]", from.getMetadata().getName()); + throw new IOE("Could not find task payload in deep storage for job [%s]", from.getMetadata().getName()); } String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); return mapper.readValue(task, Task.class); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 1a69ea3b18b5..c6ee74d4f90f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -169,10 +169,10 @@ public Job fromTask(Task task) throws IOException @Override public Task toTask(Job from) throws IOException { - if (taskConfig.isEnableTaskPayloadManagerPerTask()) { + if (taskConfig.isUseDeepStorageForTaskPayload()) { com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); if (!taskBody.isPresent()) { - throw new IOE("Could not find task payload in task logs for job [%s]", from.getMetadata().getName()); + throw new IOE("Could not find task payload in deep storage for job [%s]", from.getMetadata().getName()); } String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); return mapper.readValue(task, Task.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 32c748383c92..eb9345bee893 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -293,7 +293,7 @@ public Supplier> heartbeatDimensions(Task task) public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager, TaskConfig taskConfig) { try { - if (taskConfig.isEnableTaskPayloadManagerPerTask()) { + if (taskConfig.isUseDeepStorageForTaskPayload()) { String task = IOUtils.toString(taskPayloadManager.streamTaskPayload(taskId).get(), Charset.defaultCharset()); // write the remote task.json to task file location for ExecutorLifecycle to pickup FileUtils.write(config.getTaskFile(), task, Charset.defaultCharset()); From a74aad7dd1b6cbb2377af179c24a2f0ab2aa24ef Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 15 Sep 2023 13:56:53 -0400 Subject: [PATCH 17/27] Remove config options --- .../extensions-contrib/k8s-jobs.md | 2 - .../k8s/overlord/KubernetesPeonLifecycle.java | 14 +-- .../KubernetesPeonLifecycleFactory.java | 9 +- .../k8s/overlord/KubernetesTaskRunner.java | 3 +- .../overlord/KubernetesTaskRunnerConfig.java | 27 +----- .../overlord/KubernetesTaskRunnerFactory.java | 2 +- .../overlord/common/DruidK8sConstants.java | 2 + .../overlord/taskadapter/K8sTaskAdapter.java | 39 +++++--- .../taskadapter/PodTemplateTaskAdapter.java | 42 +++++--- .../k8s/overlord/taskadapter/TaskAdapter.java | 5 + .../overlord/KubernetesPeonLifecycleTest.java | 96 +++++++------------ .../KubernetesTaskRunnerConfigTest.java | 4 - .../overlord/KubernetesTaskRunnerTest.java | 8 +- .../k8s/overlord/KubernetesWorkItemTest.java | 8 -- .../taskadapter/K8sTaskAdapterTest.java | 16 +++- .../PodTemplateTaskAdapterTest.java | 10 +- .../indexing/common/config/TaskConfig.java | 23 +---- .../common/config/TaskConfigBuilder.java | 9 +- .../java/org/apache/druid/cli/CliPeon.java | 6 +- 19 files changed, 141 insertions(+), 184 deletions(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 8a7a85bbc69a..2132b55ea1e3 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -233,8 +233,6 @@ data: |`druid.indexer.runner.peonMonitors`| `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. |`[]`|No| |`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. |`PT30S` (K8s default)|No| |`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. |`2147483647`|No| -|`druid.indexer.runner.taskPayloadAsEnvVariable`| `Boolean` | Whether task payloads should be passed to K8s jobs as a compressed environment variable. Set this to false if you plan on using task payloads > 1 MB. |`true`|No| -|`druid.indexer.task.useDeepStorageForTaskPayload`| `Boolean` | Whether druid.indexer.logs.type(Task deep storage) should be used to pass task payloads to the K8s jobs. Set this to true if you plan on using task payloads > 1 MB. |`false`|No| ### Metrics added diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index c248e3e46bd9..d13b1d8c53a4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -30,7 +30,6 @@ import org.apache.commons.io.IOUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -73,9 +72,9 @@ public interface TaskStateListener protected enum State { - /** Lifecycle's state before {@link #run(Job, long, long)} or {@link #join(long)} is called. */ + /** Lifecycle's state before {@link #run(Job, long, long, Boolean)} or {@link #join(long)} is called. */ NOT_STARTED, - /** Lifecycle's state since {@link #run(Job, long, long)} is called. */ + /** Lifecycle's state since {@link #run(Job, long, long, Boolean)} is called. */ PENDING, /** Lifecycle's state since {@link #join(long)} is called. */ RUNNING, @@ -90,7 +89,6 @@ protected enum State private final KubernetesPeonClient kubernetesClient; private final ObjectMapper mapper; private final TaskStateListener stateListener; - private final TaskConfig taskConfig; @MonotonicNonNull private LogWatch logWatch; @@ -101,8 +99,7 @@ protected KubernetesPeonLifecycle( KubernetesPeonClient kubernetesClient, TaskLogs taskLogs, ObjectMapper mapper, - TaskStateListener stateListener, - TaskConfig taskConfig + TaskStateListener stateListener ) { this.taskId = new K8sTaskId(task); @@ -111,7 +108,6 @@ protected KubernetesPeonLifecycle( this.taskLogs = taskLogs; this.mapper = mapper; this.stateListener = stateListener; - this.taskConfig = taskConfig; } /** @@ -123,12 +119,12 @@ protected KubernetesPeonLifecycle( * @return * @throws IllegalStateException */ - protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout) throws IllegalStateException + protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, Boolean useDeepStorageForTaskPayload) throws IllegalStateException { try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); - if (taskConfig.isUseDeepStorageForTaskPayload()) { + if (useDeepStorageForTaskPayload) { writeTaskPayload(task); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java index 0ab40c4698a0..bf4e3a712577 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleFactory.java @@ -20,7 +20,6 @@ package org.apache.druid.k8s.overlord; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.tasklogs.TaskLogs; @@ -30,19 +29,16 @@ public class KubernetesPeonLifecycleFactory implements PeonLifecycleFactory private final KubernetesPeonClient client; private final TaskLogs taskLogs; private final ObjectMapper mapper; - private final TaskConfig taskConfig; public KubernetesPeonLifecycleFactory( KubernetesPeonClient client, TaskLogs taskLogs, - ObjectMapper mapper, - TaskConfig taskConfig + ObjectMapper mapper ) { this.client = client; this.taskLogs = taskLogs; this.mapper = mapper; - this.taskConfig = taskConfig; } @Override @@ -53,8 +49,7 @@ public KubernetesPeonLifecycle build(Task task, KubernetesPeonLifecycle.TaskStat client, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 14fe54778a20..0525264fdb9c 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -193,7 +193,8 @@ protected TaskStatus doTask(Task task, boolean run) taskStatus = peonLifecycle.run( adapter.fromTask(task), config.getTaskLaunchTimeout().toStandardDuration().getMillis(), - config.getTaskTimeout().toStandardDuration().getMillis() + config.getTaskTimeout().toStandardDuration().getMillis(), + adapter.shouldUseDeepStorageForTaskPayload(task) ); } else { taskStatus = peonLifecycle.join( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index 256c6c3e9053..0d67c55b30aa 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -90,11 +90,6 @@ public class KubernetesTaskRunnerConfig // how long to wait for the peon k8s job to launch private Period k8sjobLaunchTimeout = new Period("PT1H"); - // Whether to pass the task.json payload to the peon K8s Jobs as a environment variable. - @JsonProperty - @NotNull - private Boolean taskPayloadAsEnvVariable = Boolean.TRUE; - @JsonProperty // ForkingTaskRunner inherits the monitors from the MM, in k8s mode // the peon inherits the monitors from the overlord, so if someone specifies @@ -140,8 +135,7 @@ private KubernetesTaskRunnerConfig( List javaOptsArray, Map labels, Map annotations, - Integer capacity, - Boolean taskPayloadAsEnvVariable + Integer capacity ) { this.namespace = namespace; @@ -202,10 +196,6 @@ private KubernetesTaskRunnerConfig( capacity, this.capacity ); - this.taskPayloadAsEnvVariable = ObjectUtils.defaultIfNull( - taskPayloadAsEnvVariable, - Boolean.TRUE - ); } public String getNamespace() @@ -289,11 +279,6 @@ public Integer getCapacity() return capacity; } - public Boolean isTaskPayloadAsEnvVariable() - { - return taskPayloadAsEnvVariable; - } - public static Builder builder() { return new Builder(); @@ -317,7 +302,6 @@ public static class Builder private Map labels; private Map annotations; private Integer capacity; - private Boolean taskPayloadAsEnvVariable; public Builder() { @@ -419,12 +403,6 @@ public Builder withCapacity(@Min(0) @Max(Integer.MAX_VALUE) Integer capacity) return this; } - public Builder withTaskPayloadAsEnvVariable(Boolean taskPayloadAsEnvVariable) - { - this.taskPayloadAsEnvVariable = taskPayloadAsEnvVariable; - return this; - } - public KubernetesTaskRunnerConfig build() { return new KubernetesTaskRunnerConfig( @@ -443,8 +421,7 @@ public KubernetesTaskRunnerConfig build() this.javaOptsArray, this.labels, this.annotations, - this.capacity, - this.taskPayloadAsEnvVariable + this.capacity ); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index de9873b1af43..72d7ef0c00d4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -100,7 +100,7 @@ public KubernetesTaskRunner build() kubernetesTaskRunnerConfig, peonClient, httpClient, - new KubernetesPeonLifecycleFactory(peonClient, taskLogs, smileMapper, taskConfig), + new KubernetesPeonLifecycleFactory(peonClient, taskLogs, smileMapper), emitter ); return runner; diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java index a5380bef3f22..0a6fe6e73347 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java @@ -40,5 +40,7 @@ public class DruidK8sConstants public static final String DRUID_HOSTNAME_ENV = "HOSTNAME"; public static final String LABEL_KEY = "druid.k8s.peons"; public static final String DRUID_LABEL_PREFIX = "druid."; + + public static final long MAX_ENV_VARIABLE_KBS = 130048; // 127 KB static final Predicate IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index d37cfc355b83..5a39681ccdbf 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -134,25 +134,28 @@ public Job fromTask(Task task) throws IOException @Override public Task toTask(Job from) throws IOException { - if (taskConfig.isUseDeepStorageForTaskPayload()) { - com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); - if (!taskBody.isPresent()) { - throw new IOE("Could not find task payload in deep storage for job [%s]", from.getMetadata().getName()); - } - String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); - return mapper.readValue(task, Task.class); - } PodSpec podSpec = from.getSpec().getTemplate().getSpec(); massageSpec(podSpec, "main"); List envVars = podSpec.getContainers().get(0).getEnv(); Optional taskJson = envVars.stream().filter(x -> "TASK_JSON".equals(x.getName())).findFirst(); String contents = taskJson.map(envVar -> taskJson.get().getValue()).orElse(null); if (contents == null) { - throw new IOException("No TASK_JSON environment variable found in pod: " + from.getMetadata().getName()); + log.info("No TASK_JSON environment variable found in pod: %s. Trying to load task payload from deep storage.", from.getMetadata().getName()); + return toTaskUsingDeepStorage(from); } return mapper.readValue(Base64Compression.decompressBase64(contents), Task.class); } + private Task toTaskUsingDeepStorage(Job from) throws IOException + { + com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); + if (!taskBody.isPresent()) { + throw new IOE("Could not load task payload for job [%s]", from.getMetadata().getName()); + } + String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); + return mapper.readValue(task, Task.class); + } + @Override public K8sTaskId getTaskId(Job from) throws IOException { @@ -271,11 +274,12 @@ void addEnvironmentVariables(Container mainContainer, PeonCommandContext context "metadata.name" )).build()).build() ); - if (taskRunnerConfig.isTaskPayloadAsEnvVariable()) { + + if (taskContents.length() < DruidK8sConstants.MAX_ENV_VARIABLE_KBS) { envVars.add( new EnvVarBuilder() .withName(DruidK8sConstants.TASK_JSON_ENV) - .withValue(taskRunnerConfig.isTaskPayloadAsEnvVariable() ? taskContents : "") + .withValue(taskContents) .build() ); } @@ -471,5 +475,18 @@ static ResourceRequirements getResourceRequirements(ResourceRequirements require } return requirements; } + + @Override + public Boolean shouldUseDeepStorageForTaskPayload(Task task) + { + try { + String compressedTaskPayload = Base64Compression.compressBase64(mapper.writeValueAsString(task)); + return compressedTaskPayload.length() > DruidK8sConstants.MAX_ENV_VARIABLE_KBS; + } + catch (Exception e) { + // In case there's a issue with checking how large the task is, default to using deep storage so we don't lose the task payload. + return true; + } + } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index c6ee74d4f90f..06436936f7b4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -169,25 +169,29 @@ public Job fromTask(Task task) throws IOException @Override public Task toTask(Job from) throws IOException { - if (taskConfig.isUseDeepStorageForTaskPayload()) { - com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); - if (!taskBody.isPresent()) { - throw new IOE("Could not find task payload in deep storage for job [%s]", from.getMetadata().getName()); - } - String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); - return mapper.readValue(task, Task.class); - } Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); if (annotations == null) { - throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); + log.info("No annotations found on pod spec for job [%s]. Trying to load task payload from deep storage.", from.getMetadata().getName()); + return toTaskUsingDeepStorage(from); } String task = annotations.get(DruidK8sConstants.TASK); if (task == null) { - throw new IOE("No task annotation found on pod spec for job [%s]", from.getMetadata().getName()); + log.info("No task annotation found on pod spec for job [%s]. Trying to load task payload from deep storage.", from.getMetadata().getName()); + return toTaskUsingDeepStorage(from); } return mapper.readValue(Base64Compression.decompressBase64(task), Task.class); } + private Task toTaskUsingDeepStorage(Job from) throws IOException + { + com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); + if (!taskBody.isPresent()) { + throw new IOE("Could not load task payload for job [%s]", from.getMetadata().getName()); + } + String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); + return mapper.readValue(task, Task.class); + } + @Override public K8sTaskId getTaskId(Job from) throws IOException { @@ -254,7 +258,7 @@ private Collection getEnv(Task task) .withValue(Boolean.toString(task.supportsQueries())) .build() ); - if (taskRunnerConfig.isTaskPayloadAsEnvVariable()) { + if (!shouldUseDeepStorageForTaskPayload(task)) { envVars.add(new EnvVarBuilder() .withName(DruidK8sConstants.TASK_JSON_ENV) .withValueFrom(new EnvVarSourceBuilder().withFieldRef(new ObjectFieldSelector( @@ -279,7 +283,8 @@ private Map getPodTemplateAnnotations(Task task) throws IOExcept .put(DruidK8sConstants.TASK_TYPE, task.getType()) .put(DruidK8sConstants.TASK_GROUP_ID, task.getGroupId()) .put(DruidK8sConstants.TASK_DATASOURCE, task.getDataSource()); - if (taskRunnerConfig.isTaskPayloadAsEnvVariable()) { + + if (!shouldUseDeepStorageForTaskPayload(task)) { podTemplateAnnotationBuilder .put(DruidK8sConstants.TASK, Base64Compression.compressBase64(mapper.writeValueAsString(task))); } @@ -313,4 +318,17 @@ private String getDruidLabel(String baseLabel) { return DruidK8sConstants.DRUID_LABEL_PREFIX + baseLabel; } + + @Override + public Boolean shouldUseDeepStorageForTaskPayload(Task task) + { + try { + String compressedTaskPayload = Base64Compression.compressBase64(mapper.writeValueAsString(task)); + return compressedTaskPayload.length() > DruidK8sConstants.MAX_ENV_VARIABLE_KBS; + } + catch (Exception e) { + // In case there's a issue with checking how large the task is, default to using deep storage so we don't lose the task payload. + return true; + } + } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java index 718f70f2f82a..6e580767297d 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java @@ -33,4 +33,9 @@ public interface TaskAdapter Task toTask(Job from) throws IOException; K8sTaskId getTaskId(Job from) throws IOException; + + /** + * Method for exposing to external classes whether the task has its task payload bundled by the adapter or relies on a external system + */ + Boolean shouldUseDeepStorageForTaskPayload(Task task) throws IOException; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 7290b3ee960d..9fd405f4cef8 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -30,8 +30,6 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; 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.Task; import org.apache.druid.k8s.overlord.common.JobResponse; import org.apache.druid.k8s.overlord.common.K8sTaskId; @@ -70,7 +68,6 @@ public class KubernetesPeonLifecycleTest extends EasyMockSupport private ObjectMapper mapper; private Task task; private K8sTaskId k8sTaskId; - private TaskConfig taskConfig; @Before public void setup() @@ -78,7 +75,6 @@ public void setup() mapper = new TestUtils().getTestObjectMapper(); task = K8sTestUtils.createTask(ID, 0); k8sTaskId = new K8sTaskId(task); - taskConfig = new TaskConfigBuilder().build(); EasyMock.expect(logWatch.getOutput()).andReturn(IOUtils.toInputStream("", StandardCharsets.UTF_8)).anyTimes(); } @@ -90,8 +86,7 @@ public void test_run() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ) { @Override @@ -119,7 +114,7 @@ protected synchronized TaskStatus join(long timeout) replayAll(); - TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L); + TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L, false); verifyAll(); @@ -131,14 +126,12 @@ protected synchronized TaskStatus join(long timeout) @Test public void test_run_useTaskManager() throws IOException { - TaskConfig taskConfigTaskManager = new TaskConfigBuilder().setEnableTaskPayloadManagerPerTask(true).build(); KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle( task, kubernetesClient, taskLogs, mapper, - stateListener, - taskConfigTaskManager + stateListener ) { @Override @@ -166,7 +159,7 @@ protected synchronized TaskStatus join(long timeout) taskLogs.pushTaskPayload(EasyMock.anyString(), EasyMock.anyObject()); replayAll(); - TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L); + TaskStatus taskStatus = peonLifecycle.run(job, 0L, 0L, true); verifyAll(); Assert.assertTrue(taskStatus.isSuccess()); @@ -182,8 +175,7 @@ public void test_run_whenCalledMultipleTimes_raisesIllegalStateException() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ) { @Override @@ -211,12 +203,12 @@ protected synchronized TaskStatus join(long timeout) replayAll(); - peonLifecycle.run(job, 0L, 0L); + peonLifecycle.run(job, 0L, 0L, false); Assert.assertThrows( "Task [id] failed to run: invalid peon lifecycle state transition [STOPPED]->[PENDING]", IllegalStateException.class, - () -> peonLifecycle.run(job, 0L, 0L) + () -> peonLifecycle.run(job, 0L, 0L, false) ); verifyAll(); @@ -232,8 +224,7 @@ public void test_run_whenExceptionRaised_setsRunnerTaskStateToNone() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ) { @Override @@ -261,7 +252,7 @@ protected synchronized TaskStatus join(long timeout) Assert.assertThrows( Exception.class, - () -> peonLifecycle.run(job, 0L, 0L) + () -> peonLifecycle.run(job, 0L, 0L, false) ); verifyAll(); @@ -277,8 +268,7 @@ public void test_join_withoutJob_returnsFailedTaskStatus() throws IOException kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( @@ -317,8 +307,7 @@ public void test_join() throws IOException kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); Job job = new JobBuilder() @@ -370,8 +359,7 @@ public void test_join_whenCalledMultipleTimes_raisesIllegalStateException() thro kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); Job job = new JobBuilder() @@ -431,8 +419,7 @@ public void test_join_withoutTaskStatus_returnsFailedTaskStatus() throws IOExcep kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); Job job = new JobBuilder() @@ -482,8 +469,7 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskStatus_returnsFaile kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); Job job = new JobBuilder() @@ -533,8 +519,7 @@ public void test_join_whenIOExceptionThrownWhileStreamingTaskLogs_isIgnored() th kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); Job job = new JobBuilder() @@ -584,8 +569,7 @@ public void test_join_whenRuntimeExceptionThrownWhileWaitingForKubernetesJob_thr kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); EasyMock.expect(kubernetesClient.waitForPeonJobCompletion( @@ -624,8 +608,7 @@ public void test_shutdown_withNotStartedTaskState() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); peonLifecycle.shutdown(); } @@ -638,8 +621,7 @@ public void test_shutdown_withPendingTaskState() throws NoSuchFieldException, Il kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -660,8 +642,7 @@ public void test_shutdown_withRunningTaskState() throws NoSuchFieldException, Il kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -682,8 +663,7 @@ public void test_shutdown_withStoppedTaskState() throws NoSuchFieldException, Il kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); @@ -698,8 +678,7 @@ public void test_streamLogs_withNotStartedTaskState() throws NoSuchFieldExceptio kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); @@ -714,8 +693,7 @@ public void test_streamLogs_withPendingTaskState() throws NoSuchFieldException, kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -730,8 +708,7 @@ public void test_streamLogs_withRunningTaskState() throws NoSuchFieldException, kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -754,8 +731,7 @@ public void test_streamLogs_withStoppedTaskState() throws NoSuchFieldException, kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); @@ -771,8 +747,7 @@ public void test_getTaskLocation_withNotStartedTaskState_returnsUnknown() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.NOT_STARTED); @@ -788,8 +763,7 @@ public void test_getTaskLocation_withPendingTaskState_returnsUnknown() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.PENDING); @@ -805,8 +779,7 @@ public void test_getTaskLocation_withRunningTaskState_withoutPeonPod_returnsUnkn kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -828,8 +801,7 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithoutStatus_r kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -857,8 +829,7 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatus_retu kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -894,8 +865,7 @@ public void test_getTaskLocation_saveTaskLocation() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -931,8 +901,7 @@ public void test_getTaskLocation_withRunningTaskState_withPeonPodWithStatusWithT kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.RUNNING); @@ -969,8 +938,7 @@ public void test_getTaskLocation_withStoppedTaskState_returnsUnknown() kubernetesClient, taskLogs, mapper, - stateListener, - taskConfig + stateListener ); setPeonLifecycleState(peonLifecycle, KubernetesPeonLifecycle.State.STOPPED); EasyMock.expect(kubernetesClient.getPeonPod(k8sTaskId.getK8sJobName())).andReturn(Optional.absent()).once(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java index a1e0bd14f8ed..1f4a7281f649 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java @@ -55,7 +55,6 @@ public void test_deserializable() throws IOException Assert.assertEquals(ImmutableMap.of(), config.getLabels()); Assert.assertEquals(ImmutableMap.of(), config.getAnnotations()); Assert.assertEquals(Integer.valueOf(Integer.MAX_VALUE), config.getCapacity()); - Assert.assertEquals(Boolean.TRUE, config.isTaskPayloadAsEnvVariable()); } @Test @@ -81,7 +80,6 @@ public void test_builder_preservesDefaults() Assert.assertEquals(ImmutableMap.of(), config.getLabels()); Assert.assertEquals(ImmutableMap.of(), config.getAnnotations()); Assert.assertEquals(Integer.valueOf(Integer.MAX_VALUE), config.getCapacity()); - Assert.assertEquals(Boolean.TRUE, config.isTaskPayloadAsEnvVariable()); } @Test @@ -104,7 +102,6 @@ public void test_builder() .withLabels(ImmutableMap.of("key", "value")) .withAnnotations(ImmutableMap.of("key", "value")) .withCapacity(1) - .withTaskPayloadAsEnvVariable(Boolean.FALSE) .build(); Assert.assertEquals("namespace", config.getNamespace()); @@ -122,6 +119,5 @@ public void test_builder() Assert.assertEquals(ImmutableMap.of("key", "value"), config.getLabels()); Assert.assertEquals(ImmutableMap.of("key", "value"), config.getAnnotations()); Assert.assertEquals(Integer.valueOf(1), config.getCapacity()); - Assert.assertEquals(Boolean.FALSE, config.isTaskPayloadAsEnvVariable()); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 11e7a75af695..0967ea7c4196 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -140,10 +140,12 @@ public void test_run_withoutExistingTask() throws IOException, ExecutionExceptio TaskStatus taskStatus = TaskStatus.success(task.getId()); EasyMock.expect(taskAdapter.fromTask(task)).andReturn(job); + EasyMock.expect(taskAdapter.shouldUseDeepStorageForTaskPayload(task)).andReturn(false); EasyMock.expect(kubernetesPeonLifecycle.run( EasyMock.eq(job), EasyMock.anyLong(), - EasyMock.anyLong() + EasyMock.anyLong(), + EasyMock.anyBoolean() )).andReturn(taskStatus); replayAll(); @@ -175,10 +177,12 @@ public void test_run_whenExceptionThrown_throwsRuntimeException() throws IOExcep .build(); EasyMock.expect(taskAdapter.fromTask(task)).andReturn(job); + EasyMock.expect(taskAdapter.shouldUseDeepStorageForTaskPayload(task)).andReturn(false); EasyMock.expect(kubernetesPeonLifecycle.run( EasyMock.eq(job), EasyMock.anyLong(), - EasyMock.anyLong() + EasyMock.anyLong(), + EasyMock.anyBoolean() )).andThrow(new IllegalStateException()); replayAll(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java index 482a84e684b1..7d17193b1714 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesWorkItemTest.java @@ -56,7 +56,6 @@ public void test_setKubernetesPeonLifecycleTwice_throwsIllegalStateException() null, null, null, - null, null )); @@ -67,7 +66,6 @@ public void test_setKubernetesPeonLifecycleTwice_throwsIllegalStateException() null, null, null, - null, null )) ); @@ -160,7 +158,6 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_returnsPending() null, null, null, - null, null )); @@ -175,7 +172,6 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inPendingState_r null, null, null, - null, null ) { @Override @@ -198,7 +194,6 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inRunningState_r null, null, null, - null, null ) { @Override @@ -221,7 +216,6 @@ public void test_getRunnerTaskState_withKubernetesPeonLifecycle_inStoppedState_r null, null, null, - null, null ) { @Override @@ -250,7 +244,6 @@ public void test_streamTaskLogs_withKubernetesPeonLifecycle() null, null, null, - null, null )); Assert.assertFalse(workItem.streamTaskLogs().isPresent()); @@ -270,7 +263,6 @@ public void test_getLocation_withKubernetesPeonLifecycle() null, null, null, - null, null )); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 6bcc58f359bd..e8dc1cfc13ce 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -41,6 +41,7 @@ import io.fabric8.kubernetes.api.model.batch.v1.JobList; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; +import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.StringUtils; import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.TestUtils; @@ -229,7 +230,6 @@ public PodSpec getSpec() KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") - .withTaskPayloadAsEnvVariable(false) .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -240,7 +240,14 @@ public PodSpec getSpec() jsonMapper, taskLogs ); - Task task = K8sTestUtils.getTask(); + Task task = new NoopTask( + "id", + "id", + "datasource", + 0, + 0, + ImmutableMap.of("context", RandomStringUtils.randomAlphanumeric((int) DruidK8sConstants.MAX_ENV_VARIABLE_KBS * 20)) + ); Job job = adapter.fromTask(task); // TASK_JSON should not be set in env variables Assertions.assertFalse( @@ -280,7 +287,7 @@ public void toTask_useTaskPayloadManager() throws IOException K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, - new TaskConfigBuilder().setEnableTaskPayloadManagerPerTask(true).build(), + taskConfig, startupLoggingConfig, node, jsonMapper, @@ -288,9 +295,10 @@ public void toTask_useTaskPayloadManager() throws IOException ); Job job = new JobBuilder() + .editMetadata().withName("job").endMetadata() .editSpec().editTemplate().editMetadata() .addToAnnotations(DruidK8sConstants.TASK_ID, "ID") - .endMetadata().endTemplate().endSpec().build(); + .endMetadata().editSpec().addToContainers(new ContainerBuilder().withName("main").build()).endSpec().endTemplate().endSpec().build(); Task taskFromJob = adapter.toTask(job); assertEquals(taskInTaskPayloadManager, taskFromJob); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 8b9a1fdc867d..2de93f1e7e15 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -21,9 +21,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; import io.fabric8.kubernetes.api.model.PodTemplate; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; +import org.apache.commons.lang.RandomStringUtils; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; @@ -241,7 +243,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites_dontSetTaskJSON props.setProperty("druid.indexer.runner.k8s.podTemplate.noop", templatePath.toString()); PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( - KubernetesTaskRunnerConfig.builder().withTaskPayloadAsEnvVariable(false).build(), + taskRunnerConfig, taskConfig, node, mapper, @@ -255,7 +257,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites_dontSetTaskJSON "datasource", 0, 0, - null + ImmutableMap.of("context", RandomStringUtils.randomAlphanumeric((int) DruidK8sConstants.MAX_ENV_VARIABLE_KBS * 20)) ); Job actual = adapter.fromTask(task); @@ -427,7 +429,7 @@ public void test_toTask_useTaskPayloadManager() throws IOException Properties props = new Properties(); props.put("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString()); - Task expected = K8sTestUtils.createTask("id", 1); + Task expected = new NoopTask("id", null, "datasource", 0, 0, ImmutableMap.of()); TaskLogs mockTestLogs = Mockito.mock(TaskLogs.class); Mockito.when(mockTestLogs.streamTaskPayload("id")).thenReturn(Optional.of( new ByteArrayInputStream(mapper.writeValueAsString(expected).getBytes(Charset.defaultCharset())) @@ -435,7 +437,7 @@ public void test_toTask_useTaskPayloadManager() throws IOException PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( taskRunnerConfig, - new TaskConfigBuilder().setEnableTaskPayloadManagerPerTask(true).build(), + taskConfig, node, mapper, props, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java index 84bf55e7d19a..db48d6f07f7d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/config/TaskConfig.java @@ -125,9 +125,6 @@ public enum BatchProcessingMode @JsonProperty private final long tmpStorageBytesPerTask; - @JsonProperty - private final boolean useDeepStorageForTaskPayload; - @JsonCreator public TaskConfig( @JsonProperty("baseDir") String baseDir, @@ -145,8 +142,7 @@ public TaskConfig( @JsonProperty("batchProcessingMode") String batchProcessingMode, @JsonProperty("storeEmptyColumns") @Nullable Boolean storeEmptyColumns, @JsonProperty("encapsulatedTask") boolean enableTaskLevelLogPush, - @JsonProperty("tmpStorageBytesPerTask") @Nullable Long tmpStorageBytesPerTask, - @JsonProperty("useDeepStorageForTaskPayload") @Nullable boolean useDeepStorageForTaskPayload + @JsonProperty("tmpStorageBytesPerTask") @Nullable Long tmpStorageBytesPerTask ) { this.baseDir = Configs.valueOrDefault(baseDir, System.getProperty("java.io.tmpdir")); @@ -197,7 +193,6 @@ public TaskConfig( this.storeEmptyColumns = Configs.valueOrDefault(storeEmptyColumns, DEFAULT_STORE_EMPTY_COLUMNS); this.tmpStorageBytesPerTask = Configs.valueOrDefault(tmpStorageBytesPerTask, DEFAULT_TMP_STORAGE_BYTES_PER_TASK); - this.useDeepStorageForTaskPayload = useDeepStorageForTaskPayload; } private TaskConfig( @@ -215,8 +210,7 @@ private TaskConfig( BatchProcessingMode batchProcessingMode, boolean storeEmptyColumns, boolean encapsulatedTask, - long tmpStorageBytesPerTask, - boolean useDeepStorageForTaskPayload + long tmpStorageBytesPerTask ) { this.baseDir = baseDir; @@ -234,7 +228,6 @@ private TaskConfig( this.storeEmptyColumns = storeEmptyColumns; this.encapsulatedTask = encapsulatedTask; this.tmpStorageBytesPerTask = tmpStorageBytesPerTask; - this.useDeepStorageForTaskPayload = useDeepStorageForTaskPayload; } @JsonProperty @@ -345,12 +338,6 @@ public boolean isEncapsulatedTask() return encapsulatedTask; } - @JsonProperty - public boolean isUseDeepStorageForTaskPayload() - { - return useDeepStorageForTaskPayload; - } - @JsonProperty public long getTmpStorageBytesPerTask() { @@ -383,8 +370,7 @@ public TaskConfig withBaseTaskDir(File baseTaskDir) batchProcessingMode, storeEmptyColumns, encapsulatedTask, - tmpStorageBytesPerTask, - useDeepStorageForTaskPayload + tmpStorageBytesPerTask ); } @@ -405,8 +391,7 @@ public TaskConfig withTmpStorageBytesPerTask(long tmpStorageBytesPerTask) batchProcessingMode, storeEmptyColumns, encapsulatedTask, - tmpStorageBytesPerTask, - useDeepStorageForTaskPayload + tmpStorageBytesPerTask ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java index 9600d185ac63..af920ebbeb73 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/config/TaskConfigBuilder.java @@ -41,7 +41,6 @@ public class TaskConfigBuilder private Boolean storeEmptyColumns; private boolean enableTaskLevelLogPush; private Long tmpStorageBytesPerTask; - private boolean enableTaskPayloadManagerPerTask; public TaskConfigBuilder setBaseDir(String baseDir) { @@ -133,11 +132,6 @@ public TaskConfigBuilder setTmpStorageBytesPerTask(Long tmpStorageBytesPerTask) return this; } - public TaskConfigBuilder setEnableTaskPayloadManagerPerTask(boolean enableTaskPayloadManagerPerTask) - { - this.enableTaskPayloadManagerPerTask = enableTaskPayloadManagerPerTask; - return this; - } public TaskConfig build() { return new TaskConfig( @@ -155,8 +149,7 @@ public TaskConfig build() batchProcessingMode, storeEmptyColumns, enableTaskLevelLogPush, - tmpStorageBytesPerTask, - enableTaskPayloadManagerPerTask + tmpStorageBytesPerTask ); } } diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index b5b03d03aff1..123fea7d0aae 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -293,11 +293,11 @@ public Supplier> heartbeatDimensions(Task task) public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager, TaskConfig taskConfig) { try { - if (taskConfig.isUseDeepStorageForTaskPayload()) { + if (!config.getTaskFile().exists()) { + log.info("Task file not found, trying to pull task payload from deep storage"); String task = IOUtils.toString(taskPayloadManager.streamTaskPayload(taskId).get(), Charset.defaultCharset()); - // write the remote task.json to task file location for ExecutorLifecycle to pickup + // write the remote task.json to the task file location for ExecutorLifecycle to pickup FileUtils.write(config.getTaskFile(), task, Charset.defaultCharset()); - return smileMapper.readValue(task, Task.class); } return mapper.readValue(config.getTaskFile(), Task.class); } From bbe7444c4a51f930004cacaf91ed69a61f7a1033 Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 15 Sep 2023 15:36:07 -0400 Subject: [PATCH 18/27] Small fixes --- .../org/apache/druid/k8s/overlord/common/DruidK8sConstants.java | 1 - services/src/main/java/org/apache/druid/cli/CliPeon.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java index 0a6fe6e73347..7d35827f89bb 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java @@ -40,7 +40,6 @@ public class DruidK8sConstants public static final String DRUID_HOSTNAME_ENV = "HOSTNAME"; public static final String LABEL_KEY = "druid.k8s.peons"; public static final String DRUID_LABEL_PREFIX = "druid."; - public static final long MAX_ENV_VARIABLE_KBS = 130048; // 127 KB static final Predicate IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException; } diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 123fea7d0aae..a94e6cd96b83 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -293,7 +293,7 @@ public Supplier> heartbeatDimensions(Task task) public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager, TaskConfig taskConfig) { try { - if (!config.getTaskFile().exists()) { + if (!config.getTaskFile().exists() || config.getTaskFile().length() == 0) { log.info("Task file not found, trying to pull task payload from deep storage"); String task = IOUtils.toString(taskPayloadManager.streamTaskPayload(taskId).get(), Charset.defaultCharset()); // write the remote task.json to the task file location for ExecutorLifecycle to pickup From f3f8aac640af8105c27f0f01a8147e78a205c0e8 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 18 Sep 2023 09:38:25 -0400 Subject: [PATCH 19/27] Remove uneeded param --- services/src/main/java/org/apache/druid/cli/CliPeon.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index a94e6cd96b83..432ebdefb91c 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -290,7 +290,7 @@ public Supplier> heartbeatDimensions(Task task) @Provides @LazySingleton - public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager, TaskConfig taskConfig) + public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager) { try { if (!config.getTaskFile().exists() || config.getTaskFile().length() == 0) { From 776068842d5128fa7c268cafe3ef80ad126e1bc9 Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 18 Sep 2023 10:19:28 -0400 Subject: [PATCH 20/27] fix build --- .../org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java index 6e580767297d..55861a1bede3 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java @@ -37,5 +37,5 @@ public interface TaskAdapter /** * Method for exposing to external classes whether the task has its task payload bundled by the adapter or relies on a external system */ - Boolean shouldUseDeepStorageForTaskPayload(Task task) throws IOException; + Boolean shouldUseDeepStorageForTaskPayload(Task task); } From 4afa8f36704f5b74d058d789f79c61c1bb105838 Mon Sep 17 00:00:00 2001 From: George Wu Date: Tue, 19 Sep 2023 13:37:13 -0400 Subject: [PATCH 21/27] remove unneeded arg --- services/src/main/java/org/apache/druid/cli/CliPeon.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 432ebdefb91c..77623eeb1db0 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -290,7 +290,7 @@ public Supplier> heartbeatDimensions(Task task) @Provides @LazySingleton - public Task readTask(@Json ObjectMapper mapper, @Smile ObjectMapper smileMapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager) + public Task readTask(@Json ObjectMapper mapper, ExecutorLifecycleConfig config, TaskPayloadManager taskPayloadManager) { try { if (!config.getTaskFile().exists() || config.getTaskFile().length() == 0) { From 0af80bfc67a054ffec8e478397bba786b8b49d13 Mon Sep 17 00:00:00 2001 From: George Wu Date: Tue, 19 Sep 2023 14:15:49 -0400 Subject: [PATCH 22/27] Remove unused import --- services/src/main/java/org/apache/druid/cli/CliPeon.java | 1 - 1 file changed, 1 deletion(-) diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 77623eeb1db0..b6b1730199f2 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -68,7 +68,6 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.Self; -import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; From ba82cb103da3b9dadfbc5f4193451d6b3680d699 Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 22 Sep 2023 16:06:27 -0400 Subject: [PATCH 23/27] PR changes --- distribution/docker/peon.sh | 3 ++- .../k8s/overlord/KubernetesPeonLifecycle.java | 2 +- .../k8s/overlord/taskadapter/K8sTaskAdapter.java | 12 +++--------- .../taskadapter/PodTemplateTaskAdapter.java | 14 ++++---------- .../k8s/overlord/taskadapter/TaskAdapter.java | 2 +- .../org/apache/druid/storage/s3/S3TaskLogs.java | 2 +- .../apache/druid/tasklogs/TaskPayloadManager.java | 3 ++- 7 files changed, 14 insertions(+), 24 deletions(-) diff --git a/distribution/docker/peon.sh b/distribution/docker/peon.sh index 50ba9727c9d6..b5ec89ea8d46 100755 --- a/distribution/docker/peon.sh +++ b/distribution/docker/peon.sh @@ -149,7 +149,8 @@ then mkdir -p ${DRUID_DIRS_TO_CREATE} fi -# take the ${TASK_JSON} environment variable and base64 decode, unzip and throw it in ${TASK_DIR}/task.json +# take the ${TASK_JSON} environment variable and base64 decode, unzip and throw it in ${TASK_DIR}/task.json. +# If TASK_JSON is not set, CliPeon will pull the task.json file from deep storage. mkdir -p ${TASK_DIR}; [ -n "$TASK_JSON" ] && echo ${TASK_JSON} | base64 -d | gzip -d > ${TASK_DIR}/task.json; exec bin/run-java ${JAVA_OPTS} -cp $COMMON_CONF_DIR:$SERVICE_CONF_DIR:lib/*: org.apache.druid.cli.Main internal peon $@ diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index d13b1d8c53a4..038a56c90113 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -119,7 +119,7 @@ protected KubernetesPeonLifecycle( * @return * @throws IllegalStateException */ - protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, Boolean useDeepStorageForTaskPayload) throws IllegalStateException + protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, boolean useDeepStorageForTaskPayload) throws IllegalStateException { try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index 5a39681ccdbf..e3255de8da73 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -477,16 +477,10 @@ static ResourceRequirements getResourceRequirements(ResourceRequirements require } @Override - public Boolean shouldUseDeepStorageForTaskPayload(Task task) + public boolean shouldUseDeepStorageForTaskPayload(Task task) throws IOException { - try { - String compressedTaskPayload = Base64Compression.compressBase64(mapper.writeValueAsString(task)); - return compressedTaskPayload.length() > DruidK8sConstants.MAX_ENV_VARIABLE_KBS; - } - catch (Exception e) { - // In case there's a issue with checking how large the task is, default to using deep storage so we don't lose the task payload. - return true; - } + String compressedTaskPayload = Base64Compression.compressBase64(mapper.writeValueAsString(task)); + return compressedTaskPayload.length() > DruidK8sConstants.MAX_ENV_VARIABLE_KBS; } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 06436936f7b4..88b465161112 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -242,7 +242,7 @@ private Optional loadPodTemplate(String key, Properties properties) } } - private Collection getEnv(Task task) + private Collection getEnv(Task task) throws IOException { List envVars = Lists.newArrayList( new EnvVarBuilder() @@ -320,15 +320,9 @@ private String getDruidLabel(String baseLabel) } @Override - public Boolean shouldUseDeepStorageForTaskPayload(Task task) + public boolean shouldUseDeepStorageForTaskPayload(Task task) throws IOException { - try { - String compressedTaskPayload = Base64Compression.compressBase64(mapper.writeValueAsString(task)); - return compressedTaskPayload.length() > DruidK8sConstants.MAX_ENV_VARIABLE_KBS; - } - catch (Exception e) { - // In case there's a issue with checking how large the task is, default to using deep storage so we don't lose the task payload. - return true; - } + String compressedTaskPayload = Base64Compression.compressBase64(mapper.writeValueAsString(task)); + return compressedTaskPayload.length() > DruidK8sConstants.MAX_ENV_VARIABLE_KBS; } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java index 55861a1bede3..1820f6ad910d 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java @@ -37,5 +37,5 @@ public interface TaskAdapter /** * Method for exposing to external classes whether the task has its task payload bundled by the adapter or relies on a external system */ - Boolean shouldUseDeepStorageForTaskPayload(Task task); + boolean shouldUseDeepStorageForTaskPayload(Task task) throws IOException; } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java index 4032a380e520..b9fd9e42862c 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TaskLogs.java @@ -87,7 +87,7 @@ public Optional streamTaskStatus(String taskid) throws IOException public void pushTaskPayload(String taskid, File taskPayloadFile) throws IOException { final String taskKey = getTaskLogKey(taskid, "task.json"); - log.info("Pushing task payload %s to: %s", taskPayloadFile, taskKey); + log.info("Pushing task payload [%s] to location [%s]", taskPayloadFile, taskKey); pushTaskFile(taskPayloadFile, taskKey); } diff --git a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java index 66ebb70a38bd..41db8e4556b2 100644 --- a/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java +++ b/processing/src/main/java/org/apache/druid/tasklogs/TaskPayloadManager.java @@ -29,7 +29,8 @@ import java.io.InputStream; /** - * Something that knows how to manage task json paylods. + * Something that knows how to push a task payload before it is run to somewhere + * a ingestion worker will be able to stream the task payload from when trying to run the task. */ @ExtensionPoint public interface TaskPayloadManager From f200439362cba279d6ef515f446c8358fec3f1ef Mon Sep 17 00:00:00 2001 From: George Wu Date: Mon, 25 Sep 2023 11:41:05 -0400 Subject: [PATCH 24/27] more pr changes --- .../k8s/overlord/KubernetesPeonLifecycle.java | 31 ++++----- .../overlord/taskadapter/K8sTaskAdapter.java | 3 +- .../overlord/KubernetesPeonLifecycleTest.java | 4 +- .../org/apache/druid/error/InternalError.java | 68 +++++++++++++++++++ .../apache/druid/error/InternalErrorTest.java | 59 ++++++++++++++++ 5 files changed, 145 insertions(+), 20 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/error/InternalError.java create mode 100644 processing/src/test/java/org/apache/druid/error/InternalErrorTest.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java index 038a56c90113..5c6c7c6b3ebe 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java @@ -72,9 +72,9 @@ public interface TaskStateListener protected enum State { - /** Lifecycle's state before {@link #run(Job, long, long, Boolean)} or {@link #join(long)} is called. */ + /** Lifecycle's state before {@link #run(Job, long, long, boolean)} or {@link #join(long)} is called. */ NOT_STARTED, - /** Lifecycle's state since {@link #run(Job, long, long, Boolean)} is called. */ + /** Lifecycle's state since {@link #run(Job, long, long, boolean)} is called. */ PENDING, /** Lifecycle's state since {@link #join(long)} is called. */ RUNNING, @@ -119,7 +119,7 @@ protected KubernetesPeonLifecycle( * @return * @throws IllegalStateException */ - protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, boolean useDeepStorageForTaskPayload) throws IllegalStateException + protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, boolean useDeepStorageForTaskPayload) throws IllegalStateException, IOException { try { updateState(new State[]{State.NOT_STARTED}, State.PENDING); @@ -148,26 +148,23 @@ protected synchronized TaskStatus run(Job job, long launchTimeout, long timeout, } } - private void writeTaskPayload(Task task) + private void writeTaskPayload(Task task) throws IOException { + Path file = null; try { - Path file = Files.createTempFile(taskId.getOriginalTaskId(), "task.json"); - try { - FileUtils.writeStringToFile(file.toFile(), mapper.writeValueAsString(task), Charset.defaultCharset()); - taskLogs.pushTaskPayload(task.getId(), file.toFile()); - } - catch (Exception e) { - log.error("Failed to write task payload for task: %s", taskId.getOriginalTaskId()); - throw new RuntimeException(e); - } - finally { - Files.deleteIfExists(file); - } + file = Files.createTempFile(taskId.getOriginalTaskId(), "task.json"); + FileUtils.writeStringToFile(file.toFile(), mapper.writeValueAsString(task), Charset.defaultCharset()); + taskLogs.pushTaskPayload(task.getId(), file.toFile()); } - catch (IOException e) { + catch (Exception e) { log.error("Failed to write task payload for task: %s", taskId.getOriginalTaskId()); throw new RuntimeException(e); } + finally { + if (file != null) { + Files.deleteIfExists(file); + } + } } /** diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index e3255de8da73..b92a818b100b 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -43,6 +43,7 @@ import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.druid.error.InternalError; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.ForkingTaskRunner; @@ -150,7 +151,7 @@ private Task toTaskUsingDeepStorage(Job from) throws IOException { com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); if (!taskBody.isPresent()) { - throw new IOE("Could not load task payload for job [%s]", from.getMetadata().getName()); + throw InternalError.exception("Could not load task payload for job [%s]", from.getMetadata().getName()); } String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); return mapper.readValue(task, Task.class); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java index 9fd405f4cef8..1c6e429a3dc3 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycleTest.java @@ -79,7 +79,7 @@ public void setup() } @Test - public void test_run() + public void test_run() throws IOException { KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle( task, @@ -168,7 +168,7 @@ protected synchronized TaskStatus join(long timeout) } @Test - public void test_run_whenCalledMultipleTimes_raisesIllegalStateException() + public void test_run_whenCalledMultipleTimes_raisesIllegalStateException() throws IOException { KubernetesPeonLifecycle peonLifecycle = new KubernetesPeonLifecycle( task, diff --git a/processing/src/main/java/org/apache/druid/error/InternalError.java b/processing/src/main/java/org/apache/druid/error/InternalError.java new file mode 100644 index 000000000000..d6fcbac4d6ee --- /dev/null +++ b/processing/src/main/java/org/apache/druid/error/InternalError.java @@ -0,0 +1,68 @@ +/* + * 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.error; + +public class InternalError extends DruidException.Failure +{ + + public static DruidException exception() + { + return exception("Internal Error"); + } + + public static DruidException exception(String msg, Object... args) + { + return exception(null, msg, args); + } + + public static DruidException exception(Throwable t, String msg, Object... args) + { + return DruidException.fromFailure(new InternalError(t, msg, args)); + } + + private final Throwable t; + private final String msg; + private final Object[] args; + + private InternalError( + Throwable t, + String msg, + Object... args + ) + { + super("canceled"); + this.t = t; + this.msg = msg; + this.args = args; + } + + @Override + public DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + bob = bob.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.CANCELED); + + if (t == null) { + return bob.build(msg, args); + } else { + return bob.build(t, msg, args); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/error/InternalErrorTest.java b/processing/src/test/java/org/apache/druid/error/InternalErrorTest.java new file mode 100644 index 000000000000..850bec5f479e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/error/InternalErrorTest.java @@ -0,0 +1,59 @@ +/* + * 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.error; + +import org.apache.druid.matchers.DruidMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Test; + +import java.util.Map; + +public class InternalErrorTest +{ + + @Test + public void testAsErrorResponse() + { + ErrorResponse errorResponse = new ErrorResponse(InternalError.exception()); + final Map asMap = errorResponse.getAsMap(); + + MatcherAssert.assertThat( + asMap, + DruidMatchers.mapMatcher( + "error", "druidException", + "errorCode", "canceled", + "persona", "OPERATOR", + "category", "CANCELED", + "errorMessage", "Internal Error" + ) + ); + + ErrorResponse recomposed = ErrorResponse.fromMap(asMap); + + MatcherAssert.assertThat( + recomposed.getUnderlyingException(), + new DruidExceptionMatcher( + DruidException.Persona.OPERATOR, + DruidException.Category.CANCELED, + "canceled" + ).expectMessageContains("Internal Error") + ); + } +} From 59dc958a69ec58644528e41759b796e16b080493 Mon Sep 17 00:00:00 2001 From: George Wu Date: Thu, 28 Sep 2023 10:33:01 -0400 Subject: [PATCH 25/27] More pr changes --- .../overlord/taskadapter/K8sTaskAdapter.java | 15 +++++++----- .../taskadapter/PodTemplateTaskAdapter.java | 14 +++++++---- .../taskadapter/K8sTaskAdapterTest.java | 6 ++--- .../PodTemplateTaskAdapterTest.java | 12 +++++----- ...nalError.java => InternalServerError.java} | 24 +++++++------------ ...Test.java => InternalServerErrorTest.java} | 16 ++++++------- 6 files changed, 44 insertions(+), 43 deletions(-) rename processing/src/main/java/org/apache/druid/error/{InternalError.java => InternalServerError.java} (71%) rename processing/src/test/java/org/apache/druid/error/{InternalErrorTest.java => InternalServerErrorTest.java} (77%) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index b92a818b100b..95e049581846 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -43,14 +43,14 @@ import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.druid.error.InternalError; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InternalServerError; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.ForkingTaskRunner; import org.apache.druid.indexing.overlord.QuotableWhiteSpaceSplitter; import org.apache.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.Base64Compression; @@ -151,22 +151,25 @@ private Task toTaskUsingDeepStorage(Job from) throws IOException { com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); if (!taskBody.isPresent()) { - throw InternalError.exception("Could not load task payload for job [%s]", from.getMetadata().getName()); + throw InternalServerError.exception( + "Could not load task payload from deep storage for job [%s]. Check the overlord logs for errors uploading task payloads to deep storage.", + from.getMetadata().getName() + ); } String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); return mapper.readValue(task, Task.class); } @Override - public K8sTaskId getTaskId(Job from) throws IOException + public K8sTaskId getTaskId(Job from) { Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); if (annotations == null) { - throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); + throw DruidException.defensive().build("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); } String taskId = annotations.get(DruidK8sConstants.TASK_ID); if (taskId == null) { - throw new IOE("No task_id annotation found on pod spec for job [%s]", from.getMetadata().getName()); + throw DruidException.defensive().build("No task_id annotation found on pod spec for job [%s]", from.getMetadata().getName()); } return new K8sTaskId(taskId); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 88b465161112..ef0509a673f8 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -36,11 +36,12 @@ import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import io.fabric8.kubernetes.client.utils.Serialization; import org.apache.commons.io.IOUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InternalServerError; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -186,22 +187,25 @@ private Task toTaskUsingDeepStorage(Job from) throws IOException { com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); if (!taskBody.isPresent()) { - throw new IOE("Could not load task payload for job [%s]", from.getMetadata().getName()); + throw InternalServerError.exception( + "Could not load task payload from deep storage for job [%s]. Check the overlord logs for errors uploading task payloads to deep storage.", + from.getMetadata().getName() + ); } String task = IOUtils.toString(taskBody.get(), Charset.defaultCharset()); return mapper.readValue(task, Task.class); } @Override - public K8sTaskId getTaskId(Job from) throws IOException + public K8sTaskId getTaskId(Job from) { Map annotations = from.getSpec().getTemplate().getMetadata().getAnnotations(); if (annotations == null) { - throw new IOE("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); + throw DruidException.defensive().build("No annotations found on pod spec for job [%s]", from.getMetadata().getName()); } String taskId = annotations.get(DruidK8sConstants.TASK_ID); if (taskId == null) { - throw new IOE("No task_id annotation found on pod spec for job [%s]", from.getMetadata().getName()); + throw DruidException.defensive().build("No task_id annotation found on pod spec for job [%s]", from.getMetadata().getName()); } return new K8sTaskId(taskId); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index e8dc1cfc13ce..6ae6bd5cc7b5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -43,6 +43,7 @@ import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient; import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.StringUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; @@ -52,7 +53,6 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.IOE; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; @@ -345,7 +345,7 @@ public void getTaskId_noAnnotations() .endMetadata().endTemplate().endSpec() .editMetadata().withName("job").endMetadata().build(); - Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + Assert.assertThrows(DruidException.class, () -> adapter.getTaskId(job)); } @Test @@ -368,7 +368,7 @@ public void getTaskId_missingTaskIdAnnotation() .endMetadata().endTemplate().endSpec() .editMetadata().withName("job").endMetadata().build(); - Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + Assert.assertThrows(DruidException.class, () -> adapter.getTaskId(job)); } @Test void testGrabbingTheLastXmxValueFromACommand() diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 2de93f1e7e15..74dfacd1a327 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -26,13 +26,13 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import org.apache.commons.lang.RandomStringUtils; +import org.apache.druid.error.DruidException; 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.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.common.Base64Compression; @@ -267,7 +267,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites_dontSetTaskJSON } @Test - public void test_fromTask_withoutAnnotations_throwsIOE() throws IOException + public void test_fromTask_withoutAnnotations_throwsDruidException() throws IOException { Path templatePath = Files.createFile(tempDir.resolve("base.yaml")); mapper.writeValue(templatePath.toFile(), podTemplateSpec); @@ -287,7 +287,7 @@ public void test_fromTask_withoutAnnotations_throwsIOE() throws IOException Job job = K8sTestUtils.fileToResource("baseJobWithoutAnnotations.yaml", Job.class); - Assert.assertThrows(IOE.class, () -> adapter.toTask(job)); + Assert.assertThrows(DruidException.class, () -> adapter.toTask(job)); } @Test @@ -335,7 +335,7 @@ public void test_getTaskId_noAnnotations() throws IOException .endMetadata().endTemplate().endSpec() .editMetadata().withName("job").endMetadata().build(); - Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + Assert.assertThrows(DruidException.class, () -> adapter.getTaskId(job)); } @Test @@ -360,7 +360,7 @@ public void test_getTaskId_missingTaskIdAnnotation() throws IOException .endMetadata().endTemplate().endSpec() .editMetadata().withName("job").endMetadata().build(); - Assert.assertThrows(IOE.class, () -> adapter.getTaskId(job)); + Assert.assertThrows(DruidException.class, () -> adapter.getTaskId(job)); } @Test @@ -392,7 +392,7 @@ public void test_toTask_withoutTaskAnnotation_throwsIOE() throws IOException .endTemplate() .endSpec() .build(); - Assert.assertThrows(IOE.class, () -> adapter.toTask(job)); + Assert.assertThrows(DruidException.class, () -> adapter.toTask(job)); } @Test diff --git a/processing/src/main/java/org/apache/druid/error/InternalError.java b/processing/src/main/java/org/apache/druid/error/InternalServerError.java similarity index 71% rename from processing/src/main/java/org/apache/druid/error/InternalError.java rename to processing/src/main/java/org/apache/druid/error/InternalServerError.java index d6fcbac4d6ee..b730acb0e3d6 100644 --- a/processing/src/main/java/org/apache/druid/error/InternalError.java +++ b/processing/src/main/java/org/apache/druid/error/InternalServerError.java @@ -19,35 +19,29 @@ package org.apache.druid.error; -public class InternalError extends DruidException.Failure +public class InternalServerError extends DruidException.Failure { - - public static DruidException exception() + public static DruidException exception(String errorCode, String msg, Object... args) { - return exception("Internal Error"); + return exception(null, errorCode, msg, args); } - - public static DruidException exception(String msg, Object... args) - { - return exception(null, msg, args); - } - - public static DruidException exception(Throwable t, String msg, Object... args) + public static DruidException exception(Throwable t, String errorCode, String msg, Object... args) { - return DruidException.fromFailure(new InternalError(t, msg, args)); + return DruidException.fromFailure(new InternalServerError(t, errorCode, msg, args)); } private final Throwable t; private final String msg; private final Object[] args; - private InternalError( + private InternalServerError( Throwable t, + String errorCode, String msg, Object... args ) { - super("canceled"); + super(errorCode); this.t = t; this.msg = msg; this.args = args; @@ -57,7 +51,7 @@ private InternalError( public DruidException makeException(DruidException.DruidExceptionBuilder bob) { bob = bob.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.CANCELED); + .ofCategory(DruidException.Category.RUNTIME_FAILURE); if (t == null) { return bob.build(msg, args); diff --git a/processing/src/test/java/org/apache/druid/error/InternalErrorTest.java b/processing/src/test/java/org/apache/druid/error/InternalServerErrorTest.java similarity index 77% rename from processing/src/test/java/org/apache/druid/error/InternalErrorTest.java rename to processing/src/test/java/org/apache/druid/error/InternalServerErrorTest.java index 850bec5f479e..b28296b2c415 100644 --- a/processing/src/test/java/org/apache/druid/error/InternalErrorTest.java +++ b/processing/src/test/java/org/apache/druid/error/InternalServerErrorTest.java @@ -25,23 +25,23 @@ import java.util.Map; -public class InternalErrorTest +public class InternalServerErrorTest { @Test public void testAsErrorResponse() { - ErrorResponse errorResponse = new ErrorResponse(InternalError.exception()); + ErrorResponse errorResponse = new ErrorResponse(InternalServerError.exception("runtimeFailure", "Internal Server Error")); final Map asMap = errorResponse.getAsMap(); MatcherAssert.assertThat( asMap, DruidMatchers.mapMatcher( "error", "druidException", - "errorCode", "canceled", + "errorCode", "runtimeFailure", "persona", "OPERATOR", - "category", "CANCELED", - "errorMessage", "Internal Error" + "category", "RUNTIME_FAILURE", + "errorMessage", "Internal Server Error" ) ); @@ -51,9 +51,9 @@ public void testAsErrorResponse() recomposed.getUnderlyingException(), new DruidExceptionMatcher( DruidException.Persona.OPERATOR, - DruidException.Category.CANCELED, - "canceled" - ).expectMessageContains("Internal Error") + DruidException.Category.RUNTIME_FAILURE, + "runtimeFailure" + ).expectMessageContains("Internal Server Error") ); } } From 6123a3eb7bfb97fd2fdd1b6b97fc5d769f37a7cf Mon Sep 17 00:00:00 2001 From: George Wu Date: Fri, 29 Sep 2023 10:15:14 -0400 Subject: [PATCH 26/27] Fix static checks --- .../org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java | 2 +- .../druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java index 1820f6ad910d..9dacb213cf3f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/TaskAdapter.java @@ -32,7 +32,7 @@ public interface TaskAdapter Task toTask(Job from) throws IOException; - K8sTaskId getTaskId(Job from) throws IOException; + K8sTaskId getTaskId(Job from); /** * Method for exposing to external classes whether the task has its task payload bundled by the adapter or relies on a external system diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 6ae6bd5cc7b5..c1100ccc29d1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -305,7 +305,7 @@ public void toTask_useTaskPayloadManager() throws IOException } @Test - public void getTaskId() throws IOException + public void getTaskId() { TestKubernetesClient testClient = new TestKubernetesClient(client); KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); From 99b68f061b043c61b0251fe8ad74d94c8a5290d3 Mon Sep 17 00:00:00 2001 From: George Shiqi Wu Date: Fri, 29 Sep 2023 10:25:35 -0400 Subject: [PATCH 27/27] Update extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com> --- .../apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index 95e049581846..862b176b1159 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -152,7 +152,7 @@ private Task toTaskUsingDeepStorage(Job from) throws IOException com.google.common.base.Optional taskBody = taskLogs.streamTaskPayload(getTaskId(from).getOriginalTaskId()); if (!taskBody.isPresent()) { throw InternalServerError.exception( - "Could not load task payload from deep storage for job [%s]. Check the overlord logs for errors uploading task payloads to deep storage.", + "Could not load task payload from deep storage for job [%s]. Check the overlord logs for any errors in uploading task payload to deep storage.", from.getMetadata().getName() ); }