You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "abhishekagarwal87 (via GitHub)" <gi...@apache.org> on 2023/03/17 11:57:21 UTC

[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13896: Pod template task adapter

abhishekagarwal87 commented on code in PR #13896:
URL: https://github.com/apache/druid/pull/13896#discussion_r1140111190


##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapter.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.cfg.MapperConfig;
+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 io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder;
+import io.fabric8.kubernetes.api.model.ObjectFieldSelector;
+import io.fabric8.kubernetes.api.model.Pod;
+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.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;
+import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.server.DruidNode;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+public class PodTemplateTaskAdapter implements TaskAdapter

Review Comment:
   Please add javadocs on this class. 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexing.common.TestUtils;
+import org.apache.druid.indexing.common.config.TaskConfig;
+import org.apache.druid.indexing.overlord.config.TaskQueueConfig;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.server.log.StartupLoggingConfig;
+import org.apache.druid.tasklogs.NoopTaskLogs;
+import org.apache.druid.tasklogs.TaskLogPusher;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class KubernetesTaskRunnerFactoryTest
+{
+  private ObjectMapper objectMapper;
+  private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig;
+  private StartupLoggingConfig startupLoggingConfig;
+  private TaskQueueConfig taskQueueConfig;
+  private TaskLogPusher taskLogPusher;
+  private DruidNode druidNode;
+  private TaskConfig taskConfig;
+
+  @Before
+  public void setup()
+  {
+    objectMapper = new TestUtils().getTestObjectMapper();
+    kubernetesTaskRunnerConfig = new KubernetesTaskRunnerConfig();
+    startupLoggingConfig = new StartupLoggingConfig();
+    taskQueueConfig = new TaskQueueConfig(
+        1,
+        null,
+        null,
+        null
+    );
+    taskLogPusher = new NoopTaskLogs();
+    druidNode = new DruidNode(
+        "test",
+        "",
+        false,
+        0,
+        1,
+        true,
+        false
+    );
+    taskConfig = new TaskConfig(
+        "/tmp",
+        null,
+        null,
+        null,
+        null,
+        false,
+        null,
+        null,
+        null,
+        false,
+        false,
+        null,
+        null,
+        false,
+        ImmutableList.of("/tmp")
+    );
+  }
+
+  @Test
+  public void test_get_returnsSameKuberentesTaskRunner_asBuild()
+  {
+    KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
+        objectMapper,
+        kubernetesTaskRunnerConfig,
+        startupLoggingConfig,
+        taskQueueConfig,
+        taskLogPusher,
+        druidNode,
+        taskConfig
+    );
+
+    KubernetesTaskRunner expectedRunner = factory.build();
+    KubernetesTaskRunner actualRunner = factory.get();
+
+    Assert.assertEquals(expectedRunner, actualRunner);
+  }
+
+  @Test
+  public void test_build_withoutSidecarSupport_returnsKubernetesTaskRunnerWithSingleContainerTaskAdapter()
+  {
+    KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
+        objectMapper,
+        kubernetesTaskRunnerConfig,
+        startupLoggingConfig,
+        taskQueueConfig,
+        taskLogPusher,
+        druidNode,
+        taskConfig
+    );
+
+    KubernetesTaskRunner runner = factory.build();
+    Assert.assertNotNull(runner);
+  }
+
+  @Test
+  public void test_build_withSidecarSupport_returnsKubernetesTaskRunnerWithMultiContainerTaskAdapter()
+  {
+    kubernetesTaskRunnerConfig.sidecarSupport = true;
+
+    KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
+        objectMapper,
+        kubernetesTaskRunnerConfig,
+        startupLoggingConfig,
+        taskQueueConfig,
+        taskLogPusher,
+        druidNode,
+        taskConfig
+    );
+
+    KubernetesTaskRunner runner = factory.build();
+
+    Assert.assertNotNull(runner);

Review Comment:
   you are verifying that the runner is not null. but it doesn't verify that the runner has a multi-container task adapter. 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexing.common.TestUtils;
