diff --git a/.github/scripts/setup_test_profiling_env.sh b/.github/scripts/setup_test_profiling_env.sh
index e3dff36867ab..b3bf4dc1844d 100755
--- a/.github/scripts/setup_test_profiling_env.sh
+++ b/.github/scripts/setup_test_profiling_env.sh
@@ -25,10 +25,17 @@ if [ "$#" -ne 5 ]; then
echo "usage: $0 "
fi
-if [[ "$1" == "17" ]];
+if [[ "$1" -ge "17" ]];
then
curl https://static.imply.io/cp/$JAR_INPUT_FILE -s -o $JAR_OUTPUT_FILE
+ # Run 'java -version' and capture the output
+ output=$(java -version 2>&1)
+
+ # Extract the version number using grep and awk
+ jvm_version=$(echo "$output" | grep "version" | awk -F '"' '{print $2}')
+
+
echo $ENV_VAR=-javaagent:"$PWD"/$JAR_OUTPUT_FILE \
-Djfr.profiler.http.username=druid-ci \
-Djfr.profiler.http.password=w3Fb6PW8LIo849mViEkbgA== \
@@ -36,7 +43,8 @@ then
-Djfr.profiler.tags.run_id=$2 \
-Djfr.profiler.tags.run_number=$3 \
-Djfr.profiler.tags.run_attempt=$4 \
- -Djfr.profiler.tags.module=$5
+ -Djfr.profiler.tags.module=$5 \
+ -Djfr.profiler.tags.jvm_version=$jvm_version
else
echo $ENV_VAR=\"\"
fi
diff --git a/.github/workflows/distribution-checks.yml b/.github/workflows/distribution-checks.yml
new file mode 100644
index 000000000000..82d6b12923fa
--- /dev/null
+++ b/.github/workflows/distribution-checks.yml
@@ -0,0 +1,41 @@
+# 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.
+
+name: "Distribution Checks"
+on:
+ push:
+ branches:
+ - master
+ - '[0-9]+.[0-9]+.[0-9]+' # release branches
+ - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches
+ paths:
+ - 'distribution/**'
+ - '**/pom.xml'
+ pull_request:
+ branches:
+ - master
+ - '[0-9]+.[0-9]+.[0-9]+' # release branches
+ - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches
+ paths:
+ - 'distribution/**'
+ - '**/pom.xml'
+
+jobs:
+ docker-build:
+ runs-on: ubuntu-latest
+ steps:
+ - uses: actions/checkout@v3
+ - name: Build the Docker image
+ run: DOCKER_BUILDKIT=1 docker build -t apache/druid:tag -f distribution/docker/Dockerfile .
diff --git a/README.md b/README.md
index cfd87db66728..d60f60a7bc58 100644
--- a/README.md
+++ b/README.md
@@ -19,7 +19,7 @@
[![Coverage Status](https://img.shields.io/codecov/c/gh/apache/druid?logo=codecov)](https://codecov.io/gh/apache/druid)
[![Docker](https://img.shields.io/badge/container-docker-blue.svg?logo=docker)](https://hub.docker.com/r/apache/druid)
-[![Helm](https://img.shields.io/badge/helm-druid-5F90AB?logo=helm)](https://github.com/apache/druid/blob/master/helm/druid/README.md)
+[![Helm](https://img.shields.io/badge/helm-druid-5F90AB?logo=helm)](https://github.com/asdf2014/druid-helm)
diff --git a/distribution/docker/Dockerfile b/distribution/docker/Dockerfile
index 2bcd28f873a6..bebd0a1c6e01 100644
--- a/distribution/docker/Dockerfile
+++ b/distribution/docker/Dockerfile
@@ -36,7 +36,7 @@ RUN export DEBIAN_FRONTEND=noninteractive \
COPY . /src
WORKDIR /src
RUN --mount=type=cache,target=/root/.m2 if [ "$BUILD_FROM_SOURCE" = "true" ]; then \
- mvn -B -ff -q dependency:go-offline \
+ mvn -B -ff -q \
install \
-Pdist,bundle-contrib-exts \
-Pskip-static-checks,skip-tests \
diff --git a/distribution/docker/druid.sh b/distribution/docker/druid.sh
index 1bc2f4f071ab..1f0268827439 100755
--- a/distribution/docker/druid.sh
+++ b/distribution/docker/druid.sh
@@ -148,7 +148,7 @@ env | grep ^druid_ | while read evar;
do
# Can't use IFS='=' to parse since var might have = in it (e.g. password)
val=$(echo "$evar" | sed -e 's?[^=]*=??')
- var=$(echo "$evar" | sed -e 's?^\([^=]*\)=.*?\1?g' -e 's?_?.?g')
+ var=$(echo "$evar" | sed -e 's?^\([^=]*\)=.*?\1?g' -e 's?__?%UNDERSCORE%?g' -e 's?_?.?g' -e 's?%UNDERSCORE%?_?g')
setKey $SERVICE "$var" "$val"
done
diff --git a/distribution/docker/peon.sh b/distribution/docker/peon.sh
index b5ec89ea8d46..c98c3d30a303 100755
--- a/distribution/docker/peon.sh
+++ b/distribution/docker/peon.sh
@@ -107,7 +107,7 @@ env | grep ^druid_ | while read evar;
do
# Can't use IFS='=' to parse since var might have = in it (e.g. password)
val=$(echo "$evar" | sed -e 's?[^=]*=??')
- var=$(echo "$evar" | sed -e 's?^\([^=]*\)=.*?\1?g' -e 's?_?.?g')
+ var=$(echo "$evar" | sed -e 's?^\([^=]*\)=.*?\1?g' -e 's?__?%UNDERSCORE%?g' -e 's?_?.?g' -e 's?%UNDERSCORE%?_?g')
setKey $SERVICE "$var" "$val"
done
diff --git a/docs/development/experimental-features.md b/docs/development/experimental-features.md
deleted file mode 100644
index 302db71f7ecd..000000000000
--- a/docs/development/experimental-features.md
+++ /dev/null
@@ -1,59 +0,0 @@
----
-id: experimental-features
-title: "Experimental features"
----
-
-
-
-The following features are marked [experimental](./experimental.md) in the Druid docs.
-
-This document includes each page that mentions an experimental feature. To graduate a feature, remove all mentions of its experimental status on all relevant pages.
-
-Note that this document does not track the status of contrib extensions, all of which are considered experimental.
-
-## SQL-based ingestion
-
-- [SQL-based ingestion](../multi-stage-query/index.md)
-- [SQL-based ingestion concepts](../multi-stage-query/concepts.md)
-- [SQL-based ingestion and multi-stage query task API](../api-reference/sql-ingestion-api.md)
-
-## Indexer service
-
-- [Indexer service](../design/indexer.md)
-- [Data server](../design/architecture.md#indexer-service-optional)
-
-## Kubernetes
-
-- [Kubernetes](../development/extensions-core/kubernetes.md)
-
-## Segment locking
-
-- [Configuration reference](../configuration/index.md#overlord-operations)
-- [Task reference](../ingestion/tasks.md#locking)
-- [Design](../design/storage.md#availability-and-consistency)
-
-## Front coding
-
-- [Ingestion spec reference](../ingestion/ingestion-spec.md#front-coding)
-
-## Other configuration properties
-
-- [Configuration reference](../configuration/index.md)
- - `CLOSED_SEGMENTS_SINKS` mode
diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md
index 27290a9bee59..4d79319ab0b2 100644
--- a/docs/development/extensions-contrib/k8s-jobs.md
+++ b/docs/development/extensions-contrib/k8s-jobs.md
@@ -90,7 +90,10 @@ The custom template pod adapter allows you to specify a pod template file per ta
The base pod template must be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.base: /path/to/basePodSpec.yaml`
-Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{taskType}: /path/to/taskSpecificPodSpec.yaml` where {taskType} is the name of the task type i.e `index_parallel`
+Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{taskType}: /path/to/taskSpecificPodSpec.yaml` where {taskType} is the name of the task type i.e `index_parallel`.
+
+If you are trying to use the default image's environment variable parsing feature to set runtime properties, you need to add a extra escape underscore when specifying pod templates.
+e.g. set the environment variable `druid_indexer_runner_k8s_podTemplate_index__parallel` when setting `druid.indxer.runner.k8s.podTemplate.index_parallel`
The following is an example Pod Template that uses the regular druid docker image.
```
diff --git a/docs/querying/sql.md b/docs/querying/sql.md
index 6da894e4e67d..a539674933db 100644
--- a/docs/querying/sql.md
+++ b/docs/querying/sql.md
@@ -90,7 +90,7 @@ documentation.
## PIVOT
:::info
-The PIVOT operator is an [experimental feature](../development/experimental-features.md).
+The PIVOT operator is an [experimental feature](../development/experimental.md).
:::
The PIVOT operator carries out an aggregation and transforms rows into columns in the output.
@@ -147,7 +147,7 @@ LIMIT 15
## UNPIVOT
:::info
-The UNPIVOT operator is an [experimental feature](../development/experimental-features.md).
+The UNPIVOT operator is an [experimental feature](../development/experimental.md).
:::
The UNPIVOT operator transforms existing column values into rows.
diff --git a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java
index 74a412a203e6..a9f5e14fbeaa 100644
--- a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java
+++ b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java
@@ -51,8 +51,11 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -128,8 +131,13 @@ public void tearDown()
new TestEvent()
);
+ /**
+ * Unit test to validate the handling of {@link ServiceMetricEvent}s.
+ * Only {@link KafkaEmitterConfig.EventType}s is subscribed in the config, so the expectation is that the
+ * events are emitted without any drops.
+ */
@Test(timeout = 10_000)
- public void testServiceMetricEvents() throws JsonProcessingException, InterruptedException
+ public void testServiceMetricEvents() throws InterruptedException, JsonProcessingException
{
final KafkaEmitterConfig kafkaEmitterConfig = new KafkaEmitterConfig(
"",
@@ -149,12 +157,12 @@ public void testServiceMetricEvents() throws JsonProcessingException, Interrupte
final List inputEvents = flattenEvents(SERVICE_METRIC_EVENTS);
final CountDownLatch eventLatch = new CountDownLatch(inputEvents.size());
- final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
+ final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
inputEvents,
kafkaEmitterConfig.getClusterName(),
kafkaEmitterConfig.getExtraDimensions()
);
- final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
+ final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
emitEvents(kafkaEmitter, inputEvents, eventLatch);
@@ -167,8 +175,14 @@ public void testServiceMetricEvents() throws JsonProcessingException, Interrupte
Assert.assertEquals(0, kafkaEmitter.getInvalidLostCount());
}
+ /**
+ * Unit test to validate the handling of all event types, including {@link ServiceMetricEvent},
+ * {@link AlertEvent}, {@link org.apache.druid.server.log.RequestLogEvent}, and {@link SegmentMetadataEvent}.
+ * All {@link KafkaEmitterConfig.EventType}s are subscribed in the config, so the expectation is that all the
+ * events are emitted without any drops.
+ */
@Test(timeout = 10_000)
- public void testAllEvents() throws JsonProcessingException, InterruptedException
+ public void testAllEvents() throws InterruptedException, JsonProcessingException
{
final KafkaEmitterConfig kafkaEmitterConfig = new KafkaEmitterConfig(
"",
@@ -193,12 +207,12 @@ public void testAllEvents() throws JsonProcessingException, InterruptedException
);
final CountDownLatch eventLatch = new CountDownLatch(inputEvents.size());
- final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
+ final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
inputEvents,
kafkaEmitterConfig.getClusterName(),
kafkaEmitterConfig.getExtraDimensions()
);
- final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
+ final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
emitEvents(kafkaEmitter, inputEvents, eventLatch);
@@ -212,8 +226,13 @@ public void testAllEvents() throws JsonProcessingException, InterruptedException
}
+ /**
+ * Unit test to validate the handling of the default event types - {@link ServiceMetricEvent} and {@link AlertEvent}.
+ * The default event types (alerts and metrics) are subscribed in the config, so the expectation is that both input
+ * event types should be emitted without any drops.
+ */
@Test(timeout = 10_000)
- public void testDefaultEvents() throws JsonProcessingException, InterruptedException
+ public void testDefaultEvents() throws InterruptedException, JsonProcessingException
{
final KafkaEmitterConfig kafkaEmitterConfig = new KafkaEmitterConfig(
"",
@@ -236,12 +255,12 @@ public void testDefaultEvents() throws JsonProcessingException, InterruptedExcep
);
final CountDownLatch eventLatch = new CountDownLatch(inputEvents.size());
- final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
+ final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
inputEvents,
kafkaEmitterConfig.getClusterName(),
kafkaEmitterConfig.getExtraDimensions()
);
- final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
+ final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
emitEvents(kafkaEmitter, inputEvents, eventLatch);
@@ -254,8 +273,14 @@ public void testDefaultEvents() throws JsonProcessingException, InterruptedExcep
Assert.assertEquals(0, kafkaEmitter.getInvalidLostCount());
}
+ /**
+ * Unit test to validate the handling of all valid event types, including {@link ServiceMetricEvent},
+ * {@link AlertEvent}, {@link org.apache.druid.server.log.RequestLogEvent}, and {@link SegmentMetadataEvent}.
+ * Only alerts are subscribed in the config, so the expectation is that only alert events
+ * should be emitted, and everything else should be dropped.
+ */
@Test(timeout = 10_000)
- public void testAlertsPlusUnsubscribedEvents() throws JsonProcessingException, InterruptedException
+ public void testAlertsPlusUnsubscribedEvents() throws InterruptedException, JsonProcessingException
{
final KafkaEmitterConfig kafkaEmitterConfig = new KafkaEmitterConfig(
"",
@@ -282,12 +307,12 @@ public void testAlertsPlusUnsubscribedEvents() throws JsonProcessingException, I
final CountDownLatch eventLatch = new CountDownLatch(ALERT_EVENTS.size());
- final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
+ final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
ALERT_EVENTS,
kafkaEmitterConfig.getClusterName(),
kafkaEmitterConfig.getExtraDimensions()
);
- final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
+ final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
emitEvents(kafkaEmitter, inputEvents, eventLatch);
@@ -302,8 +327,17 @@ public void testAlertsPlusUnsubscribedEvents() throws JsonProcessingException, I
Assert.assertEquals(REQUEST_LOG_EVENTS.size(), kafkaEmitter.getRequestLostCount());
}
+ /**
+ * Similar to {@link #testAllEvents()}, this test configures all event feeds to emit to the same topic.
+ *
+ * Unit test to validate the handling of all valid event types, including {@link ServiceMetricEvent},
+ * {@link AlertEvent}, {@link org.apache.druid.server.log.RequestLogEvent}, and {@link SegmentMetadataEvent}.
+ * All {@link KafkaEmitterConfig.EventType}s are subscribed to the same topic in the config, so the expectation
+ * is that all input events are emitted without any drops.
+ *
+ */
@Test(timeout = 10_000)
- public void testAllEventsWithCommonTopic() throws JsonProcessingException, InterruptedException
+ public void testAllEventsWithCommonTopic() throws InterruptedException, JsonProcessingException
{
final KafkaEmitterConfig kafkaEmitterConfig = new KafkaEmitterConfig(
"",
@@ -329,12 +363,12 @@ public void testAllEventsWithCommonTopic() throws JsonProcessingException, Inter
final CountDownLatch eventLatch = new CountDownLatch(inputEvents.size());
- final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
+ final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
inputEvents,
kafkaEmitterConfig.getClusterName(),
kafkaEmitterConfig.getExtraDimensions()
);
- final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
+ final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
emitEvents(kafkaEmitter, inputEvents, eventLatch);
@@ -347,8 +381,14 @@ public void testAllEventsWithCommonTopic() throws JsonProcessingException, Inter
Assert.assertEquals(0, kafkaEmitter.getInvalidLostCount());
}
+ /**
+ * Unit test to validate the handling of {@link ServiceMetricEvent}s and {@link TestEvent}s.
+ * The default event types (alerts and metrics) are subscribed in the config, so the expectation is that only
+ * {@link ServiceMetricEvent} is expected to be emitted, while dropping all unknown {@link TestEvent}s.
+ *
+ */
@Test(timeout = 10_000)
- public void testUnknownEvents() throws JsonProcessingException, InterruptedException
+ public void testUnknownEvents() throws InterruptedException, JsonProcessingException
{
final KafkaEmitterConfig kafkaEmitterConfig = new KafkaEmitterConfig(
"",
@@ -372,12 +412,12 @@ public void testUnknownEvents() throws JsonProcessingException, InterruptedExcep
final CountDownLatch eventLatch = new CountDownLatch(SERVICE_METRIC_EVENTS.size());
- final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
+ final Map> feedToExpectedEvents = trackExpectedEventsPerFeed(
SERVICE_METRIC_EVENTS,
kafkaEmitterConfig.getClusterName(),
kafkaEmitterConfig.getExtraDimensions()
);
- final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
+ final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
emitEvents(kafkaEmitter, inputEvents, eventLatch);
@@ -390,6 +430,13 @@ public void testUnknownEvents() throws JsonProcessingException, InterruptedExcep
Assert.assertEquals(UNKNOWN_EVENTS.size(), kafkaEmitter.getInvalidLostCount());
}
+ /**
+ * Unit test to validate the handling of {@link ServiceMetricEvent}s when the Kafka emitter queue, which buffers up events
+ * becomes full. The queue size in the config is set via {@code buffer.memory} and is computed from
+ * the input events using {@code bufferEventsDrop}. The default event types (alerts and metrics) are subscribed in
+ * the config, so the expectation is that all {@link ServiceMetricEvent}s up to {@code n - bufferEventsDrop} will be
+ * emitted, {@code n} being the total number of input events, while dropping the last {@code bufferEventsDrop} events.
+ */
@Test(timeout = 10_000)
public void testDropEventsWhenQueueFull() throws JsonProcessingException, InterruptedException
{
@@ -398,7 +445,7 @@ public void testDropEventsWhenQueueFull() throws JsonProcessingException, Interr
);
final ImmutableMap extraDimensions = ImmutableMap.of("clusterId", "cluster-101");
- final Map> feedToAllEventsBeforeDrop = trackExpectedEventsPerFeed(
+ final Map> feedToAllEventsBeforeDrop = trackExpectedEventsPerFeed(
inputEvents,
null,
extraDimensions
@@ -424,15 +471,15 @@ public void testDropEventsWhenQueueFull() throws JsonProcessingException, Interr
// we should track the minimum buffer size per feed, compute the global maximum across all the feeds and prune the
// expected set of events accordingly. For the sake of testing simplicity, we skip that for now.
int totalBufferSize = 0;
- for (final List feedEvents : feedToAllEventsBeforeDrop.values()) {
+ for (final List feedEvents : feedToAllEventsBeforeDrop.values()) {
for (int idx = 0; idx < feedEvents.size() - bufferEventsDrop; idx++) {
- totalBufferSize += feedEvents.get(idx).getBytes(StandardCharsets.UTF_8).length;
+ totalBufferSize += MAPPER.writeValueAsString(feedEvents.get(idx)).getBytes(StandardCharsets.UTF_8).length;
}
}
- final Map> feedToExpectedEvents = new HashMap<>();
- for (final Map.Entry> expectedEvent : feedToAllEventsBeforeDrop.entrySet()) {
- List expectedEvents = expectedEvent.getValue();
+ final Map> feedToExpectedEvents = new HashMap<>();
+ for (final Map.Entry> expectedEvent : feedToAllEventsBeforeDrop.entrySet()) {
+ List expectedEvents = expectedEvent.getValue();
feedToExpectedEvents.put(expectedEvent.getKey(), expectedEvents.subList(0, expectedEvents.size() - bufferEventsDrop));
}
@@ -452,7 +499,7 @@ public void testDropEventsWhenQueueFull() throws JsonProcessingException, Interr
final KafkaEmitter kafkaEmitter = initKafkaEmitter(kafkaEmitterConfig);
final CountDownLatch eventLatch = new CountDownLatch(inputEvents.size() - bufferEventsDrop);
- final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
+ final Map> feedToActualEvents = trackActualEventsPerFeed(eventLatch);
emitEvents(kafkaEmitter, inputEvents, eventLatch);
@@ -509,18 +556,20 @@ private List flattenEvents(List... eventLists)
return flattenedList;
}
- private Map> trackActualEventsPerFeed(
+ private Map> trackActualEventsPerFeed(
final CountDownLatch eventLatch
)
{
- final Map> feedToActualEvents = new HashMap<>();
+
+ // A concurrent hashmap because the producer callback can trigger concurrently and can override the map initialization
+ final ConcurrentHashMap> feedToActualEvents = new ConcurrentHashMap<>();
when(producer.send(any(), any())).then((invocation) -> {
final ProducerRecord, ?> producerRecord = invocation.getArgument(0);
final String value = String.valueOf(producerRecord.value());
final EventMap eventMap = MAPPER.readValue(value, EventMap.class);
feedToActualEvents.computeIfAbsent(
(String) eventMap.get("feed"), k -> new ArrayList<>()
- ).add(value);
+ ).add(eventMap);
eventLatch.countDown();
return null;
@@ -528,38 +577,37 @@ private Map> trackActualEventsPerFeed(
return feedToActualEvents;
}
- private Map> trackExpectedEventsPerFeed(
+ private Map> trackExpectedEventsPerFeed(
final List events,
final String clusterName,
final Map extraDimensions
) throws JsonProcessingException
{
- final Map> feedToExpectedEvents = new HashMap<>();
+ final Map> feedToExpectedEvents = new HashMap<>();
for (final Event event : events) {
- final EventMap eventMap = event.toMap();
+ final EventMap eventMap = MAPPER.readValue(MAPPER.writeValueAsString(event.toMap()), EventMap.class);
eventMap.computeIfAbsent("clusterName", k -> clusterName);
if (extraDimensions != null) {
eventMap.putAll(extraDimensions);
}
feedToExpectedEvents.computeIfAbsent(
- event.getFeed(), k -> new ArrayList<>()).add(MAPPER.writeValueAsString(eventMap)
- );
+ event.getFeed(), k -> new ArrayList<>()).add(eventMap);
}
return feedToExpectedEvents;
}
private void validateEvents(
- final Map> feedToExpectedEvents,
- final Map> feedToActualEvents
+ final Map> feedToExpectedEvents,
+ final Map> feedToActualEvents
)
{
Assert.assertEquals(feedToExpectedEvents.size(), feedToActualEvents.size());
- for (final Map.Entry> actualEntry : feedToActualEvents.entrySet()) {
+ for (final Map.Entry> actualEntry : feedToActualEvents.entrySet()) {
final String feed = actualEntry.getKey();
- final List actualEvents = actualEntry.getValue();
- final List expectedEvents = feedToExpectedEvents.get(feed);
- Assert.assertEquals(expectedEvents, actualEvents);
+ final List actualEvents = actualEntry.getValue();
+ final List expectedEvents = feedToExpectedEvents.get(feed);
+ assertThat(actualEvents, containsInAnyOrder(expectedEvents.toArray(new Map[0])));
}
}
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 5a6a1954fb4d..68549f5412bf 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
@@ -221,7 +221,7 @@ protected void emitTaskStateMetrics(KubernetesPeonLifecycle.State state, String
synchronized (tasks) {
workItem = tasks.get(taskId);
if (workItem == null) {
- log.error("Task [%s] disappeared", taskId);
+ log.warn("Task [%s] disappeared. This could happen if the task was canceled or if it crashed.", taskId);
return;
}
}
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 ef0509a673f8..87a27688902b 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
@@ -20,10 +20,6 @@
package org.apache.druid.k8s.overlord.taskadapter;
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.ImmutableMap;
import com.google.common.collect.Lists;
import io.fabric8.kubernetes.api.model.EnvVar;
@@ -60,10 +56,12 @@
import java.nio.file.Files;
import java.util.Collection;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
+import java.util.Set;
/**
* A PodTemplate {@link TaskAdapter} to transform tasks to kubernetes jobs and kubernetes pods to tasks
@@ -84,7 +82,9 @@ public class PodTemplateTaskAdapter implements TaskAdapter
public static final String TYPE = "customTemplateAdapter";
private static final Logger log = new Logger(PodTemplateTaskAdapter.class);
- private static final String TASK_PROPERTY = IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX + ".k8s.podTemplate.%s";
+
+
+ private static final String TASK_PROPERTY = IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX + ".k8s.podTemplate.";
private final KubernetesTaskRunnerConfig taskRunnerConfig;
private final TaskConfig taskConfig;
@@ -212,37 +212,46 @@ public K8sTaskId getTaskId(Job from)
private HashMap initializePodTemplates(Properties properties)
{
- HashMap podTemplateMap = new HashMap<>();
- Optional basePodTemplate = loadPodTemplate("base", properties);
- if (!basePodTemplate.isPresent()) {
- throw new IAE("Pod template task adapter requires a base pod template to be specified");
+ Set taskAdapterTemplateKeys = getTaskAdapterTemplates(properties);
+ if (!taskAdapterTemplateKeys.contains("base")) {
+ throw new IAE("Pod template task adapter requires a base pod template to be specified under druid.indexer.runner.k8s.podTemplate.base");
}
- podTemplateMap.put("base", basePodTemplate.get());
- MapperConfig config = mapper.getDeserializationConfig();
- AnnotatedClass cls = AnnotatedClassResolver.resolveWithoutSuperTypes(config, Task.class);
- Collection taskSubtypes = mapper.getSubtypeResolver().collectAndResolveSubtypesByClass(config, cls);
- for (NamedType namedType : taskSubtypes) {
- String taskType = namedType.getName();
- Optional template = loadPodTemplate(taskType, properties);
- template.ifPresent(podTemplate -> podTemplateMap.put(taskType, podTemplate));
+ HashMap podTemplateMap = new HashMap<>();
+ for (String taskAdapterTemplateKey : taskAdapterTemplateKeys) {
+ Optional template = loadPodTemplate(taskAdapterTemplateKey, properties);
+ template.ifPresent(podTemplate -> podTemplateMap.put(taskAdapterTemplateKey, podTemplate));
}
return podTemplateMap;
}
+ private static Set getTaskAdapterTemplates(Properties properties)
+ {
+ Set taskAdapterTemplates = new HashSet<>();
+
+ for (String runtimeProperty : properties.stringPropertyNames()) {
+ if (runtimeProperty.startsWith(TASK_PROPERTY)) {
+ String[] taskAdapterPropertyPaths = runtimeProperty.split("\\.");
+ taskAdapterTemplates.add(taskAdapterPropertyPaths[taskAdapterPropertyPaths.length - 1]);
+ }
+ }
+
+ return taskAdapterTemplates;
+ }
+
private Optional loadPodTemplate(String key, Properties properties)
{
- String property = StringUtils.format(TASK_PROPERTY, key);
+ String property = TASK_PROPERTY + key;
String podTemplateFile = properties.getProperty(property);
if (podTemplateFile == null) {
- log.debug("Pod template file not specified for [%s]", key);
- return Optional.empty();
+ throw new IAE("Pod template file not specified for [%s]", property);
+
}
try {
return Optional.of(Serialization.unmarshal(Files.newInputStream(new File(podTemplateFile).toPath()), PodTemplate.class));
}
catch (Exception e) {
- throw new ISE(e, "Failed to load pod template file for [%s] at [%s]", property, podTemplateFile);
+ throw new IAE(e, "Failed to load pod template file for [%s] at [%s]", property, podTemplateFile);
}
}
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 74dfacd1a327..1796bb2b4396 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
@@ -23,6 +23,8 @@
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.PodTemplateBuilder;
+import io.fabric8.kubernetes.api.model.VolumeBuilder;
import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder;
import org.apache.commons.lang.RandomStringUtils;
@@ -33,7 +35,6 @@
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.ISE;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
import org.apache.druid.k8s.overlord.common.Base64Compression;
import org.apache.druid.k8s.overlord.common.DruidK8sConstants;
@@ -93,8 +94,8 @@ public void setup()
@Test
public void test_fromTask_withoutBasePodTemplateInRuntimeProperites_raisesIAE()
{
- Assert.assertThrows(
- "Pod template task adapter requires a base pod template to be specified",
+ Exception exception = Assert.assertThrows(
+ "No base prop should throw an IAE",
IAE.class,
() -> new PodTemplateTaskAdapter(
taskRunnerConfig,
@@ -104,19 +105,20 @@ public void test_fromTask_withoutBasePodTemplateInRuntimeProperites_raisesIAE()
new Properties(),
taskLogs
));
+ Assert.assertEquals(exception.getMessage(), "Pod template task adapter requires a base pod template to be specified under druid.indexer.runner.k8s.podTemplate.base");
}
@Test
- public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_raisesISE() throws IOException
+ public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_raisesIAE() throws IOException
{
Path templatePath = Files.createFile(tempDir.resolve("empty.yaml"));
Properties props = new Properties();
props.setProperty("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString());
- Assert.assertThrows(
- "Pod template task adapter requires a base pod template to be specified",
- ISE.class,
+ Exception exception = Assert.assertThrows(
+ "Empty base pod template should throw a exception",
+ IAE.class,
() -> new PodTemplateTaskAdapter(
taskRunnerConfig,
taskConfig,
@@ -125,6 +127,9 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_r
props,
taskLogs
));
+
+ Assert.assertTrue(exception.getMessage().contains("Failed to load pod template file for"));
+
}
@Test
@@ -186,7 +191,7 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_andTlsEnabled()
}
@Test
- public void test_fromTask_withNoopPodTemplateInRuntimeProperties_withEmptyFile_raisesISE() throws IOException
+ public void test_fromTask_withNoopPodTemplateInRuntimeProperties_withEmptyFile_raisesIAE() throws IOException
{
Path baseTemplatePath = Files.createFile(tempDir.resolve("base.yaml"));
Path noopTemplatePath = Files.createFile(tempDir.resolve("noop.yaml"));
@@ -196,7 +201,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperties_withEmptyFile_r
props.setProperty("druid.indexer.runner.k8s.podTemplate.base", baseTemplatePath.toString());
props.setProperty("druid.indexer.runner.k8s.podTemplate.noop", noopTemplatePath.toString());
- Assert.assertThrows(ISE.class, () -> new PodTemplateTaskAdapter(
+ Assert.assertThrows(IAE.class, () -> new PodTemplateTaskAdapter(
taskRunnerConfig,
taskConfig,
node,
@@ -520,7 +525,51 @@ public void test_fromTask_taskSupportsQueries() throws IOException
.collect(Collectors.toList()).get(0).getValue());
}
+ @Test
+ public void test_fromTask_withIndexKafkaPodTemplateInRuntimeProperites() throws IOException
+ {
+ Path baseTemplatePath = Files.createFile(tempDir.resolve("base.yaml"));
+ mapper.writeValue(baseTemplatePath.toFile(), podTemplateSpec);
+ Path kafkaTemplatePath = Files.createFile(tempDir.resolve("kafka.yaml"));
+ PodTemplate kafkaPodTemplate = new PodTemplateBuilder(podTemplateSpec)
+ .editTemplate()
+ .editSpec()
+ .setNewVolumeLike(0, new VolumeBuilder().withName("volume").build())
+ .endVolume()
+ .endSpec()
+ .endTemplate()
+ .build();
+ mapper.writeValue(kafkaTemplatePath.toFile(), kafkaPodTemplate);
+
+ Properties props = new Properties();
+ props.setProperty("druid.indexer.runner.k8s.podTemplate.base", baseTemplatePath.toString());
+ props.setProperty("druid.indexer.runner.k8s.podTemplate.index_kafka", kafkaTemplatePath.toString());
+
+ PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter(
+ taskRunnerConfig,
+ taskConfig,
+ node,
+ mapper,
+ props,
+ taskLogs
+ );
+
+ Task kafkaTask = new NoopTask("id", "id", "datasource", 0, 0, null) {
+ @Override
+ public String getType()
+ {
+ return "index_kafka";
+ }
+ };
+
+ Task noopTask = new NoopTask("id", "id", "datasource", 0, 0, null);
+ Job actual = adapter.fromTask(kafkaTask);
+ Assert.assertEquals(1, actual.getSpec().getTemplate().getSpec().getVolumes().size(), 1);
+
+ actual = adapter.fromTask(noopTask);
+ Assert.assertEquals(0, actual.getSpec().getTemplate().getSpec().getVolumes().size(), 1);
+ }
private void assertJobSpecsEqual(Job actual, Job expected) throws IOException
{
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
index 231705418f53..c2aee78b3cbd 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
@@ -79,6 +79,8 @@ public void testSerdeWithDefaults() throws Exception
Assert.assertNull(config.getTopicPattern());
Assert.assertEquals(1, (int) config.getReplicas());
Assert.assertEquals(1, (int) config.getTaskCount());
+ Assert.assertNull(config.getStopTaskCount());
+ Assert.assertEquals((int) config.getTaskCount(), config.getMaxAllowedStops());
Assert.assertEquals(Duration.standardMinutes(60), config.getTaskDuration());
Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
Assert.assertEquals(100, config.getPollTimeout());
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java
index 4de35cf5e5d0..9f5c0bf75042 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java
@@ -62,6 +62,8 @@ public void testSerdeWithDefaults() throws Exception
Assert.assertEquals(KinesisRegion.US_EAST_1.getEndpoint(), config.getEndpoint());
Assert.assertEquals(1, (int) config.getReplicas());
Assert.assertEquals(1, (int) config.getTaskCount());
+ Assert.assertNull(config.getStopTaskCount());
+ Assert.assertEquals((int) config.getTaskCount(), config.getMaxAllowedStops());
Assert.assertEquals(Duration.standardMinutes(60), config.getTaskDuration());
Assert.assertEquals(Duration.standardSeconds(5), config.getStartDelay());
Assert.assertEquals(Duration.standardSeconds(30), config.getPeriod());
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java
index da98254e4de5..503e17755cf0 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java
@@ -24,6 +24,7 @@
import com.google.common.collect.ImmutableMap;
import com.google.inject.Injector;
import com.google.inject.Module;
+import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.msq.exec.WorkerMemoryParameters;
@@ -215,4 +216,18 @@ public void testJoinMultipleTablesWithWhereCondition()
)
.run();
}
+
+ @Override
+ public void testFilterParseLongNullable()
+ {
+ // this isn't really correct in default value mode, the result should be ImmutableList.of(new Object[]{0L})
+ // but MSQ is missing default aggregator values in empty group results. this override can be removed when this
+ // is fixed
+ testBuilder().queryContext(QUERY_CONTEXT_DEFAULT)
+ .sql("select count(*) from druid.foo where parse_long(dim1, 10) is null")
+ .expectedResults(
+ NullHandling.sqlCompatible() ? ImmutableList.of(new Object[]{4L}) : ImmutableList.of()
+ )
+ .run();
+ }
}
diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java
index c7338e1a28fd..45f92ae0a5ea 100644
--- a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java
+++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java
@@ -314,8 +314,11 @@ public void testJsonPathFunctions() throws IOException
//deviation of [7,8,9] is 1/3, stddev is sqrt(1/3), approximately 0.8165
Assert.assertEquals(0.8165, Double.parseDouble(Iterables.getOnlyElement(row.getDimension("stddev"))), 0.0001);
- //append is not supported
- Assert.assertEquals(Collections.emptyList(), row.getDimension("append"));
+ // we do not support json-path append function for ORC format (see https://github.com/apache/druid/pull/11722)
+ Exception exception = Assert.assertThrows(UnsupportedOperationException.class, () -> {
+ row.getDimension("append");
+ });
+ Assert.assertEquals("Unused", exception.getMessage());
}
Assert.assertEquals(1, actualRowCount);
}
diff --git a/helm/druid/README.md b/helm/druid/README.md
deleted file mode 100644
index 85c72ef2a818..000000000000
--- a/helm/druid/README.md
+++ /dev/null
@@ -1,248 +0,0 @@
-
-
-# Apache Druid
-
-[Apache Druid](https://druid.apache.org/) is a high performance real-time analytics database.
-
-## Dependency Update
-
-Before you install the Druid Chart, update the dependencies :
-```bash
-helm dependency update helm/druid
-```
-
-## Install Chart
-
-To install the Druid Chart into your Kubernetes cluster :
-
-```bash
-helm install druid helm/druid --namespace dev --create-namespace
-```
-
-After installation succeeds, you can get a status of Chart
-
-```bash
-helm status druid -n dev
-```
-
-If you want to delete your Chart, use this command:
-
-```bash
-helm uninstall druid -n dev
-```
-
-### Helm ingresses
-
-The Chart provides ingress configuration to allow customization the installation by adapting
-the `values.yaml` depending on your setup.
-Please read the comments in the `values.yaml` file for more details on how to configure your reverse
-proxy or load balancer.
-
-### Chart Prefix
-
-This Helm automatically prefixes all names using the release name to avoid collisions.
-
-### URL prefix
-
-This chart exposes 6 endpoints:
-
-- Druid Overlord
-- Druid Broker
-- Druid Coordinator
-- Druid Historical
-- Druid Middle Manager
-- Druid Router
-
-### Druid configuration
-
-Druid configuration can be changed by using environment variables from Docker image.
-
-See the
-[Druid Docker entry point](https://github.com/apache/druid/blob/master/distribution/docker/druid.sh)
-for more informations
-
-### Middle Manager and Historical Statefulset
-
-Middle Managers and Historicals uses StatefulSet. Persistence is enabled by default.
-
-## Helm chart Configuration
-
-The following table lists the configurable parameters of the Druid chart and their default values.
-
-| Parameter | Description | Default |
-|------------------------------------------|---------------------------------------------------------|--------------------------------------------|
-| `image.repository` | container image name | `apache/druid` |
-| `image.tag` | container image tag | `0.19.0` |
-| `image.pullPolicy` | container pull policy | `IfNotPresent` |
-| `image.pullSecrets` | image pull secrest for private repositoty | `[]` |
-| `configMap.enabled` | enable druid configuration as configmap | `true` |
-| `configVars` | druid configuration variables for all components | `` |
-| `gCloudStorage.enabled` | look for secret to set google cloud credentials | `false` |
-| `gCloudStorage.secretName` | secretName to be mounted as google cloud credentials | `false` |
-| `rbac.create` | Create roles and roleBindings for service Accounts | `true` |
-| `broker.enabled` | enable broker | `true` |
-| `broker.name` | broker component name | `broker` |
-| `broker.replicaCount` | broker node replicas (deployment) | `1` |
-| `broker.port` | port of broker component | `8082` |
-| `broker.serviceAccount.create` | Create a service account for broker service | `true` |
-| `broker.serviceAccount.name` | Service account name | Derived from the name of service |
-| `broker.serviceAccount.annotations` | Annotations applied to created service account | `{}` |
-| `broker.serviceAccount.labels` | Labels applied to created service account | `{}` |
-| `broker.serviceAccount.automountServiceAccountToken` | Automount API credentials for the Service Account | `true` |
-| `broker.serviceType` | service type for service | `ClusterIP` |
-| `broker.resources` | broker node resources requests & limits | `{}` |
-| `broker.podAnnotations` | broker deployment annotations | `{}` |
-| `broker.nodeSelector` | Node labels for broker pod assignment | `{}` |
-| `broker.tolerations` | broker tolerations | `[]` |
-| `broker.config` | broker private config such as `JAVA_OPTS` | |
-| `broker.affinity` | broker affinity policy | `{}` |
-| `broker.ingress.enabled` | enable ingress | `false` |
-| `broker.ingress.hosts` | hosts for the broker api | `[ "chart-example.local" ]` |
-| `broker.ingress.path` | path of the broker api | `/` |
-| `broker.ingress.annotations` | annotations for the broker api ingress | `{}` |
-| `broker.ingress.tls` | TLS configuration for the ingress | `[]` |
-| `coordinator.enabled` | enable coordinator | `true` |
-| `coordinator.name` | coordinator component name | `coordinator` |
-| `coordinator.replicaCount` | coordinator node replicas (deployment) | `1` |
-| `coordinator.port` | port of coordinator component | `8081` |
-| `coordinator.serviceType` | service type for service | `ClusterIP` |
-| `coordinator.serviceAccount.create` | Create a service account for coordinator service | `true` |
-| `coordinator.serviceAccount.name` | Service account name | Derived from the name of service |
-| `coordinator.serviceAccount.annotations` | Annotations applied to created service account | `{}` |
-| `coordinator.serviceAccount.labels` | Labels applied to created service account | `{}` |
-| `coordinator.serviceAccount.automountServiceAccountToken` | Automount API credentials for the Service Account | `true` |
-| `coordinator.resources` | coordinator node resources requests & limits | `{}` |
-| `coordinator.podAnnotations` | coordinator Deployment annotations | `{}` |
-| `coordinator.nodeSelector` | node labels for coordinator pod assignment | `{}` |
-| `coordinator.tolerations` | coordinator tolerations | `[]` |
-| `coordinator.config` | coordinator private config such as `JAVA_OPTS` | |
-| `coordinator.affinity` | coordinator affinity policy | `{}` |
-| `coordinator.ingress.enabled` | enable ingress | `false` |
-| `coordinator.ingress.hosts` | hosts for the coordinator api | `[ "chart-example.local" ]` |
-| `coordinator.ingress.path` | path of the coordinator api | `/` |
-| `coordinator.ingress.annotations` | annotations for the coordinator api ingress | `{}` |
-| `coordinator.ingress.tls` | TLS configuration for the ingress | `[]` |
-| `overlord.enabled` | enable overlord | `false` |
-| `overlord.name` | overlord component name | `overlord` |
-| `overlord.replicaCount` | overlord node replicas (deployment) | `1` |
-| `overlord.port` | port of overlord component | `8081` |
-| `overlord.serviceType` | service type for service | `ClusterIP` |
-| `overlord.serviceAccount.create` | Create a service account for overlord service | `true` |
-| `overlord.serviceAccount.name` | Service account name | Derived from the name of service |
-| `overlord.serviceAccount.annotations` | Annotations applied to created service account | `{}` |
-| `overlord.serviceAccount.labels` | Labels applied to created service account | `{}` |
-| `overlord.serviceAccount.automountServiceAccountToken` | Automount API credentials for the Service Account | `true` |
-| `overlord.resources` | overlord node resources requests & limits | `{}` |
-| `overlord.podAnnotations` | overlord Deployment annotations | `{}` |
-| `overlord.nodeSelector` | node labels for overlord pod assignment | `{}` |
-| `overlord.tolerations` | overlord tolerations | `[]` |
-| `overlord.config` | overlord private config such as `JAVA_OPTS` | |
-| `overlord.affinity` | overlord affinity policy | `{}` |
-| `overlord.ingress.enabled` | enable ingress | `false` |
-| `overlord.ingress.hosts` | hosts for the overlord api | `[ "chart-example.local" ]` |
-| `overlord.ingress.path` | path of the overlord api | `/` |
-| `overlord.ingress.annotations` | annotations for the overlord api ingress | `{}` |
-| `overlord.ingress.tls` | TLS configuration for the ingress | `[]` |
-| `historical.enabled` | enable historical | `true` |
-| `historical.name` | historical component name | `historical` |
-| `historical.replicaCount` | historical node replicas (statefulset) | `1` |
-| `historical.port` | port of historical component | `8083` |
-| `historical.serviceType` | service type for service | `ClusterIP` |
-| `historical.serviceAccount.create` | Create a service account for historical service | `true` |
-| `historical.serviceAccount.name` | Service account name | Derived from the name of service |
-| `historical.serviceAccount.annotations` | Annotations applied to created service account | `{}` |
-| `historical.serviceAccount.labels` | Labels applied to created service account | `{}` |
-| `historical.serviceAccount.automountServiceAccountToken` | Automount API credentials for the Service Account | `true` |
-| `historical.resources` | historical node resources requests & limits | `{}` |
-| `historical.livenessProbeInitialDelaySeconds` | historical node liveness probe initial delay in seconds | `60` |
-| `historical.readinessProbeInitialDelaySeconds` | historical node readiness probe initial delay in seconds | `60` |
-| `historical.podAnnotations` | historical Deployment annotations | `{}` |
-| `historical.nodeSelector` | node labels for historical pod assignment | `{}` |
-| `historical.securityContext` | custom security context for historical containers | `{ fsGroup: 1000 }` |
-| `historical.tolerations` | historical tolerations | `[]` |
-| `historical.config` | historical node private config such as `JAVA_OPTS` | |
-| `historical.persistence.enabled` | historical persistent enabled/disabled | `true` |
-| `historical.persistence.size` | historical persistent volume size | `4Gi` |
-| `historical.persistence.storageClass` | historical persistent volume Class | `nil` |
-| `historical.persistence.accessMode` | historical persistent Access Mode | `ReadWriteOnce` |
-| `historical.antiAffinity` | historical anti-affinity policy | `soft` |
-| `historical.nodeAffinity` | historical node affinity policy | `{}` |
-| `historical.ingress.enabled` | enable ingress | `false` |
-| `historical.ingress.hosts` | hosts for the historical api | `[ "chart-example.local" ]` |
-| `historical.ingress.path` | path of the historical api | `/` |
-| `historical.ingress.annotations` | annotations for the historical api ingress | `{}` |
-| `historical.ingress.tls` | TLS configuration for the ingress | `[]` |
-| `middleManager.enabled` | enable middleManager | `true` |
-| `middleManager.name` | middleManager component name | `middleManager` |
-| `middleManager.replicaCount` | middleManager node replicas (statefulset) | `1` |
-| `middleManager.port` | port of middleManager component | `8091` |
-| `middleManager.serviceType` | service type for service | `ClusterIP` |
-| `middleManager.serviceAccount.create` | Create a service account for middleManager service | `true` |
-| `middleManager.serviceAccount.name` | Service account name | `` |
-| `middleManager.serviceAccount.annotations` | Annotations applied to created service account | `{}` |
-| `middleManager.serviceAccount.labels` | Labels applied to created service account | `{}` |
-| `middleManager.serviceAccount.automountServiceAccountToken` | Automount API credentials for the Service Account | `true` |
-| `middleManager.resources` | middleManager node resources requests & limits | `{}` |
-| `middleManager.podAnnotations` | middleManager Deployment annotations | `{}` |
-| `middleManager.nodeSelector` | Node labels for middleManager pod assignment | `{}` |
-| `middleManager.securityContext` | custom security context for middleManager containers | `{ fsGroup: 1000 }` |
-| `middleManager.tolerations` | middleManager tolerations | `[]` |
-| `middleManager.config` | middleManager private config such as `JAVA_OPTS` | |
-| `middleManager.persistence.enabled` | middleManager persistent enabled/disabled | `true` |
-| `middleManager.persistence.size` | middleManager persistent volume size | `4Gi` |
-| `middleManager.persistence.storageClass` | middleManager persistent volume Class | `nil` |
-| `middleManager.persistence.accessMode` | middleManager persistent Access Mode | `ReadWriteOnce` |
-| `middleManager.antiAffinity` | middleManager anti-affinity policy | `soft` |
-| `middleManager.nodeAffinity` | middleManager node affinity policy | `{}` |
-| `middleManager.autoscaling.enabled` | enable horizontal pod autoscaling | `false` |
-| `middleManager.autoscaling.minReplicas` | middleManager autoscaling min replicas | `2` |
-| `middleManager.autoscaling.maxReplicas` | middleManager autoscaling max replicas | `5` |
-| `middleManager.autoscaling.metrics` | middleManager autoscaling metrics | `{}` |
-| `middleManager.ingress.enabled` | enable ingress | `false` |
-| `middleManager.ingress.hosts` | hosts for the middleManager api | `[ "chart-example.local" ]` |
-| `middleManager.ingress.path` | path of the middleManager api | `/` |
-| `middleManager.ingress.annotations` | annotations for the middleManager api ingress | `{}` |
-| `middleManager.ingress.tls` | TLS configuration for the ingress | `[]` |
-| `router.enabled` | enable router | `false` |
-| `router.name` | router component name | `router` |
-| `router.replicaCount` | router node replicas (deployment) | `1` |
-| `router.port` | port of router component | `8888` |
-| `router.serviceType` | service type for service | `ClusterIP` |
-| `router.serviceAccount.create` | Create a service account for router service | `true` |
-| `router.serviceAccount.name` | Service account name | Derived from the name of service |
-| `router.serviceAccount.annotations` | Annotations applied to created service account | `{}` |
-| `router.serviceAccount.labels` | Labels applied to created service account | `{}` |
-| `router.serviceAccount.automountServiceAccountToken` | Automount API credentials for the Service Account | `true` |
-| `router.resources` | router node resources requests & limits | `{}` |
-| `router.podAnnotations` | router Deployment annotations | `{}` |
-| `router.nodeSelector` | node labels for router pod assignment | `{}` |
-| `router.tolerations` | router tolerations | `[]` |
-| `router.config` | router private config such as `JAVA_OPTS` | |
-| `router.affinity` | router affinity policy | `{}` |
-| `router.ingress.enabled` | enable ingress | `false` |
-| `router.ingress.hosts` | hosts for the router api | `[ "chart-example.local" ]` |
-| `router.ingress.path` | path of the router api | `/` |
-| `router.ingress.annotations` | annotations for the router api ingress | `{}` |
-| `router.ingress.tls` | TLS configuration for the ingress | `[]` |
-| `prometheus.enabled` | Support scraping from prometheus | `false` |
-| `prometheus.port` | expose prometheus port | `9090` |
-| `prometheus.annotation` | pods annotation to notify prometheus scraping | `{prometheus.io/scrape: "true", prometheus.io/port: "9090"}` |
-
-Full and up-to-date documentation can be found in the comments of the `values.yaml` file.
diff --git a/helm/druid/templates/NOTES.txt b/helm/druid/templates/NOTES.txt
deleted file mode 100644
index be1c96faa244..000000000000
--- a/helm/druid/templates/NOTES.txt
+++ /dev/null
@@ -1,38 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-1. Get the router URL by running these commands:
-{{- if .Values.router.ingress.enabled }}
-{{- range .Values.router.ingress.hosts }}
- http{{ if $.Values.router.ingress.tls }}s{{ end }}://{{ . }}{{ $.Values.router.ingress.path }}
-{{- end }}
-{{- else if contains "NodePort" .Values.router.serviceType }}
- export NODE_PORT=$(kubectl get --namespace {{ .Release.Namespace }} -o jsonpath="{.spec.ports[0].nodePort}" services {{ include "druid.fullname" . }})
- export NODE_IP=$(kubectl get nodes --namespace {{ .Release.Namespace }} -o jsonpath="{.items[0].status.addresses[0].address}")
- echo http://$NODE_IP:$NODE_PORT
-{{- else if contains "LoadBalancer" .Values.router.serviceType }}
- NOTE: It may take a few minutes for the LoadBalancer IP to be available.
- You can watch the status of by running 'kubectl get svc -w {{ include "druid.fullname" . }}'
- export SERVICE_IP=$(kubectl get svc --namespace {{ .Release.Namespace }} {{ include "druid.fullname" . }} -o jsonpath='{.status.loadBalancer.ingress[0].ip}')
- echo http://$SERVICE_IP:{{ .Values.router.port }}
-{{- else if contains "ClusterIP" .Values.router.serviceType }}
- export POD_NAME=$(kubectl get pods --namespace {{ .Release.Namespace }} -l "app={{ include "druid.name" . }},release={{ .Release.Name }}" -o jsonpath="{.items[0].metadata.name}")
- echo "Visit http://127.0.0.1:8080 to use your application"
- kubectl port-forward $POD_NAME 8080:{{ .Values.router.port }}
-{{- end }}
diff --git a/helm/druid/templates/_capabilities.tpl b/helm/druid/templates/_capabilities.tpl
deleted file mode 100644
index 2cccff8a2724..000000000000
--- a/helm/druid/templates/_capabilities.tpl
+++ /dev/null
@@ -1,32 +0,0 @@
-{{/*
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
- http://www.apache.org/licenses/LICENSE-2.0
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-*/}}
-
-{{/*
-Return the target Kubernetes version
-*/}}
-{{- define "capabilities.kubeVersion" -}}
-{{- default .Capabilities.KubeVersion.Version .Values.kubeVersion -}}
-{{- end -}}
-
-{{/*
-Return the appropriate apiVersion for poddisruptionbudget.
-*/}}
-{{- define "capabilities.policy.apiVersion" -}}
-{{- if semverCompare "<1.21-0" (include "capabilities.kubeVersion" .) -}}
-{{- print "policy/v1beta1" -}}
-{{- else -}}
-{{- print "policy/v1" -}}
-{{- end -}}
-{{- end -}}
diff --git a/helm/druid/templates/_helpers.tpl b/helm/druid/templates/_helpers.tpl
deleted file mode 100644
index a4ca3c9a0ee2..000000000000
--- a/helm/druid/templates/_helpers.tpl
+++ /dev/null
@@ -1,166 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{/* vim: set filetype=mustache: */}}
-{{/*
-Expand the name of the chart.
-*/}}
-{{- define "druid.name" -}}
-{{- default .Chart.Name .Values.nameOverride | trunc 63 | trimSuffix "-" -}}
-{{- end -}}
-
-{{/*
-Create a default fully qualified app name.
-We truncate at 63 chars because some Kubernetes name fields are limited to this (by the DNS naming spec).
-If release name contains chart name it will be used as a full name.
-*/}}
-{{- define "druid.fullname" -}}
-{{- if .Values.fullnameOverride -}}
-{{- .Values.fullnameOverride | trunc 63 | trimSuffix "-" -}}
-{{- else -}}
-{{- $name := default .Chart.Name .Values.nameOverride -}}
-{{- if contains $name .Release.Name -}}
-{{- .Release.Name | trunc 63 | trimSuffix "-" -}}
-{{- else -}}
-{{- printf "%s-%s" .Release.Name $name | trunc 63 | trimSuffix "-" -}}
-{{- end -}}
-{{- end -}}
-{{- end -}}
-
-{{/*
-Create chart name and version as used by the chart label.
-*/}}
-{{- define "druid.chart" -}}
-{{- printf "%s-%s" .Chart.Name .Chart.Version | replace "+" "_" | trunc 63 | trimSuffix "-" -}}
-{{- end -}}
-
-{{/*
-Create a default fully qualified historical name.
-We truncate at 63 chars because some Kubernetes name fields are limited to this (by the DNS naming spec).
-*/}}
-{{- define "druid.historical.fullname" -}}
-{{ template "druid.fullname" . }}-{{ .Values.historical.name }}
-{{- end -}}
-
-{{/*
-Create a default fully qualified middleManager name.
-We truncate at 63 chars because some Kubernetes name fields are limited to this (by the DNS naming spec).
-*/}}
-{{- define "druid.middleManager.fullname" -}}
-{{ template "druid.fullname" . }}-{{ .Values.middleManager.name }}
-{{- end -}}
-
-
-{{/*
-Create a default fully qualified broker name.
-We truncate at 63 chars because some Kubernetes name fields are limited to this (by the DNS naming spec).
-*/}}
-{{- define "druid.broker.fullname" -}}
-{{ template "druid.fullname" . }}-{{ .Values.broker.name }}
-{{- end -}}
-
-{{/*
-Create a default fully qualified overlord name.
-We truncate at 63 chars because some Kubernetes name fields are limited to this (by the DNS naming spec).
-*/}}
-{{- define "druid.overlord.fullname" -}}
-{{ template "druid.fullname" . }}-{{ .Values.overlord.name }}
-{{- end -}}
-
-{{/*
-Create a default fully qualified coordinator name.
-We truncate at 63 chars because some Kubernetes name fields are limited to this (by the DNS naming spec).
-*/}}
-{{- define "druid.coordinator.fullname" -}}
-{{ template "druid.fullname" . }}-{{ .Values.coordinator.name }}
-{{- end -}}
-
-{{/*
-Create a default fully qualified router name.
-We truncate at 63 chars because some Kubernetes name fields are limited to this (by the DNS naming spec).
-*/}}
-{{- define "druid.router.fullname" -}}
-{{ template "druid.fullname" . }}-{{ .Values.router.name }}
-{{- end -}}
-
-{{/*
-Create the name of the broker service account
-*/}}
-{{- define "druid.broker.serviceAccountName" -}}
- {{- if .Values.broker.serviceAccount.create }}
- {{- default (include "druid.broker.fullname" .) .Values.broker.serviceAccount.name }}
- {{- else }}
- {{- default "default" .Values.broker.serviceAccount.name }}
- {{- end }}
-{{- end }}
-
-{{/*
-Create the name of the historical service account
-*/}}
-{{- define "druid.historical.serviceAccountName" -}}
- {{- if .Values.historical.serviceAccount.create }}
- {{- default (include "druid.historical.fullname" .) .Values.historical.serviceAccount.name }}
- {{- else }}
- {{- default "default" .Values.historical.serviceAccount.name }}
- {{- end }}
-{{- end }}
-
-{{/*
-Create the name of the middleManager service account
-*/}}
-{{- define "druid.middleManager.serviceAccountName" -}}
- {{- if .Values.middleManager.serviceAccount.create }}
- {{- default (include "druid.middleManager.fullname" .) .Values.middleManager.serviceAccount.name }}
- {{- else }}
- {{- default "default" .Values.middleManager.serviceAccount.name }}
- {{- end }}
-{{- end }}
-
-{{/*
-Create the name of the coordinator service account
-*/}}
-{{- define "druid.coordinator.serviceAccountName" -}}
- {{- if .Values.coordinator.serviceAccount.create }}
- {{- default (include "druid.coordinator.fullname" .) .Values.coordinator.serviceAccount.name }}
- {{- else }}
- {{- default "default" .Values.coordinator.serviceAccount.name }}
- {{- end }}
-{{- end }}
-
-{{/*
-Create the name of the overlord service account
-*/}}
-{{- define "druid.overlord.serviceAccountName" -}}
- {{- if .Values.overlord.serviceAccount.create }}
- {{- default (include "druid.overlord.fullname" .) .Values.overlord.serviceAccount.name }}
- {{- else }}
- {{- default "default" .Values.overlord.serviceAccount.name }}
- {{- end }}
-{{- end }}
-
-{{/*
-Create the name of the router service account
-*/}}
-{{- define "druid.router.serviceAccountName" -}}
- {{- if .Values.router.serviceAccount.create }}
- {{- default (include "druid.router.fullname" .) .Values.router.serviceAccount.name }}
- {{- else }}
- {{- default "default" .Values.router.serviceAccount.name }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/broker/deployment.yaml b/helm/druid/templates/broker/deployment.yaml
deleted file mode 100644
index 80251de1ec15..000000000000
--- a/helm/druid/templates/broker/deployment.yaml
+++ /dev/null
@@ -1,112 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.broker.enabled -}}
-apiVersion: apps/v1
-kind: Deployment
-metadata:
- name: {{ include "druid.broker.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.broker.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- replicas: {{ .Values.broker.replicaCount }}
- selector:
- matchLabels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.broker.name }}
- template:
- metadata:
- labels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.broker.name }}
- annotations:
- druid.k8s.enablePatching: "true"
- {{- with .Values.broker.podAnnotations }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- if .Values.prometheus.enabled }}
- {{- with .Values.prometheus.annotation }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- end }}
- spec:
- {{- if .Values.broker.serviceAccount.create }}
- serviceAccountName: {{ include "druid.broker.serviceAccountName" . }}
- {{- end }}
- containers:
- - name: {{ .Chart.Name }}
- image: "{{ .Values.image.repository }}:{{ .Values.image.tag }}"
- imagePullPolicy: {{ .Values.image.pullPolicy }}
- args: [ "broker" ]
- env:
- - name: POD_NAME
- valueFrom: {fieldRef: {fieldPath: metadata.name}}
- - name: POD_NAMESPACE
- valueFrom: {fieldRef: {fieldPath: metadata.namespace}}
- {{- range $key, $val := .Values.broker.config }}
- - name: {{ $key }}
- value: {{ $val | quote }}
- {{- end}}
- envFrom:
- - configMapRef:
- name: {{ template "druid.name" . }}
- ports:
- - name: http
- containerPort: {{ .Values.broker.port }}
- protocol: TCP
- {{- if .Values.prometheus.enabled }}
- - name: prometheus
- containerPort: {{ .Values.prometheus.port }}
- protocol: TCP
- {{- end }}
- livenessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.broker.port }}
- readinessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.broker.port }}
- resources:
-{{ toYaml .Values.broker.resources | indent 12 }}
- {{- with .Values.broker.nodeSelector }}
- nodeSelector:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.broker.affinity }}
- affinity:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.broker.tolerations }}
- tolerations:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.image.pullSecrets }}
- imagePullSecrets:
-{{ toYaml . | indent 8 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/broker/ingress.yaml b/helm/druid/templates/broker/ingress.yaml
deleted file mode 100644
index df27d7e35c59..000000000000
--- a/helm/druid/templates/broker/ingress.yaml
+++ /dev/null
@@ -1,58 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.broker.ingress.enabled -}}
-{{- $fullName := include "druid.broker.fullname" . -}}
-{{- $ingressPath := .Values.broker.ingress.path -}}
-apiVersion: extensions/v1beta1
-kind: Ingress
-metadata:
- name: {{ $fullName }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.broker.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-{{- with .Values.broker.ingress.annotations }}
- annotations:
-{{ toYaml . | indent 4 }}
-{{- end }}
-spec:
-{{- if .Values.broker.ingress.tls }}
- tls:
- {{- range .Values.broker.ingress.tls }}
- - hosts:
- {{- range .hosts }}
- - {{ . | quote }}
- {{- end }}
- secretName: {{ .secretName }}
- {{- end }}
-{{- end }}
- rules:
- {{- range .Values.broker.ingress.hosts }}
- - host: {{ . | quote }}
- http:
- paths:
- - path: {{ $ingressPath }}
- backend:
- serviceName: {{ $fullName }}
- servicePort: http
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/broker/role.yaml b/helm/druid/templates/broker/role.yaml
deleted file mode 100644
index f6ff22d8b546..000000000000
--- a/helm/druid/templates/broker/role.yaml
+++ /dev/null
@@ -1,40 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.broker.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: Role
-metadata:
- name: {{ template "druid.broker.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.broker.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-rules:
- - apiGroups:
- - ""
- resources:
- - pods
- - configmaps
- verbs:
- - '*'
-{{- end }}
-
diff --git a/helm/druid/templates/broker/roleBinding.yaml b/helm/druid/templates/broker/roleBinding.yaml
deleted file mode 100644
index 0667c282cfaf..000000000000
--- a/helm/druid/templates/broker/roleBinding.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.broker.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: RoleBinding
-metadata:
- name: {{ template "druid.broker.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.broker.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-roleRef:
- apiGroup: rbac.authorization.k8s.io
- kind: Role
- name: {{ template "druid.broker.fullname" . }}
-subjects:
- - kind: ServiceAccount
- name: {{ include "druid.broker.serviceAccountName" . }}
- namespace: {{ .Release.Namespace }}
-{{- end }}
diff --git a/helm/druid/templates/broker/service.yaml b/helm/druid/templates/broker/service.yaml
deleted file mode 100644
index 4c767855340e..000000000000
--- a/helm/druid/templates/broker/service.yaml
+++ /dev/null
@@ -1,42 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.broker.enabled -}}
-apiVersion: v1
-kind: Service
-metadata:
- name: {{ include "druid.broker.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.broker.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- type: {{ .Values.broker.serviceType }}
- ports:
- - port: {{ .Values.broker.port }}
- targetPort: http
- protocol: TCP
- name: http
- selector:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.broker.name }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/broker/serviceAccount.yaml b/helm/druid/templates/broker/serviceAccount.yaml
deleted file mode 100644
index 644aa7dad8f8..000000000000
--- a/helm/druid/templates/broker/serviceAccount.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.broker.serviceAccount.create) (.Values.broker.enabled) }}
-apiVersion: v1
-kind: ServiceAccount
-automountServiceAccountToken: {{ .Values.broker.serviceAccount.automountServiceAccountToken }}
-metadata:
- name: {{ include "druid.broker.serviceAccountName" . }}
- {{- with .Values.broker.serviceAccount.annotations }}
- annotations:
- {{- toYaml . | nindent 4 }}
- {{- end }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.broker.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
- {{- with .Values.broker.serviceAccount.labels }}
- {{- toYaml . | nindent 4 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/configmap.yaml b/helm/druid/templates/configmap.yaml
deleted file mode 100644
index ef8e67af4c7f..000000000000
--- a/helm/druid/templates/configmap.yaml
+++ /dev/null
@@ -1,52 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.configMap.enabled -}}
-apiVersion: v1
-kind: ConfigMap
-metadata:
- name: {{ template "druid.name" . }}
- labels:
- app: {{ template "druid.name" . }}
- chart: {{ template "druid.chart" . }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-data:
-{{ toYaml .Values.configVars | indent 2 }}
-{{- if .Values.zookeeper.enabled }}
- druid_zk_service_host: {{ .Release.Name }}-zookeeper-headless:2181
-{{- else if .Values.zkHosts }}
- druid_zk_service_host: {{ .Values.zkHosts }}
-{{- end }}
-{{- if .Values.mysql.enabled }}
- druid_metadata_storage_type: mysql
- druid_metadata_storage_connector_connectURI: jdbc:mysql://{{ .Release.Name }}-mysql:3306/{{ .Values.mysql.mysqlDatabase}}
- druid_metadata_storage_connector_user: {{ .Values.mysql.mysqlUser }}
- druid_metadata_storage_connector_password: {{ .Values.mysql.mysqlPassword }}
-{{- end }}
-{{- if .Values.postgresql.enabled }}
- druid_metadata_storage_type: postgresql
- druid_metadata_storage_connector_connectURI: jdbc:postgresql://{{ .Release.Name }}-postgresql:{{ .Values.postgresql.service.port}}/{{ .Values.postgresql.postgresqlDatabase }}
- druid_metadata_storage_connector_user: {{ .Values.postgresql.postgresqlUsername }}
- druid_metadata_storage_connector_password: {{ .Values.postgresql.postgresqlPassword }}
-{{- end }}
-{{- if .Values.gCloudStorage.enabled }}
- GOOGLE_APPLICATION_CREDENTIALS: /var/secrets/google/key.json
-{{- end }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/coordinator/deployment.yaml b/helm/druid/templates/coordinator/deployment.yaml
deleted file mode 100644
index 66456a5112b3..000000000000
--- a/helm/druid/templates/coordinator/deployment.yaml
+++ /dev/null
@@ -1,123 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.coordinator.enabled -}}
-apiVersion: apps/v1
-kind: Deployment
-metadata:
- name: {{ include "druid.coordinator.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.coordinator.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- replicas: {{ .Values.coordinator.replicaCount }}
- selector:
- matchLabels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.coordinator.name }}
- template:
- metadata:
- labels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.coordinator.name }}
- annotations:
- druid.k8s.enablePatching: "true"
- {{- with .Values.coordinator.podAnnotations }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- if .Values.prometheus.enabled }}
- {{- with .Values.prometheus.annotation }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- end }}
- spec:
- {{- if .Values.coordinator.serviceAccount.create }}
- serviceAccountName: {{ include "druid.coordinator.serviceAccountName" . }}
- {{- end }}
- containers:
- - name: {{ .Chart.Name }}
- image: "{{ .Values.image.repository }}:{{ .Values.image.tag }}"
- imagePullPolicy: {{ .Values.image.pullPolicy }}
- args: [ "coordinator" ]
- env:
- - name: POD_NAME
- valueFrom: {fieldRef: {fieldPath: metadata.name}}
- - name: POD_NAMESPACE
- valueFrom: {fieldRef: {fieldPath: metadata.namespace}}
- {{- range $key, $val := .Values.coordinator.config }}
- - name: {{ $key }}
- value: {{ $val | quote }}
- {{- end}}
- envFrom:
- - configMapRef:
- name: {{ template "druid.name" . }}
- ports:
- - name: http
- containerPort: {{ .Values.coordinator.port }}
- protocol: TCP
- {{- if .Values.prometheus.enabled }}
- - name: prometheus
- containerPort: {{ .Values.prometheus.port }}
- protocol: TCP
- {{- end }}
- livenessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.coordinator.port }}
- readinessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.coordinator.port }}
- resources:
-{{ toYaml .Values.coordinator.resources | indent 12 }}
- volumeMounts:
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- mountPath: /var/secrets/google
- {{- end }}
- volumes:
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- secret:
- secretName: {{ .Values.gCloudStorage.secretName }}
- {{- end }}
- {{- with .Values.coordinator.nodeSelector }}
- nodeSelector:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.coordinator.affinity }}
- affinity:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.coordinator.tolerations }}
- tolerations:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.image.pullSecrets }}
- imagePullSecrets:
-{{ toYaml . | indent 8 }}
- {{- end }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/coordinator/ingress.yaml b/helm/druid/templates/coordinator/ingress.yaml
deleted file mode 100644
index 8d64ea77de17..000000000000
--- a/helm/druid/templates/coordinator/ingress.yaml
+++ /dev/null
@@ -1,58 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.coordinator.ingress.enabled -}}
-{{- $fullName := include "druid.coordinator.fullname" . -}}
-{{- $ingressPath := .Values.coordinator.ingress.path -}}
-apiVersion: extensions/v1beta1
-kind: Ingress
-metadata:
- name: {{ $fullName }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.coordinator.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-{{- with .Values.coordinator.ingress.annotations }}
- annotations:
-{{ toYaml . | indent 4 }}
-{{- end }}
-spec:
-{{- if .Values.coordinator.ingress.tls }}
- tls:
- {{- range .Values.coordinator.ingress.tls }}
- - hosts:
- {{- range .hosts }}
- - {{ . | quote }}
- {{- end }}
- secretName: {{ .secretName }}
- {{- end }}
-{{- end }}
- rules:
- {{- range .Values.coordinator.ingress.hosts }}
- - host: {{ . | quote }}
- http:
- paths:
- - path: {{ $ingressPath }}
- backend:
- serviceName: {{ $fullName }}
- servicePort: http
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/coordinator/role.yaml b/helm/druid/templates/coordinator/role.yaml
deleted file mode 100644
index c4dc3ba13671..000000000000
--- a/helm/druid/templates/coordinator/role.yaml
+++ /dev/null
@@ -1,40 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.coordinator.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: Role
-metadata:
- name: {{ template "druid.coordinator.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.coordinator.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-rules:
- - apiGroups:
- - ""
- resources:
- - pods
- - configmaps
- verbs:
- - '*'
-{{- end }}
-
diff --git a/helm/druid/templates/coordinator/roleBinding.yaml b/helm/druid/templates/coordinator/roleBinding.yaml
deleted file mode 100644
index fd60eb5d06ff..000000000000
--- a/helm/druid/templates/coordinator/roleBinding.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.coordinator.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: RoleBinding
-metadata:
- name: {{ template "druid.coordinator.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.coordinator.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-roleRef:
- apiGroup: rbac.authorization.k8s.io
- kind: Role
- name: {{ template "druid.coordinator.fullname" . }}
-subjects:
- - kind: ServiceAccount
- name: {{ include "druid.coordinator.serviceAccountName" . }}
- namespace: {{ .Release.Namespace }}
-{{- end }}
diff --git a/helm/druid/templates/coordinator/service.yaml b/helm/druid/templates/coordinator/service.yaml
deleted file mode 100644
index 953b03fec675..000000000000
--- a/helm/druid/templates/coordinator/service.yaml
+++ /dev/null
@@ -1,42 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.coordinator.enabled -}}
-apiVersion: v1
-kind: Service
-metadata:
- name: {{ include "druid.coordinator.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.coordinator.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- type: {{ .Values.coordinator.serviceType }}
- ports:
- - port: {{ .Values.coordinator.port }}
- targetPort: http
- protocol: TCP
- name: http
- selector:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.coordinator.name }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/coordinator/serviceAccount.yaml b/helm/druid/templates/coordinator/serviceAccount.yaml
deleted file mode 100644
index b7778322c787..000000000000
--- a/helm/druid/templates/coordinator/serviceAccount.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.coordinator.serviceAccount.create) (.Values.coordinator.enabled) }}
-apiVersion: v1
-kind: ServiceAccount
-automountServiceAccountToken: {{ .Values.coordinator.serviceAccount.automountServiceAccountToken }}
-metadata:
- name: {{ include "druid.coordinator.serviceAccountName" . }}
- {{- with .Values.coordinator.serviceAccount.annotations }}
- annotations:
- {{- toYaml . | nindent 4 }}
- {{- end }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.coordinator.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
- {{- with .Values.coordinator.serviceAccount.labels }}
- {{- toYaml . | nindent 4 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/historical/ingress.yaml b/helm/druid/templates/historical/ingress.yaml
deleted file mode 100644
index 466e3c5ae594..000000000000
--- a/helm/druid/templates/historical/ingress.yaml
+++ /dev/null
@@ -1,58 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.historical.ingress.enabled -}}
-{{- $fullName := include "druid.historical.fullname" . -}}
-{{- $ingressPath := .Values.historical.ingress.path -}}
-apiVersion: extensions/v1beta1
-kind: Ingress
-metadata:
- name: {{ $fullName }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.historical.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-{{- with .Values.historical.ingress.annotations }}
- annotations:
-{{ toYaml . | indent 4 }}
-{{- end }}
-spec:
-{{- if .Values.historical.ingress.tls }}
- tls:
- {{- range .Values.historical.ingress.tls }}
- - hosts:
- {{- range .hosts }}
- - {{ . | quote }}
- {{- end }}
- secretName: {{ .secretName }}
- {{- end }}
-{{- end }}
- rules:
- {{- range .Values.historical.ingress.hosts }}
- - host: {{ . | quote }}
- http:
- paths:
- - path: {{ $ingressPath }}
- backend:
- serviceName: {{ $fullName }}
- servicePort: http
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/historical/pdb.yaml b/helm/druid/templates/historical/pdb.yaml
deleted file mode 100644
index c921b1909da4..000000000000
--- a/helm/druid/templates/historical/pdb.yaml
+++ /dev/null
@@ -1,43 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.historical.podDisruptionBudget.enabled }}
-apiVersion: {{ include "capabilities.policy.apiVersion" . }}
-kind: PodDisruptionBudget
-metadata:
- labels:
- app: {{ template "druid.name" . }}
- chart: {{ .Chart.Name }}-{{ .Chart.Version }}
- component: {{ .Values.historical.name }}
- heritage: {{ .Release.Service }}
- release: {{ .Release.Name }}
- name: {{ template "druid.historical.fullname" . }}
-spec:
-{{- if .Values.historical.podDisruptionBudget.minAvailable }}
- minAvailable: {{ .Values.historical.podDisruptionBudget.minAvailable }}
-{{- end }}
-{{- if .Values.historical.podDisruptionBudget.maxUnavailable }}
- maxUnavailable: {{ .Values.historical.podDisruptionBudget.maxUnavailable }}
-{{- end }}
- selector:
- matchLabels:
- app: {{ template "druid.name" . }}
- component: {{ .Values.historical.name }}
- release: {{ .Release.Name }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/historical/role.yaml b/helm/druid/templates/historical/role.yaml
deleted file mode 100644
index b4be8cc42ffe..000000000000
--- a/helm/druid/templates/historical/role.yaml
+++ /dev/null
@@ -1,40 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.historical.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: Role
-metadata:
- name: {{ template "druid.historical.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.historical.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-rules:
- - apiGroups:
- - ""
- resources:
- - pods
- - configmaps
- verbs:
- - '*'
-{{- end }}
-
diff --git a/helm/druid/templates/historical/roleBinding.yaml b/helm/druid/templates/historical/roleBinding.yaml
deleted file mode 100644
index aa9e9fbeb352..000000000000
--- a/helm/druid/templates/historical/roleBinding.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.historical.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: RoleBinding
-metadata:
- name: {{ template "druid.historical.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.historical.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-roleRef:
- apiGroup: rbac.authorization.k8s.io
- kind: Role
- name: {{ template "druid.historical.fullname" . }}
-subjects:
- - kind: ServiceAccount
- name: {{ include "druid.historical.serviceAccountName" . }}
- namespace: {{ .Release.Namespace }}
-{{- end }}
diff --git a/helm/druid/templates/historical/service.yaml b/helm/druid/templates/historical/service.yaml
deleted file mode 100644
index 2ccbe539bc78..000000000000
--- a/helm/druid/templates/historical/service.yaml
+++ /dev/null
@@ -1,42 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.historical.enabled -}}
-apiVersion: v1
-kind: Service
-metadata:
- name: {{ include "druid.historical.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.historical.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- type: {{ .Values.historical.serviceType }}
- ports:
- - port: {{ .Values.historical.port }}
- targetPort: http
- protocol: TCP
- name: http
- selector:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.historical.name }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/historical/serviceAccount.yaml b/helm/druid/templates/historical/serviceAccount.yaml
deleted file mode 100644
index 2524503c89eb..000000000000
--- a/helm/druid/templates/historical/serviceAccount.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.historical.serviceAccount.create) (.Values.historical.enabled) }}
-apiVersion: v1
-kind: ServiceAccount
-automountServiceAccountToken: {{ .Values.historical.serviceAccount.automountServiceAccountToken }}
-metadata:
- name: {{ include "druid.historical.serviceAccountName" . }}
- {{- with .Values.historical.serviceAccount.annotations }}
- annotations:
- {{- toYaml . | nindent 4 }}
- {{- end }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.historical.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
- {{- with .Values.historical.serviceAccount.labels }}
- {{- toYaml . | nindent 4 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/historical/statefulset.yaml b/helm/druid/templates/historical/statefulset.yaml
deleted file mode 100644
index 0e12c6508c1e..000000000000
--- a/helm/druid/templates/historical/statefulset.yaml
+++ /dev/null
@@ -1,177 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.historical.enabled -}}
-apiVersion: apps/v1
-kind: StatefulSet
-metadata:
- labels:
- app: {{ template "druid.name" . }}
- chart: {{ .Chart.Name }}-{{ .Chart.Version }}
- component: {{ .Values.historical.name }}
- heritage: {{ .Release.Service }}
- release: {{ .Release.Name }}
- name: {{ template "druid.historical.fullname" . }}
-spec:
- serviceName: {{ template "druid.historical.fullname" . }}
- replicas: {{ .Values.historical.replicaCount }}
- selector:
- matchLabels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.historical.name }}
- template:
- metadata:
- labels:
- app: {{ template "druid.name" . }}
- component: {{ .Values.historical.name }}
- release: {{ .Release.Name }}
- annotations:
- druid.k8s.enablePatching: "true"
- {{- with .Values.historical.podAnnotations }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- if .Values.prometheus.enabled }}
- {{- with .Values.prometheus.annotation }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- end }}
- spec:
- {{- if or .Values.historical.antiAffinity .Values.historical.nodeAffinity }}
- affinity:
- {{- end }}
- {{- if eq .Values.historical.antiAffinity "hard" }}
- podAntiAffinity:
- requiredDuringSchedulingIgnoredDuringExecution:
- - topologyKey: "kubernetes.io/hostname"
- labelSelector:
- matchLabels:
- app: "{{ template "druid.name" . }}"
- release: "{{ .Release.Name }}"
- component: "{{ .Values.historical.name }}"
- {{- else if eq .Values.historical.antiAffinity "soft" }}
- podAntiAffinity:
- preferredDuringSchedulingIgnoredDuringExecution:
- - weight: 1
- podAffinityTerm:
- topologyKey: kubernetes.io/hostname
- labelSelector:
- matchLabels:
- app: "{{ template "druid.name" . }}"
- release: "{{ .Release.Name }}"
- component: "{{ .Values.historical.name }}"
- {{- end }}
- {{- with .Values.historical.nodeAffinity }}
- nodeAffinity:
-{{ toYaml . | indent 10 }}
- {{- end }}
-{{- if .Values.historical.nodeSelector }}
- nodeSelector:
-{{ toYaml .Values.historical.nodeSelector | indent 8 }}
-{{- end }}
-{{- if .Values.historical.securityContext }}
- securityContext:
-{{ toYaml .Values.historical.securityContext | indent 8 }}
-{{- end }}
-{{- if .Values.historical.tolerations }}
- tolerations:
-{{ toYaml .Values.historical.tolerations | indent 8 }}
-{{- end }}
-{{- if .Values.image.pullSecrets }}
- imagePullSecrets:
-{{ toYaml .Values.image.pullSecrets | indent 8 }}
-{{- end }}
- {{- if .Values.historical.serviceAccount.create }}
- serviceAccountName: {{ include "druid.historical.serviceAccountName" . }}
- {{- end }}
- containers:
- - name: druid
- args: [ "historical" ]
- env:
- - name: POD_NAME
- valueFrom: {fieldRef: {fieldPath: metadata.name}}
- - name: POD_NAMESPACE
- valueFrom: {fieldRef: {fieldPath: metadata.namespace}}
- {{- range $key, $val := .Values.historical.config }}
- - name: {{ $key }}
- value: {{ $val | quote }}
- {{- end}}
- envFrom:
- - configMapRef:
- name: {{ template "druid.name" . }}
- resources:
-{{ toYaml .Values.historical.resources | indent 12 }}
- livenessProbe:
- initialDelaySeconds: {{ .Values.historical.livenessProbeInitialDelaySeconds }}
- httpGet:
- path: /status/health
- port: {{ .Values.historical.port }}
- readinessProbe:
- initialDelaySeconds: {{ .Values.historical.readinessProbeInitialDelaySeconds }}
- httpGet:
- path: /status/health
- port: {{ .Values.historical.port }}
- image: "{{ .Values.image.repository }}:{{ .Values.image.tag }}"
- imagePullPolicy: {{ .Values.image.pullPolicy | quote }}
- ports:
- - containerPort: {{ .Values.historical.port }}
- name: http
- {{- if .Values.prometheus.enabled }}
- - name: prometheus
- containerPort: {{ .Values.prometheus.port }}
- protocol: TCP
- {{- end }}
- volumeMounts:
- - mountPath: /opt/druid/var/druid/
- name: data
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- mountPath: /var/secrets/google
- {{- end }}
- volumes:
- {{- if not .Values.historical.persistence.enabled }}
- - name: data
- emptyDir: {}
- {{- end }}
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- secret:
- secretName: {{ .Values.gCloudStorage.secretName }}
- {{- end }}
- updateStrategy:
- type: {{ .Values.historical.updateStrategy.type }}
- {{- if .Values.historical.persistence.enabled }}
- volumeClaimTemplates:
- - metadata:
- name: data
- spec:
- accessModes:
- - {{ .Values.historical.persistence.accessMode | quote }}
- {{- if .Values.historical.persistence.storageClass }}
- {{- if (eq "-" .Values.historical.persistence.storageClass) }}
- storageClassName: ""
- {{- else }}
- storageClassName: "{{ .Values.historical.persistence.storageClass }}"
- {{- end }}
- {{- end }}
- resources:
- requests:
- storage: "{{ .Values.historical.persistence.size }}"
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/middleManager/hpa.yaml b/helm/druid/templates/middleManager/hpa.yaml
deleted file mode 100644
index fcda2f2f1b2f..000000000000
--- a/helm/druid/templates/middleManager/hpa.yaml
+++ /dev/null
@@ -1,40 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.middleManager.autoscaling.enabled }}
-apiVersion: autoscaling/v2beta2
-kind: HorizontalPodAutoscaler
-metadata:
- name: {{ include "druid.middleManager.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: "{{ .Values.name }}"
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- scaleTargetRef:
- apiVersion: apps/v1
- kind: StatefulSet
- name: {{ include "druid.middleManager.fullname" . }}
- minReplicas: {{ .Values.middleManager.autoscaling.minReplicas }}
- maxReplicas: {{ .Values.middleManager.autoscaling.maxReplicas }}
- metrics:
-{{ toYaml .Values.middleManager.autoscaling.metrics | indent 4 }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/middleManager/ingress.yaml b/helm/druid/templates/middleManager/ingress.yaml
deleted file mode 100644
index 672580a2e7fc..000000000000
--- a/helm/druid/templates/middleManager/ingress.yaml
+++ /dev/null
@@ -1,58 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.middleManager.ingress.enabled -}}
-{{- $fullName := include "druid.middleManager.fullname" . -}}
-{{- $ingressPath := .Values.middleManager.ingress.path -}}
-apiVersion: extensions/v1beta1
-kind: Ingress
-metadata:
- name: {{ $fullName }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.middleManager.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-{{- with .Values.middleManager.ingress.annotations }}
- annotations:
-{{ toYaml . | indent 4 }}
-{{- end }}
-spec:
-{{- if .Values.middleManager.ingress.tls }}
- tls:
- {{- range .Values.middleManager.ingress.tls }}
- - hosts:
- {{- range .hosts }}
- - {{ . | quote }}
- {{- end }}
- secretName: {{ .secretName }}
- {{- end }}
-{{- end }}
- rules:
- {{- range .Values.middleManager.ingress.hosts }}
- - host: {{ . | quote }}
- http:
- paths:
- - path: {{ $ingressPath }}
- backend:
- serviceName: {{ $fullName }}
- servicePort: http
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/middleManager/pdb.yaml b/helm/druid/templates/middleManager/pdb.yaml
deleted file mode 100644
index 71a592979a79..000000000000
--- a/helm/druid/templates/middleManager/pdb.yaml
+++ /dev/null
@@ -1,43 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.middleManager.podDisruptionBudget.enabled }}
-apiVersion: {{ include "capabilities.policy.apiVersion" . }}
-kind: PodDisruptionBudget
-metadata:
- labels:
- app: {{ template "druid.name" . }}
- chart: {{ .Chart.Name }}-{{ .Chart.Version }}
- component: {{ .Values.middleManager.name }}
- heritage: {{ .Release.Service }}
- release: {{ .Release.Name }}
- name: {{ template "druid.middleManager.fullname" . }}
-spec:
-{{- if .Values.middleManager.podDisruptionBudget.minAvailable }}
- minAvailable: {{ .Values.middleManager.podDisruptionBudget.minAvailable }}
-{{- end }}
-{{- if .Values.middleManager.podDisruptionBudget.maxUnavailable }}
- maxUnavailable: {{ .Values.middleManager.podDisruptionBudget.maxUnavailable }}
-{{- end }}
- selector:
- matchLabels:
- app: {{ template "druid.name" . }}
- component: {{ .Values.middleManager.name }}
- release: {{ .Release.Name }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/middleManager/role.yaml b/helm/druid/templates/middleManager/role.yaml
deleted file mode 100644
index 22e0622f2ce7..000000000000
--- a/helm/druid/templates/middleManager/role.yaml
+++ /dev/null
@@ -1,40 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.middleManager.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: Role
-metadata:
- name: {{ template "druid.middleManager.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.middleManager.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-rules:
- - apiGroups:
- - ""
- resources:
- - pods
- - configmaps
- verbs:
- - '*'
-{{- end }}
-
diff --git a/helm/druid/templates/middleManager/roleBinding.yaml b/helm/druid/templates/middleManager/roleBinding.yaml
deleted file mode 100644
index fbca0994280b..000000000000
--- a/helm/druid/templates/middleManager/roleBinding.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.middleManager.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: RoleBinding
-metadata:
- name: {{ template "druid.middleManager.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.middleManager.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-roleRef:
- apiGroup: rbac.authorization.k8s.io
- kind: Role
- name: {{ template "druid.middleManager.fullname" . }}
-subjects:
- - kind: ServiceAccount
- name: {{ include "druid.middleManager.serviceAccountName" . }}
- namespace: {{ .Release.Namespace }}
-{{- end }}
diff --git a/helm/druid/templates/middleManager/service.yaml b/helm/druid/templates/middleManager/service.yaml
deleted file mode 100644
index a0817015150c..000000000000
--- a/helm/druid/templates/middleManager/service.yaml
+++ /dev/null
@@ -1,42 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.middleManager.enabled -}}
-apiVersion: v1
-kind: Service
-metadata:
- name: {{ include "druid.middleManager.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.middleManager.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- type: {{ .Values.middleManager.serviceType }}
- ports:
- - port: {{ .Values.middleManager.port }}
- targetPort: http
- protocol: TCP
- name: http
- selector:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.middleManager.name }}
-{{- end }}
\ No newline at end of file
diff --git a/helm/druid/templates/middleManager/serviceAccount.yaml b/helm/druid/templates/middleManager/serviceAccount.yaml
deleted file mode 100644
index 00dc83cb1f3e..000000000000
--- a/helm/druid/templates/middleManager/serviceAccount.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.middleManager.serviceAccount.create) (.Values.middleManager.enabled) }}
-apiVersion: v1
-kind: ServiceAccount
-automountServiceAccountToken: {{ .Values.middleManager.serviceAccount.automountServiceAccountToken }}
-metadata:
- name: {{ include "druid.middleManager.serviceAccountName" . }}
- {{- with .Values.middleManager.serviceAccount.annotations }}
- annotations:
- {{- toYaml . | nindent 4 }}
- {{- end }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.middleManager.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
- {{- with .Values.middleManager.serviceAccount.labels }}
- {{- toYaml . | nindent 4 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/middleManager/statefulset.yaml b/helm/druid/templates/middleManager/statefulset.yaml
deleted file mode 100644
index b91824e2c597..000000000000
--- a/helm/druid/templates/middleManager/statefulset.yaml
+++ /dev/null
@@ -1,177 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.middleManager.enabled -}}
-apiVersion: apps/v1
-kind: StatefulSet
-metadata:
- labels:
- app: {{ template "druid.name" . }}
- chart: {{ .Chart.Name }}-{{ .Chart.Version }}
- component: {{ .Values.middleManager.name }}
- heritage: {{ .Release.Service }}
- release: {{ .Release.Name }}
- name: {{ template "druid.middleManager.fullname" . }}
-spec:
- serviceName: {{ template "druid.middleManager.fullname" . }}
- replicas: {{ .Values.middleManager.replicaCount }}
- selector:
- matchLabels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.middleManager.name }}
- template:
- metadata:
- labels:
- app: {{ template "druid.name" . }}
- component: {{ .Values.middleManager.name }}
- release: {{ .Release.Name }}
- annotations:
- druid.k8s.enablePatching: "true"
- {{- with .Values.middleManager.podAnnotations }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- if .Values.prometheus.enabled }}
- {{- with .Values.prometheus.annotation }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- end }}
- spec:
- {{- if or .Values.middleManager.antiAffinity .Values.middleManager.nodeAffinity }}
- affinity:
- {{- end }}
- {{- if eq .Values.middleManager.antiAffinity "hard" }}
- podAntiAffinity:
- requiredDuringSchedulingIgnoredDuringExecution:
- - topologyKey: "kubernetes.io/hostname"
- labelSelector:
- matchLabels:
- app: "{{ template "druid.name" . }}"
- release: "{{ .Release.Name }}"
- component: "{{ .Values.middleManager.name }}"
- {{- else if eq .Values.middleManager.antiAffinity "soft" }}
- podAntiAffinity:
- preferredDuringSchedulingIgnoredDuringExecution:
- - weight: 1
- podAffinityTerm:
- topologyKey: kubernetes.io/hostname
- labelSelector:
- matchLabels:
- app: "{{ template "druid.name" . }}"
- release: "{{ .Release.Name }}"
- component: "{{ .Values.middleManager.name }}"
- {{- end }}
- {{- with .Values.middleManager.nodeAffinity }}
- nodeAffinity:
-{{ toYaml . | indent 10 }}
- {{- end }}
-{{- if .Values.middleManager.nodeSelector }}
- nodeSelector:
-{{ toYaml .Values.middleManager.nodeSelector | indent 8 }}
-{{- end }}
-{{- if .Values.middleManager.securityContext }}
- securityContext:
-{{ toYaml .Values.middleManager.securityContext | indent 8 }}
-{{- end }}
-{{- if .Values.middleManager.tolerations }}
- tolerations:
-{{ toYaml .Values.middleManager.tolerations | indent 8 }}
-{{- end }}
-{{- if .Values.image.pullSecrets }}
- imagePullSecrets:
-{{ toYaml .Values.image.pullSecrets | indent 8 }}
-{{- end }}
- {{- if .Values.middleManager.serviceAccount.create }}
- serviceAccountName: {{ include "druid.middleManager.serviceAccountName" . }}
- {{- end }}
- containers:
- - name: druid
- args: [ "middleManager" ]
- env:
- - name: POD_NAME
- valueFrom: {fieldRef: {fieldPath: metadata.name}}
- - name: POD_NAMESPACE
- valueFrom: {fieldRef: {fieldPath: metadata.namespace}}
- {{- range $key, $val := .Values.middleManager.config }}
- - name: {{ $key }}
- value: {{ $val | quote }}
- {{- end}}
- envFrom:
- - configMapRef:
- name: {{ template "druid.name" . }}
- resources:
-{{ toYaml .Values.middleManager.resources | indent 12 }}
- livenessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.middleManager.port }}
- readinessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.middleManager.port }}
- image: "{{ .Values.image.repository }}:{{ .Values.image.tag }}"
- imagePullPolicy: {{ .Values.image.pullPolicy | quote }}
- ports:
- - containerPort: {{ .Values.middleManager.port }}
- name: http
- {{- if .Values.prometheus.enabled}}
- - name: prometheus
- containerPort: {{ .Values.prometheus.port }}
- protocol: TCP
- {{- end }}
- volumeMounts:
- - mountPath: /opt/druid/var/druid/
- name: data
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- mountPath: /var/secrets/google
- {{- end }}
- volumes:
- {{- if not .Values.middleManager.persistence.enabled }}
- - name: data
- emptyDir: {}
- {{- end }}
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- secret:
- secretName: {{ .Values.gCloudStorage.secretName }}
- {{- end }}
- updateStrategy:
- type: {{ .Values.middleManager.updateStrategy.type }}
- {{- if .Values.middleManager.persistence.enabled }}
- volumeClaimTemplates:
- - metadata:
- name: data
- spec:
- accessModes:
- - {{ .Values.middleManager.persistence.accessMode | quote }}
- {{- if .Values.middleManager.persistence.storageClass }}
- {{- if (eq "-" .Values.middleManager.persistence.storageClass) }}
- storageClassName: ""
- {{- else }}
- storageClassName: "{{ .Values.middleManager.persistence.storageClass }}"
- {{- end }}
- {{- end }}
- resources:
- requests:
- storage: "{{ .Values.middleManager.persistence.size }}"
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/overlord/deployment.yaml b/helm/druid/templates/overlord/deployment.yaml
deleted file mode 100644
index f34722885940..000000000000
--- a/helm/druid/templates/overlord/deployment.yaml
+++ /dev/null
@@ -1,123 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.overlord.enabled -}}
-apiVersion: apps/v1
-kind: Deployment
-metadata:
- name: {{ include "druid.overlord.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.overlord.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- replicas: {{ .Values.overlord.replicaCount }}
- selector:
- matchLabels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.overlord.name }}
- template:
- metadata:
- labels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.overlord.name }}
- annotations:
- druid.k8s.enablePatching: "true"
- {{- with .Values.overlord.podAnnotations }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- if .Values.prometheus.enabled }}
- {{- with .Values.prometheus.annotation }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- end }}
- spec:
- {{- if .Values.overlord.serviceAccount.create }}
- serviceAccountName: {{ include "druid.overlord.serviceAccountName" . }}
- {{- end }}
- containers:
- - name: {{ .Chart.Name }}
- image: "{{ .Values.image.repository }}:{{ .Values.image.tag }}"
- imagePullPolicy: {{ .Values.image.pullPolicy }}
- args: [ "overlord" ]
- env:
- - name: POD_NAME
- valueFrom: {fieldRef: {fieldPath: metadata.name}}
- - name: POD_NAMESPACE
- valueFrom: {fieldRef: {fieldPath: metadata.namespace}}
- {{- range $key, $val := .Values.overlord.config }}
- - name: {{ $key }}
- value: {{ $val | quote }}
- {{- end}}
- envFrom:
- - configMapRef:
- name: {{ template "druid.name" . }}
- ports:
- - name: http
- containerPort: {{ .Values.overlord.port }}
- protocol: TCP
- {{- if .Values.prometheus.enabled }}
- - name: prometheus
- containerPort: {{ .Values.prometheus.port }}
- protocol: TCP
- {{- end }}
- livenessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.overlord.port }}
- readinessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.overlord.port }}
- resources:
-{{ toYaml .Values.overlord.resources | indent 12 }}
- volumeMounts:
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- mountPath: /var/secrets/google
- {{- end }}
- volumes:
- {{- if .Values.gCloudStorage.enabled }}
- - name: google-cloud-key
- secret:
- secretName: {{ .Values.gCloudStorage.secretName }}
- {{- end }}
- {{- with .Values.overlord.nodeSelector }}
- nodeSelector:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.overlord.affinity }}
- affinity:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.overlord.tolerations }}
- tolerations:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.image.pullSecrets }}
- imagePullSecrets:
-{{ toYaml . | indent 8 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/overlord/ingress.yaml b/helm/druid/templates/overlord/ingress.yaml
deleted file mode 100644
index f1f3051dbf42..000000000000
--- a/helm/druid/templates/overlord/ingress.yaml
+++ /dev/null
@@ -1,58 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.overlord.ingress.enabled -}}
-{{- $fullName := include "druid.overlord.fullname" . -}}
-{{- $ingressPath := .Values.overlord.ingress.path -}}
-apiVersion: extensions/v1beta1
-kind: Ingress
-metadata:
- name: {{ $fullName }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.overlord.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-{{- with .Values.overlord.ingress.annotations }}
- annotations:
-{{ toYaml . | indent 4 }}
-{{- end }}
-spec:
-{{- if .Values.overlord.ingress.tls }}
- tls:
- {{- range .Values.overlord.ingress.tls }}
- - hosts:
- {{- range .hosts }}
- - {{ . | quote }}
- {{- end }}
- secretName: {{ .secretName }}
- {{- end }}
-{{- end }}
- rules:
- {{- range .Values.overlord.ingress.hosts }}
- - host: {{ . | quote }}
- http:
- paths:
- - path: {{ $ingressPath }}
- backend:
- serviceName: {{ $fullName }}
- servicePort: http
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/overlord/role.yaml b/helm/druid/templates/overlord/role.yaml
deleted file mode 100644
index 37ffbd891816..000000000000
--- a/helm/druid/templates/overlord/role.yaml
+++ /dev/null
@@ -1,46 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.overlord.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: Role
-metadata:
- name: {{ template "druid.overlord.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.overlord.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-rules:
- - apiGroups:
- - ""
- resources:
- - pods
- - configmaps
- verbs:
- - '*'
- - apiGroups:
- - batch
- resources:
- - jobs
- verbs:
- - '*'
-{{- end }}
-
diff --git a/helm/druid/templates/overlord/roleBinding.yaml b/helm/druid/templates/overlord/roleBinding.yaml
deleted file mode 100644
index 2a5284e606a4..000000000000
--- a/helm/druid/templates/overlord/roleBinding.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.overlord.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: RoleBinding
-metadata:
- name: {{ template "druid.overlord.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.overlord.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-roleRef:
- apiGroup: rbac.authorization.k8s.io
- kind: Role
- name: {{ template "druid.overlord.fullname" . }}
-subjects:
- - kind: ServiceAccount
- name: {{ include "druid.overlord.serviceAccountName" . }}
- namespace: {{ .Release.Namespace }}
-{{- end }}
diff --git a/helm/druid/templates/overlord/service.yaml b/helm/druid/templates/overlord/service.yaml
deleted file mode 100644
index 6a9b856d875c..000000000000
--- a/helm/druid/templates/overlord/service.yaml
+++ /dev/null
@@ -1,42 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.overlord.enabled -}}
-apiVersion: v1
-kind: Service
-metadata:
- name: {{ include "druid.overlord.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.overlord.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- type: {{ .Values.overlord.serviceType }}
- ports:
- - port: {{ .Values.overlord.port }}
- targetPort: http
- protocol: TCP
- name: http
- selector:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.overlord.name }}
-{{- end }}
diff --git a/helm/druid/templates/overlord/serviceAccount.yaml b/helm/druid/templates/overlord/serviceAccount.yaml
deleted file mode 100644
index b27ea8b4b31d..000000000000
--- a/helm/druid/templates/overlord/serviceAccount.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.overlord.serviceAccount.create) (.Values.overlord.enabled) }}
-apiVersion: v1
-kind: ServiceAccount
-automountServiceAccountToken: {{ .Values.overlord.serviceAccount.automountServiceAccountToken }}
-metadata:
- name: {{ include "druid.overlord.serviceAccountName" . }}
- {{- with .Values.overlord.serviceAccount.annotations }}
- annotations:
- {{- toYaml . | nindent 4 }}
- {{- end }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.overlord.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
- {{- with .Values.overlord.serviceAccount.labels }}
- {{- toYaml . | nindent 4 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/router/deployment.yaml b/helm/druid/templates/router/deployment.yaml
deleted file mode 100644
index 79960fe13b99..000000000000
--- a/helm/druid/templates/router/deployment.yaml
+++ /dev/null
@@ -1,112 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.router.enabled -}}
-apiVersion: apps/v1
-kind: Deployment
-metadata:
- name: {{ include "druid.router.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.router.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- replicas: {{ .Values.router.replicaCount }}
- selector:
- matchLabels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.router.name }}
- template:
- metadata:
- labels:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.router.name }}
- annotations:
- druid.k8s.enablePatching: "true"
- {{- with .Values.router.podAnnotations }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- if .Values.prometheus.enabled }}
- {{- with .Values.prometheus.annotation }}
- {{- toYaml . | nindent 8 }}
- {{- end }}
- {{- end }}
- spec:
- {{- if .Values.router.serviceAccount.create }}
- serviceAccountName: {{ include "druid.router.serviceAccountName" . }}
- {{- end }}
- containers:
- - name: {{ .Chart.Name }}
- image: "{{ .Values.image.repository }}:{{ .Values.image.tag }}"
- imagePullPolicy: {{ .Values.image.pullPolicy }}
- args: [ "router" ]
- env:
- - name: POD_NAME
- valueFrom: {fieldRef: {fieldPath: metadata.name}}
- - name: POD_NAMESPACE
- valueFrom: {fieldRef: {fieldPath: metadata.namespace}}
- {{- range $key, $val := .Values.router.config }}
- - name: {{ $key }}
- value: {{ $val | quote }}
- {{- end}}
- envFrom:
- - configMapRef:
- name: {{ template "druid.name" . }}
- ports:
- - name: http
- containerPort: {{ .Values.router.port }}
- protocol: TCP
- {{- if .Values.prometheus.enabled }}
- - name: prometheus
- containerPort: {{ .Values.prometheus.port }}
- protocol: TCP
- {{- end }}
- livenessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.router.port }}
- readinessProbe:
- initialDelaySeconds: 60
- httpGet:
- path: /status/health
- port: {{ .Values.router.port }}
- resources:
-{{ toYaml .Values.router.resources | indent 12 }}
- {{- with .Values.router.nodeSelector }}
- nodeSelector:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.router.affinity }}
- affinity:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.router.tolerations }}
- tolerations:
-{{ toYaml . | indent 8 }}
- {{- end }}
- {{- with .Values.image.pullSecrets }}
- imagePullSecrets:
-{{ toYaml . | indent 8 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/router/ingress.yaml b/helm/druid/templates/router/ingress.yaml
deleted file mode 100644
index aab27fc79b68..000000000000
--- a/helm/druid/templates/router/ingress.yaml
+++ /dev/null
@@ -1,58 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.router.ingress.enabled -}}
-{{- $fullName := include "druid.router.fullname" . -}}
-{{- $ingressPath := .Values.router.ingress.path -}}
-apiVersion: extensions/v1beta1
-kind: Ingress
-metadata:
- name: {{ $fullName }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.router.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-{{- with .Values.router.ingress.annotations }}
- annotations:
-{{ toYaml . | indent 4 }}
-{{- end }}
-spec:
-{{- if .Values.router.ingress.tls }}
- tls:
- {{- range .Values.router.ingress.tls }}
- - hosts:
- {{- range .hosts }}
- - {{ . | quote }}
- {{- end }}
- secretName: {{ .secretName }}
- {{- end }}
-{{- end }}
- rules:
- {{- range .Values.router.ingress.hosts }}
- - host: {{ . | quote }}
- http:
- paths:
- - path: {{ $ingressPath }}
- backend:
- serviceName: {{ $fullName }}
- servicePort: http
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/router/role.yaml b/helm/druid/templates/router/role.yaml
deleted file mode 100644
index 52afc0a685a7..000000000000
--- a/helm/druid/templates/router/role.yaml
+++ /dev/null
@@ -1,40 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.router.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: Role
-metadata:
- name: {{ template "druid.router.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.router.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-rules:
- - apiGroups:
- - ""
- resources:
- - pods
- - configmaps
- verbs:
- - '*'
-{{- end }}
-
diff --git a/helm/druid/templates/router/roleBinding.yaml b/helm/druid/templates/router/roleBinding.yaml
deleted file mode 100644
index 1b9e129305d0..000000000000
--- a/helm/druid/templates/router/roleBinding.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.rbac.create) (.Values.router.enabled) }}
-apiVersion: rbac.authorization.k8s.io/v1
-kind: RoleBinding
-metadata:
- name: {{ template "druid.router.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.router.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-roleRef:
- apiGroup: rbac.authorization.k8s.io
- kind: Role
- name: {{ template "druid.router.fullname" . }}
-subjects:
- - kind: ServiceAccount
- name: {{ include "druid.router.serviceAccountName" . }}
- namespace: {{ .Release.Namespace }}
-{{- end }}
diff --git a/helm/druid/templates/router/service.yaml b/helm/druid/templates/router/service.yaml
deleted file mode 100644
index 07d3659c5a94..000000000000
--- a/helm/druid/templates/router/service.yaml
+++ /dev/null
@@ -1,42 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.router.enabled -}}
-apiVersion: v1
-kind: Service
-metadata:
- name: {{ include "druid.router.fullname" . }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.router.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
-spec:
- type: {{ .Values.router.serviceType }}
- ports:
- - port: {{ .Values.router.port }}
- targetPort: http
- protocol: TCP
- name: http
- selector:
- app: {{ include "druid.name" . }}
- release: {{ .Release.Name }}
- component: {{ .Values.router.name }}
-{{- end }}
diff --git a/helm/druid/templates/router/serviceAccount.yaml b/helm/druid/templates/router/serviceAccount.yaml
deleted file mode 100644
index d357b2bd0c40..000000000000
--- a/helm/druid/templates/router/serviceAccount.yaml
+++ /dev/null
@@ -1,39 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if and (.Values.router.serviceAccount.create) (.Values.router.enabled) }}
-apiVersion: v1
-kind: ServiceAccount
-automountServiceAccountToken: {{ .Values.router.serviceAccount.automountServiceAccountToken }}
-metadata:
- name: {{ include "druid.router.serviceAccountName" . }}
- {{- with .Values.router.serviceAccount.annotations }}
- annotations:
- {{- toYaml . | nindent 4 }}
- {{- end }}
- labels:
- app: {{ include "druid.name" . }}
- chart: {{ include "druid.chart" . }}
- component: {{ .Values.router.name }}
- release: {{ .Release.Name }}
- heritage: {{ .Release.Service }}
- {{- with .Values.router.serviceAccount.labels }}
- {{- toYaml . | nindent 4 }}
- {{- end }}
-{{- end }}
diff --git a/helm/druid/templates/secrets.yaml b/helm/druid/templates/secrets.yaml
deleted file mode 100644
index 52451802f57a..000000000000
--- a/helm/druid/templates/secrets.yaml
+++ /dev/null
@@ -1,28 +0,0 @@
-{{/*
-
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
-*/}}
-
-{{- if .Values.gCloudStorage.enabled -}}
-apiVersion: v1
-kind: Secret
-metadata:
- name: google-cloud-key
-type: Opaque
-data:
- key.json: {{ .Values.google.gcsAPIKey }}
-{{- end }}
diff --git a/helm/druid/values.yaml b/helm/druid/values.yaml
deleted file mode 100644
index dab6a49b807b..000000000000
--- a/helm/druid/values.yaml
+++ /dev/null
@@ -1,503 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Default values for druid.
-
-image:
- repository: apache/druid
- tag: 24.0.0
- pullPolicy: IfNotPresent
- pullSecrets: []
-
-configMap:
- ## If false, configMap will not be applied
- ##
- enabled: true
-
-# Required if using kubernetes extensions which modify resources like 'druid-kubernetes-extensions' or 'druid-kubernetes-overlord-extensions'
-rbac:
- create: true
-
-## Define the key value pairs in the configmap
-configVars:
- ## DRUID env vars. ref: https://github.com/apache/druid/blob/master/distribution/docker/druid.sh#L29
- # DRUID_LOG_LEVEL: "warn"
- # DRUID_LOG4J:
- DRUID_USE_CONTAINER_IP: "true"
-
- ## Druid Common Configurations. ref: https://druid.apache.org/docs/latest/configuration/index.html#common-configurations
- druid_extensions_loadList: '["druid-histogram", "druid-datasketches", "druid-lookups-cached-global", "postgresql-metadata-storage"]'
- druid_metadata_storage_type: postgresql
- druid_metadata_storage_connector_connectURI: jdbc:postgresql://postgres:5432/druid
- druid_metadata_storage_connector_user: druid
- druid_metadata_storage_connector_password: druid
- druid_storage_type: local
- druid_indexer_logs_type: file
- druid_indexer_logs_directory: /opt/data/indexing-logs
-
- ## Druid Emitting Metrics. ref: https://druid.apache.org/docs/latest/configuration/index.html#emitting-metrics
- druid_emitter: noop
- druid_emitter_logging_logLevel: debug
- druid_emitter_http_recipientBaseUrl: http://druid_exporter_url:druid_exporter_port/druid
-
-gCloudStorage:
- enabled: false
- secretName: google-cloud-key
-
-broker:
- ## If false, broker will not be installed
- ##
- enabled: true
- name: broker
- replicaCount: 1
- port: 8082
- serviceType: ClusterIP
-
- config:
- DRUID_XMX: 512m
- DRUID_XMS: 512m
- DRUID_MAXDIRECTMEMORYSIZE: 400m
- druid_processing_buffer_sizeBytes: '50000000'
- druid_processing_numMergeBuffers: 2
- druid_processing_numThreads: 1
- # druid_monitoring_monitors: '["org.apache.druid.client.cache.CacheMonitor", "org.apache.druid.server.metrics.QueryCountStatsMonitor"]'
-
- ingress:
- enabled: false
- annotations: {}
- # kubernetes.io/ingress.class: nginx
- # kubernetes.io/tls-acme: "true"
- path: /
- hosts:
- - chart-example.local
- tls: []
- # - secretName: chart-example-tls
- # hosts:
- # - chart-example.local
-
- resources: {}
- # limits:
- # cpu: 1
- # memory: 1Gi
- # requests:
- # cpu: 250m
- # memory: 512Mi
-
- serviceAccount:
- # -- Create a service account for the broker
- create: true
- # -- Service Account name
- name:
- # -- Annotations applied to created service account
- annotations: {}
- # -- Labels applied to created service account
- labels: {}
- # -- Automount API credentials for the service account
- automountServiceAccountToken: true
-
- nodeSelector: {}
-
- tolerations: []
-
- affinity: {}
-
- podAnnotations: {}
-
-coordinator:
- ## If false, coordinator will not be installed
- ##
- enabled: true
- name: coordinator
- replicaCount: 1
- port: 8081
- serviceType: ClusterIP
-
- config:
- DRUID_XMX: 256m
- DRUID_XMS: 256m
- # druid_monitoring_monitors: '["org.apache.druid.server.metrics.TaskCountStatsMonitor"]'
-
- ingress:
- enabled: false
- annotations: {}
- # kubernetes.io/ingress.class: nginx
- # kubernetes.io/tls-acme: "true"
- path: /
- hosts:
- - chart-example.local
- tls: []
- # - secretName: chart-example-tls
- # hosts:
- # - chart-example.local
-
- resources: {}
- # limits:
- # cpu: 500m
- # memory: 1Gi
- # requests:
- # cpu: 250m
- # memory: 512Mi
-
- serviceAccount:
- # -- Create a service account for the coordinator
- create: true
- # -- Service Account name
- name:
- # -- Annotations applied to created service account
- annotations: {}
- # -- Labels applied to created service account
- labels: {}
- # -- Automount API credentials for the service account
- automountServiceAccountToken: true
-
- nodeSelector: {}
-
- tolerations: []
-
- affinity: {}
-
- podAnnotations: {}
-
-overlord:
- ## If true, the separate overlord will be installed
- ##
- enabled: false
- name: overlord
- replicaCount: 1
- port: 8081
- serviceType: ClusterIP
-
- javaOpts: "-Xms1G -Xmx1G"
-
- ingress:
- enabled: false
- annotations: {}
- # kubernetes.io/ingress.class: nginx
- # kubernetes.io/tls-acme: "true"
- path: /
- hosts:
- - chart-example.local
- tls: []
- # - secretName: chart-example-tls
- # hosts:
- # - chart-example.local
-
- resources: {}
-
- serviceAccount:
- # -- Create a service account for the overlord
- create: true
- # -- Service Account name
- name:
- # -- Annotations applied to created service account
- annotations: {}
- # -- Labels applied to created service account
- labels: {}
- # -- Automount API credentials for the service account
- automountServiceAccountToken: true
-
- nodeSelector: {}
-
- tolerations: []
-
- affinity: {}
-
- podAnnotations: {}
-
-historical:
- ## If false, historical will not be installed
- ##
- enabled: true
- name: historical
- replicaCount: 1
- port: 8083
- serviceType: ClusterIP
-
- config:
- DRUID_XMX: 512m
- DRUID_XMS: 512m
- DRUID_MAXDIRECTMEMORYSIZE: 400m
- druid_processing_buffer_sizeBytes: '50000000'
- druid_processing_numMergeBuffers: 2
- druid_processing_numThreads: 1
- # druid_monitoring_monitors: '["org.apache.druid.client.cache.CacheMonitor", "org.apache.druid.server.metrics.HistoricalMetricsMonitor", "org.apache.druid.server.metrics.QueryCountStatsMonitor"]'
- # druid_segmentCache_locations: '[{"path":"/opt/druid/var/druid/segment-cache","maxSize":300000000000}]'
-
- ingress:
- enabled: false
- annotations: {}
- # kubernetes.io/ingress.class: nginx
- # kubernetes.io/tls-acme: "true"
- path: /
- hosts:
- - chart-example.local
- tls: []
- # - secretName: chart-example-tls
- # hosts:
- # - chart-example.local
-
- persistence:
- enabled: true
- accessMode: ReadWriteOnce
- size: "4Gi"
- # storageClass: "ssd"
-
- antiAffinity: "soft"
-
- nodeAffinity: {}
-
- nodeSelector: {}
-
- securityContext:
- fsGroup: 1000
-
- tolerations: []
-
- resources: {}
- # limits:
- # cpu: 2
- # memory: 2Gi
- # requests:
- # cpu: 500m
- # memory: 512Mi
-
- serviceAccount:
- # -- Create a service account for the overlord
- create: true
- # -- Service Account name
- name:
- # -- Annotations applied to created service account
- annotations: {}
- # -- Labels applied to created service account
- labels: {}
- # -- Automount API credentials for the service account
- automountServiceAccountToken: true
-
- livenessProbeInitialDelaySeconds: 60
- readinessProbeInitialDelaySeconds: 60
-
- ## (dict) If specified, apply these annotations to each master Pod
- podAnnotations: {}
-
- podDisruptionBudget:
- enabled: false
- # minAvailable: 2
- maxUnavailable: 1
-
- updateStrategy:
- type: RollingUpdate
-
-middleManager:
- ## If false, middleManager will not be installed
- ##
- enabled: true
- name: middle-manager
- replicaCount: 1
- port: 8091
- serviceType: ClusterIP
-
- config:
- DRUID_XMX: 64m
- DRUID_XMS: 64m
- druid_indexer_runner_javaOptsArray: '["-server", "-Xms256m", "-Xmx256m", "-XX:MaxDirectMemorySize=300m", "-Duser.timezone=UTC", "-Dfile.encoding=UTF-8", "-XX:+ExitOnOutOfMemoryError", "-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager"]'
- druid_indexer_fork_property_druid_processing_buffer_sizeBytes: '25000000'
-
- autoscaling:
- enabled: false
- minReplicas: 2
- maxReplicas: 5
- metrics:
- - type: Resource
- resource:
- name: cpu
- target:
- type: Utilization
- averageUtilization: 60
- - type: Resource
- resource:
- name: memory
- target:
- type: Utilization
- averageUtilization: 60
-
- ingress:
- enabled: false
- annotations: {}
- # kubernetes.io/ingress.class: nginx
- # kubernetes.io/tls-acme: "true"
- path: /
- hosts:
- - chart-example.local
- tls: []
- # - secretName: chart-example-tls
- # hosts:
- # - chart-example.local
-
- persistence:
- enabled: true
- accessMode: ReadWriteOnce
- size: "4Gi"
- # storageClass: "ssd"
-
- antiAffinity: "soft"
-
- nodeAffinity: {}
-
- nodeSelector: {}
-
- securityContext:
- fsGroup: 1000
-
- tolerations: []
-
- resources: {}
- # limits:
- # cpu: 500m
- # memory: 1Gi
- # requests:
- # cpu: 250m
- # memory: 256Mi
-
- serviceAccount:
- # -- Create a service account for the overlord
- create: true
- # -- Service Account name
- name:
- # -- Annotations applied to created service account
- annotations: {}
- # -- Labels applied to created service account
- labels: {}
- # -- Automount API credentials for the service account
- automountServiceAccountToken: true
-
- ## (dict) If specified, apply these annotations to each master Pod
- podAnnotations: {}
-
- podDisruptionBudget:
- enabled: false
- # minAvailable: 2
- maxUnavailable: 1
-
- updateStrategy:
- type: RollingUpdate
-
-router:
- ## If false, router will not be installed
- ##
- enabled: true
- name: router
- replicaCount: 1
- port: 8888
- serviceType: ClusterIP
-
- config:
- DRUID_XMX: 128m
- DRUID_XMS: 128m
- DRUID_MAXDIRECTMEMORYSIZE: 128m
-
- ingress:
- enabled: false
- annotations: {}
- # kubernetes.io/ingress.class: nginx
- # kubernetes.io/tls-acme: "true"
- path: /
- hosts:
- - chart-example.local
- tls: []
- # - secretName: chart-example-tls
- # hosts:
- # - chart-example.local
-
- resources: {}
- # limits:
- # cpu: 250m
- # memory: 256Mi
- # requests:
- # cpu: 100m
- # memory: 128Mi
-
- serviceAccount:
- # -- Create a service account for the overlord
- create: true
- # -- Service Account name
- name:
- # -- Annotations applied to created service account
- annotations: {}
- # -- Labels applied to created service account
- labels: {}
- # -- Automount API credentials for the service account
- automountServiceAccountToken: true
-
- nodeSelector: {}
-
- tolerations: []
-
- affinity: {}
-
- podAnnotations: {}
-
-# ------------------------------------------------------------------------------
-# Zookeeper:
-# ------------------------------------------------------------------------------
-
-# If using a zookeeper installed outside of this chart you must uncomment and set this line
-# zkHosts: druid-zookeeper-headless:2181
-
-zookeeper:
- enabled: true
- ## Environmental variables to set in Zookeeper
- ##
- env:
- ## The JVM heap size to allocate to Zookeeper
- ZK_HEAP_SIZE: "512M"
- ## Configure Zookeeper headless
- headless:
- publishNotReadyAddresses: true
-
-
-# ------------------------------------------------------------------------------
-# MySQL:
-# ------------------------------------------------------------------------------
-mysql:
- enabled: false
- mysqlRootPassword: druidroot
- mysqlUser: druid
- mysqlPassword: druid
- mysqlDatabase: druid
- configurationFiles:
- mysql_collate.cnf: |-
- [mysqld]
- character-set-server=utf8
- collation-server=utf8_unicode_ci
-
-# ------------------------------------------------------------------------------
-# postgres:
-# ------------------------------------------------------------------------------
-postgresql:
- enabled: true
- postgresqlUsername: druid
- postgresqlPassword: druid
- postgresqlDatabase: druid
- service:
- port: 5432
-
-# Secrets
-prometheus:
- enabled: false
- #pick the any port what you want
- port: 9090
- annotation:
- prometheus.io/scrape: "true"
- prometheus.io/port: "9090"
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
index 37596bffde9c..5e937fe69eb0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
@@ -3105,45 +3105,59 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException
final List>> futures = new ArrayList<>();
final List futureGroupIds = new ArrayList<>();
- boolean stopTasksEarly = false;
+ boolean stopTasksEarly;
if (earlyStopTime != null && (earlyStopTime.isBeforeNow() || earlyStopTime.isEqualNow())) {
log.info("Early stop requested - signalling tasks to complete");
earlyStopTime = null;
stopTasksEarly = true;
- }
-
- int stoppedTasks = 0;
- for (Entry entry : activelyReadingTaskGroups.entrySet()) {
- Integer groupId = entry.getKey();
- TaskGroup group = entry.getValue();
-
- if (stopTasksEarly) {
- log.info("Stopping task group [%d] early. It has run for [%s]", groupId, ioConfig.getTaskDuration());
- futureGroupIds.add(groupId);
- futures.add(checkpointTaskGroup(group, true));
- } else {
- // find the longest running task from this group
- DateTime earliestTaskStart = DateTimes.nowUtc();
- for (TaskData taskData : group.tasks.values()) {
- if (taskData.startTime != null && earliestTaskStart.isAfter(taskData.startTime)) {
- earliestTaskStart = taskData.startTime;
- }
- }
-
- if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) {
- // if this task has run longer than the configured duration
- // as long as the pending task groups are less than the configured stop task count.
- if (pendingCompletionTaskGroups.values().stream().mapToInt(CopyOnWriteArrayList::size).sum() + stoppedTasks
- < ioConfig.getStopTaskCount()) {
- log.info("Task group [%d] has run for [%s]. Stopping.", groupId, ioConfig.getTaskDuration());
+ } else {
+ stopTasksEarly = false;
+ }
+
+ AtomicInteger stoppedTasks = new AtomicInteger();
+ // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing
+ activelyReadingTaskGroups
+ .entrySet().stream().sorted(
+ Comparator.comparingLong(
+ (Entry entry) ->
+ computeEarliestTaskStartTime(entry.getValue())
+ .getMillis()))
+ .forEach(entry -> {
+ Integer groupId = entry.getKey();
+ TaskGroup group = entry.getValue();
+
+ if (stopTasksEarly) {
+ log.info(
+ "Stopping task group [%d] early. It has run for [%s]",
+ groupId,
+ ioConfig.getTaskDuration()
+ );
futureGroupIds.add(groupId);
futures.add(checkpointTaskGroup(group, true));
- stoppedTasks++;
+ } else {
+ DateTime earliestTaskStart = computeEarliestTaskStartTime(group);
+
+ if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) {
+ // if this task has run longer than the configured duration
+ // as long as the pending task groups are less than the configured stop task count.
+ if (pendingCompletionTaskGroups.values()
+ .stream()
+ .mapToInt(CopyOnWriteArrayList::size)
+ .sum() + stoppedTasks.get()
+ < ioConfig.getMaxAllowedStops()) {
+ log.info(
+ "Task group [%d] has run for [%s]. Stopping.",
+ groupId,
+ ioConfig.getTaskDuration()
+ );
+ futureGroupIds.add(groupId);
+ futures.add(checkpointTaskGroup(group, true));
+ stoppedTasks.getAndIncrement();
+ }
+ }
}
- }
- }
- }
+ });
List>> results = coalesceAndAwait(futures);
for (int j = 0; j < results.size(); j++) {
@@ -3200,6 +3214,15 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException
}
}
+ private DateTime computeEarliestTaskStartTime(TaskGroup group)
+ {
+ return group.tasks.values().stream()
+ .filter(taskData -> taskData.startTime != null)
+ .map(taskData -> taskData.startTime)
+ .min(DateTime::compareTo)
+ .orElse(DateTimes.nowUtc());
+ }
+
private ListenableFuture