+import org.apache.druid.indexing.common.config.TaskConfig;
+import org.apache.druid.indexing.overlord.config.TaskQueueConfig;
+import org.apache.druid.server.DruidNode;
+import org.apache.druid.server.log.StartupLoggingConfig;
+import org.apache.druid.tasklogs.NoopTaskLogs;
+import org.apache.druid.tasklogs.TaskLogPusher;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class KubernetesTaskRunnerFactoryTest
+{
+  private ObjectMapper objectMapper;
+  private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig;
+  private StartupLoggingConfig startupLoggingConfig;
+  private TaskQueueConfig taskQueueConfig;
+  private TaskLogPusher taskLogPusher;
+  private DruidNode druidNode;
+  private TaskConfig taskConfig;
+
+  @Before
+  public void setup()
+  {
+    objectMapper = new TestUtils().getTestObjectMapper();
+    kubernetesTaskRunnerConfig = new KubernetesTaskRunnerConfig();
+    startupLoggingConfig = new StartupLoggingConfig();
+    taskQueueConfig = new TaskQueueConfig(
+        1,
+        null,
+        null,
+        null
+    );
+    taskLogPusher = new NoopTaskLogs();
+    druidNode = new DruidNode(
+        "test",
+        "",
+        false,
+        0,
+        1,
+        true,
+        false
+    );
+    taskConfig = new TaskConfig(
+        "/tmp",
+        null,
+        null,
+        null,
+        null,
+        false,
+        null,
+        null,
+        null,
+        false,
+        false,
+        null,
+        null,
+        false,
+        ImmutableList.of("/tmp")
+    );
+  }
+
+  @Test
+  public void test_get_returnsSameKuberentesTaskRunner_asBuild()
+  {
+    KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
+        objectMapper,
+        kubernetesTaskRunnerConfig,
+        startupLoggingConfig,
+        taskQueueConfig,
+        taskLogPusher,
+        druidNode,
+        taskConfig
+    );
+
+    KubernetesTaskRunner expectedRunner = factory.build();
+    KubernetesTaskRunner actualRunner = factory.get();
+
+    Assert.assertEquals(expectedRunner, actualRunner);
+  }
+
+  @Test
+  public void test_build_withoutSidecarSupport_returnsKubernetesTaskRunnerWithSingleContainerTaskAdapter()
+  {
+    KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
+        objectMapper,
+        kubernetesTaskRunnerConfig,
+        startupLoggingConfig,
+        taskQueueConfig,
+        taskLogPusher,
+        druidNode,
+        taskConfig
+    );
+
+    KubernetesTaskRunner runner = factory.build();
+    Assert.assertNotNull(runner);
+  }
+
+  @Test
+  public void test_build_withSidecarSupport_returnsKubernetesTaskRunnerWithMultiContainerTaskAdapter()
+  {
+    kubernetesTaskRunnerConfig.sidecarSupport = true;
+
+    KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
+        objectMapper,
+        kubernetesTaskRunnerConfig,
+        startupLoggingConfig,
+        taskQueueConfig,
+        taskLogPusher,
+        druidNode,
+        taskConfig
+    );
+
+    KubernetesTaskRunner runner = factory.build();
+
+    Assert.assertNotNull(runner);
+  }
+
+  @Test
+  public void test_build_withClientProxyDisabled_returnsKubernetesTaskRunnerWithDruidKubernetesClientWithoutClientProxySupport()
+  {
+    kubernetesTaskRunnerConfig.disableClientProxy = true;
+
+    KubernetesTaskRunnerFactory factory = new KubernetesTaskRunnerFactory(
+        objectMapper,
+        kubernetesTaskRunnerConfig,
+        startupLoggingConfig,
+        taskQueueConfig,
+        taskLogPusher,
+        druidNode,
+        taskConfig
+    );
+
+    KubernetesTaskRunner runner = factory.build();
+
+    Assert.assertNotNull(runner);

Review Comment:
   same comment as above here. 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapterTest.java:
##########


Review Comment:
   Please do add error message verification as well. 



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/TaskAdapter.java:
##########
@@ -19,15 +19,17 @@
 
 package org.apache.druid.k8s.overlord.common;
 
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
 import org.apache.druid.indexing.common.task.Task;
 
 import java.io.IOException;
 
-public interface TaskAdapter<K, V>
+public interface TaskAdapter
 {
 
-  V fromTask(Task task, PeonCommandContext context) throws IOException;

Review Comment:
   why would Fargate be implemented through a different runner? doesn't fargate offer k8s api? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapter.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.cfg.MapperConfig;
+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 io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder;
+import io.fabric8.kubernetes.api.model.ObjectFieldSelector;
+import io.fabric8.kubernetes.api.model.Pod;
+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.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;
+import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.server.DruidNode;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+public class PodTemplateTaskAdapter implements TaskAdapter
+{
+  public static String TYPE = "PodTemplate";
+
+  private static final Logger log = new Logger(PodTemplateTaskAdapter.class);
+  private static final String TASK_PROPERTY = IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX + ".k8s.podTemplate.%s";

Review Comment:
   are you going to document this in a follow up PR? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapter.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.cfg.MapperConfig;
+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 io.fabric8.kubernetes.api.model.EnvVar;
+import io.fabric8.kubernetes.api.model.EnvVarBuilder;
+import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder;
+import io.fabric8.kubernetes.api.model.ObjectFieldSelector;
+import io.fabric8.kubernetes.api.model.Pod;
+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.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;
+import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
+import org.apache.druid.server.DruidNode;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+public class PodTemplateTaskAdapter implements TaskAdapter
+{
+  public static String TYPE = "PodTemplate";
+
+  private static final Logger log = new Logger(PodTemplateTaskAdapter.class);
+  private static final String TASK_PROPERTY = IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX + ".k8s.podTemplate.%s";
+
+  private final KubernetesClientApi client;
+  private final KubernetesTaskRunnerConfig taskRunnerConfig;
+  private final TaskConfig taskConfig;
+  private final DruidNode node;
+  private final ObjectMapper mapper;
+  private final HashMap<String, PodTemplate> templates;
+
+  public PodTemplateTaskAdapter(
+      KubernetesClientApi client,
+      KubernetesTaskRunnerConfig taskRunnerConfig,
+      TaskConfig taskConfig,
+      DruidNode node,
+      ObjectMapper mapper,
+      Properties properties
+  )
+  {
+    this.client = client;
+    this.taskRunnerConfig = taskRunnerConfig;
+    this.taskConfig = taskConfig;
+    this.node = node;
+    this.mapper = mapper;
+    this.templates = initializePodTemplates(properties);
+  }
+
+  @Override
+  public Job fromTask(Task task) throws IOException

Review Comment:
   This method can use some documentation for future readers E.g. why is the backoff limit set to 0? 



##########
extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/PodTemplateTaskAdapterTest.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.k8s.overlord.common;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import io.fabric8.kubernetes.api.model.Pod;
+import io.fabric8.kubernetes.api.model.PodBuilder;
+import io.fabric8.kubernetes.api.model.PodTemplate;
+import io.fabric8.kubernetes.api.model.batch.v1.Job;
+import io.fabric8.kubernetes.client.KubernetesClient;
+import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
+import org.apache.druid.indexing.common.TestUtils;
+import org.apache.druid.indexing.common.config.TaskConfig;
+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.server.DruidNode;
+import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.Properties;
+
+@EnableKubernetesMockClient()
+public class PodTemplateTaskAdapterTest
+{
+  @TempDir private Path tempDir;
+  private KubernetesClient client;
+  private KubernetesTaskRunnerConfig taskRunnerConfig;
+  private TestKubernetesClient testClient;
+  private PodTemplate podTemplateSpec;
+  private TaskConfig taskConfig;
+  private DruidNode node;
+  private ObjectMapper mapper;
+
+  @BeforeEach
+  public void setup()
+  {
+    taskRunnerConfig = new KubernetesTaskRunnerConfig();
+    testClient = new TestKubernetesClient(client);
+    taskConfig = new TaskConfig(
+        "/tmp",
+        null,
+        null,
+        null,
+        null,
+        false,
+        null,
+        null,
+        null,
+        false,
+        false,
+        null,
+        null,
+        false,
+        ImmutableList.of("/tmp")
+    );
+    node = new DruidNode(
+        "test",
+        "",
+        false,
+        0,
+        1,
+        true,
+        false
+    );
+    mapper = new TestUtils().getTestObjectMapper();
+    podTemplateSpec = client
+        .v1()
+        .podTemplates()
+        .load(this.getClass()
+            .getClassLoader()
+            .getResourceAsStream("basePodTemplate.yaml")
+        )
+        .get();
+  }
+
+  @Test
+  public void test_fromTask_withoutBasePodTemplateInRuntimeProperites_raisesIAE()
+  {
+    Assert.assertThrows(IAE.class, () -> new PodTemplateTaskAdapter(

Review Comment:
   Please also verify that the correct error message is included in the exception. 



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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org