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> checkpointTaskGroup( final TaskGroup taskGroup, final boolean finalize diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index d49ceaa260cf..90ba05e56ccb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -49,8 +49,7 @@ public abstract class SeekableStreamSupervisorIOConfig private final Optional lateMessageRejectionStartDateTime; @Nullable private final AutoScalerConfig autoScalerConfig; @Nullable private final IdleConfig idleConfig; - - private final int stopTaskCount; + @Nullable private final Integer stopTaskCount; public SeekableStreamSupervisorIOConfig( String stream, @@ -81,8 +80,9 @@ public SeekableStreamSupervisorIOConfig( } else { this.taskCount = taskCount != null ? taskCount : 1; } - this.stopTaskCount = stopTaskCount == null ? this.taskCount : stopTaskCount; - Preconditions.checkArgument(this.stopTaskCount > 0, "stopTaskCount must be greater than 0"); + Preconditions.checkArgument(stopTaskCount == null || stopTaskCount > 0, + "stopTaskCount must be greater than 0"); + this.stopTaskCount = stopTaskCount; this.taskDuration = defaultDuration(taskDuration, "PT1H"); this.startDelay = defaultDuration(startDelay, "PT5S"); this.period = defaultDuration(period, "PT30S"); @@ -205,9 +205,15 @@ public IdleConfig getIdleConfig() return idleConfig; } + @Nullable @JsonProperty - public int getStopTaskCount() + public Integer getStopTaskCount() { return stopTaskCount; } + + public int getMaxAllowedStops() + { + return stopTaskCount == null ? taskCount : stopTaskCount; + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java index 100c06a016c3..4231f318efd8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskReportSerdeTest.java @@ -19,9 +19,12 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import com.google.common.io.Files; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; @@ -34,6 +37,7 @@ import org.junit.rules.TemporaryFolder; import java.io.File; +import java.nio.charset.StandardCharsets; import java.util.Map; public class TaskReportSerdeTest @@ -47,6 +51,7 @@ public TaskReportSerdeTest() { TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); + jsonMapper.registerSubtypes(ExceptionalTaskReport.class); } @Test @@ -87,4 +92,47 @@ public void testSerde() throws Exception ); Assert.assertEquals(reportMap1, reportMap2); } + + @Test + public void testExceptionWhileWritingReport() throws Exception + { + final File reportFile = temporaryFolder.newFile(); + final SingleFileTaskReportFileWriter writer = new SingleFileTaskReportFileWriter(reportFile); + writer.setObjectMapper(jsonMapper); + writer.write("theTask", ImmutableMap.of("report", new ExceptionalTaskReport())); + + // Read the file, ensure it's incomplete and not valid JSON. This allows callers to determine the report was + // not complete when written. + Assert.assertEquals( + "{\"report\":{\"type\":\"exceptional\"", + Files.asCharSource(reportFile, StandardCharsets.UTF_8).read() + ); + } + + /** + * Task report that throws an exception while being serialized. + */ + @JsonTypeName("exceptional") + private static class ExceptionalTaskReport implements TaskReport + { + @Override + @JsonProperty + public String getTaskId() + { + throw new UnsupportedOperationException("cannot serialize task ID"); + } + + @Override + public String getReportKey() + { + return "report"; + } + + @Override + @JsonProperty + public Object getPayload() + { + throw new UnsupportedOperationException("cannot serialize payload"); + } + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 2a69ff064cb4..840b4e9f69a0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -979,6 +979,228 @@ public Duration getEmissionDuration() Assert.assertTrue(supervisor.getNoticesQueueSize() == 0); } + @Test(timeout = 60_000L) + public void testEarlyStoppingOfTaskGroupBasedOnStopTaskCount() throws InterruptedException, JsonProcessingException + { + // Assuming tasks have surpassed their duration limit at test execution + DateTime startTime = DateTimes.nowUtc().minusHours(2); + // Configure supervisor to stop only one task at a time + int stopTaskCount = 1; + SeekableStreamSupervisorIOConfig ioConfig = new SeekableStreamSupervisorIOConfig( + STREAM, + new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), false, false, false), + 1, + 3, + new Period("PT1H"), + new Period("PT1S"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + null, + null, + new IdleConfig(true, 200L), + stopTaskCount + ) + { + }; + + EasyMock.reset(spec); + EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); + EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() + { + @Override + public Duration getEmissionDuration() + { + return new Period("PT2S").toStandardDuration(); + } + }).anyTimes(); + EasyMock.expect(spec.getType()).andReturn("stream").anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getContextValue("tags")).andReturn("").anyTimes(); + EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); + + SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(); + + TreeMap> sequenceOffsets = new TreeMap<>(); + sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L)); + + Map context = new HashMap<>(); + context.put("checkpoints", new ObjectMapper().writeValueAsString(sequenceOffsets)); + + TestSeekableStreamIndexTask id1 = new TestSeekableStreamIndexTask( + "id1", + null, + getDataSchema(), + taskTuningConfig, + createTaskIoConfigExt( + 0, + Collections.singletonMap("0", "10"), + Collections.singletonMap("0", "20"), + "test", + startTime, + null, + Collections.emptySet(), + ioConfig + ), + context, + "0" + ); + + TestSeekableStreamIndexTask id2 = new TestSeekableStreamIndexTask( + "id2", + null, + getDataSchema(), + taskTuningConfig, + createTaskIoConfigExt( + 1, + Collections.singletonMap("1", "10"), + Collections.singletonMap("1", "20"), + "test", + startTime, + null, + Collections.emptySet(), + ioConfig + ), + context, + "1" + ); + + TestSeekableStreamIndexTask id3 = new TestSeekableStreamIndexTask( + "id3", + null, + getDataSchema(), + taskTuningConfig, + createTaskIoConfigExt( + 2, + Collections.singletonMap("2", "10"), + Collections.singletonMap("2", "20"), + "test", + startTime, + null, + Collections.emptySet(), + ioConfig + ), + context, + "2" + ); + + final TaskLocation location1 = TaskLocation.create("testHost", 1234, -1); + final TaskLocation location2 = TaskLocation.create("testHost2", 145, -1); + final TaskLocation location3 = TaskLocation.create("testHost3", 145, -1); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + workItems.add(new TestTaskRunnerWorkItem(id3, null, location3)); + + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)) + .andReturn(ImmutableList.of(id1, id2, id3)) + .anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id2)).anyTimes(); + + EasyMock.reset(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)) + .andReturn(new TestSeekableStreamDataSourceMetadata(null)).anyTimes(); + EasyMock.expect(indexTaskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING)) + .anyTimes(); + EasyMock.expect(indexTaskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING)) + .anyTimes(); + EasyMock.expect(indexTaskClient.getStatusAsync("id3")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING)) + .anyTimes(); + + EasyMock.expect(indexTaskClient.getStartTimeAsync("id1")) + .andReturn(Futures.immediateFuture(startTime.plusSeconds(1))) + .anyTimes(); + // Mocking to return the earliest start time for task id2, indicating it's the first group to start + EasyMock.expect(indexTaskClient.getStartTimeAsync("id2")) + .andReturn(Futures.immediateFuture(startTime)).anyTimes(); + EasyMock.expect(indexTaskClient.getStartTimeAsync("id3")) + .andReturn(Futures.immediateFuture(startTime.plusSeconds(2))) + .anyTimes(); + + ImmutableMap partitionOffset = ImmutableMap.of("0", "10"); + final TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, partitionOffset); + + EasyMock.expect(indexTaskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .anyTimes(); + EasyMock.expect(indexTaskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .anyTimes(); + EasyMock.expect(indexTaskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .anyTimes(); + EasyMock.expect(indexTaskClient.setEndOffsetsAsync("id1", partitionOffset, false)) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.setEndOffsetsAsync("id2", partitionOffset, false)) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.setEndOffsetsAsync("id3", partitionOffset, false)) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.resumeAsync("id1")) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.resumeAsync("id2")) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.resumeAsync("id3")) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.pauseAsync("id1")) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + EasyMock.expect(indexTaskClient.pauseAsync("id3")) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + + // Expect the earliest-started task (id2) to transition to publishing first + taskQueue.shutdown("id2", "All tasks in group[%s] failed to transition to publishing state", 1); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + supervisor.start(); + supervisor.runInternal(); + + supervisor.checkpoint( + 0, + new TestSeekableStreamDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>(STREAM, checkpoints.get(0), ImmutableSet.of()) + ) + ); + + while (supervisor.getNoticesQueueSize() > 0) { + Thread.sleep(100); + } + + verifyAll(); + + Assert.assertTrue(supervisor.getNoticesQueueSize() == 0); + } + @Test public void testEmitBothLag() throws Exception { diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env b/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env index fec58cc0a000..5a2dbeadac80 100644 --- a/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env @@ -26,28 +26,32 @@ druid_auth_authenticator_basic_type=basic druid_auth_authenticatorChain=["basic"] druid_auth_authorizer_basic_type=basic druid_auth_authorizers=["basic"] -druid_client_https_certAlias=druid -druid_client_https_keyManagerPassword=druid123 -druid_client_https_keyStorePassword=druid123 -druid_client_https_keyStorePath=/tls/server.p12 -druid_client_https_protocol=TLSv1.2 -druid_client_https_trustStoreAlgorithm=PKIX -druid_client_https_trustStorePassword=druid123 -druid_client_https_trustStorePath=/tls/truststore.jks -druid_enableTlsPort=true -druid_server_http_allowedHttpMethods=["OPTIONS"] -druid_server_https_certAlias=druid -druid_server_https_keyManagerPassword=druid123 -druid_server_https_keyStorePassword=druid123 -druid_server_https_keyStorePath=/tls/server.p12 -druid_server_https_keyStoreType=PKCS12 -druid_server_https_requireClientCertificate=true -druid_server_https_trustStoreAlgorithm=PKIX -druid_server_https_trustStorePassword=druid123 -druid_server_https_trustStorePath=/tls/truststore.jks -druid_server_https_validateHostnames=true + druid_escalator_authorizerName=basic druid_escalator_internalClientPassword=warlock druid_escalator_internalClientUsername=druid_system druid_escalator_type=basic druid_server_https_crlPath=/tls/revocations.crl + +# /tls is not currently present in integration-tests-ex containers, this will need to be generated using the scripts in +# integration-tests/docker/tls in the future. +# druid_client_https_certAlias=druid +# druid_client_https_keyManagerPassword=druid123 +# druid_client_https_keyStorePassword=druid123 +# druid_client_https_keyStorePath=/tls/server.p12 +# druid_client_https_protocol=TLSv1.2 +# druid_client_https_trustStoreAlgorithm=PKIX +# druid_client_https_trustStorePassword=druid123 +# druid_client_https_trustStorePath=/tls/truststore.jks +# druid_enableTlsPort=true +# druid_server_http_allowedHttpMethods=["OPTIONS"] +# druid_server_https_certAlias=druid +# druid_server_https_keyManagerPassword=druid123 +# druid_server_https_keyStorePassword=druid123 +# druid_server_https_keyStorePath=/tls/server.p12 +# druid_server_https_keyStoreType=PKCS12 +# druid_server_https_requireClientCertificate=true +# druid_server_https_trustStoreAlgorithm=PKIX +# druid_server_https_trustStorePassword=druid123 +# druid_server_https_trustStorePath=/tls/truststore.jks +# druid_server_https_validateHostnames=true diff --git a/integration-tests-ex/cases/cluster/Security/docker-compose.py b/integration-tests-ex/cases/cluster/Security/docker-compose.py new file mode 100644 index 000000000000..452026750bc5 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Security/docker-compose.py @@ -0,0 +1,32 @@ +# 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. + +from template import BaseTemplate, generate + +class Template(BaseTemplate): + + def define_indexer(self): + service = super().define_indexer() + self.add_property(service, 'druid.msq.intermediate.storage.enable', 'true') + self.add_property(service, 'druid.msq.intermediate.storage.type', 'local') + self.add_property(service, 'druid.msq.intermediate.storage.basePath', '/shared/durablestorage/') + self.add_property(service, 'druid.export.storage.baseDir', '/') + + def extend_druid_service(self, service): + self.add_env_file(service, '../Common/environment-configs/auth.env') + self.add_env(service, 'druid_test_loadList', 'druid-basic-security') + + +generate(__file__, Template()) diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/auth/ITSecurityBasicQuery.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/auth/ITSecurityBasicQuery.java new file mode 100644 index 000000000000..9d76af07de8e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/auth/ITSecurityBasicQuery.java @@ -0,0 +1,279 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.auth; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.sql.http.SqlQuery; +import org.apache.druid.storage.local.LocalFileExportStorageProvider; +import org.apache.druid.storage.s3.output.S3ExportStorageProvider; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.clients.SecurityClient; +import org.apache.druid.testing.utils.DataLoaderHelper; +import org.apache.druid.testing.utils.MsqTestQueryHelper; +import org.apache.druid.testsEx.categories.Security; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutionException; + +@RunWith(DruidTestRunner.class) +@Category(Security.class) +public class ITSecurityBasicQuery +{ + @Inject + private MsqTestQueryHelper msqHelper; + + @Inject + private DataLoaderHelper dataLoaderHelper; + + @Inject + private CoordinatorResourceTestClient coordinatorClient; + @Inject + private SecurityClient securityClient; + + public static final String USER_1 = "user1"; + public static final String ROLE_1 = "role1"; + public static final String USER_1_PASSWORD = "password1"; + + @Before + public void setUp() throws IOException + { + // Authentication setup + securityClient.createAuthenticationUser(USER_1); + securityClient.setUserPassword(USER_1, USER_1_PASSWORD); + + // Authorizer setup + securityClient.createAuthorizerUser(USER_1); + securityClient.createAuthorizerRole(ROLE_1); + securityClient.assignUserToRole(USER_1, ROLE_1); + } + + @After + public void tearDown() throws Exception + { + securityClient.deleteAuthenticationUser(USER_1); + securityClient.deleteAuthorizerUser(USER_1); + securityClient.deleteAuthorizerRole(ROLE_1); + } + + @Test + public void testIngestionWithoutPermissions() throws Exception + { + List permissions = ImmutableList.of(); + securityClient.setPermissionsToRole(ROLE_1, permissions); + + String queryLocal = + StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot,\n" + + " diffUrl,\n" + + " added,\n" + + " countryIsoCode,\n" + + " regionName,\n" + + " channel,\n" + + " flags,\n" + + " delta,\n" + + " isUnpatrolled,\n" + + " isNew,\n" + + " deltaBucket,\n" + + " isMinor,\n" + + " isAnonymous,\n" + + " deleted,\n" + + " cityName,\n" + + " metroCode,\n" + + " namespace,\n" + + " comment,\n" + + " page,\n" + + " commentLength,\n" + + " countryName,\n" + + " user,\n" + + " regionIsoCode\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY DAY\n", + "dst" + ); + + // Submit the task and wait for the datasource to get loaded + StatusResponseHolder statusResponseHolder = msqHelper.submitMsqTask( + new SqlQuery(queryLocal, null, false, false, false, ImmutableMap.of(), ImmutableList.of()), + USER_1, + USER_1_PASSWORD + ); + + Assert.assertEquals(HttpResponseStatus.FORBIDDEN, statusResponseHolder.getStatus()); + } + + @Test + public void testIngestionWithPermissions() throws Exception + { + List permissions = ImmutableList.of( + new ResourceAction(new Resource(".*", "DATASOURCE"), Action.READ), + new ResourceAction(new Resource("EXTERNAL", "EXTERNAL"), Action.READ), + new ResourceAction(new Resource("STATE", "STATE"), Action.READ), + new ResourceAction(new Resource(".*", "DATASOURCE"), Action.WRITE) + ); + securityClient.setPermissionsToRole(ROLE_1, permissions); + + String queryLocal = + StringUtils.format( + "INSERT INTO %s\n" + + "SELECT\n" + + " TIME_PARSE(\"timestamp\") AS __time,\n" + + " isRobot,\n" + + " diffUrl,\n" + + " added,\n" + + " countryIsoCode,\n" + + " regionName,\n" + + " channel,\n" + + " flags,\n" + + " delta,\n" + + " isUnpatrolled,\n" + + " isNew,\n" + + " deltaBucket,\n" + + " isMinor,\n" + + " isAnonymous,\n" + + " deleted,\n" + + " cityName,\n" + + " metroCode,\n" + + " namespace,\n" + + " comment,\n" + + " page,\n" + + " commentLength,\n" + + " countryName,\n" + + " user,\n" + + " regionIsoCode\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n" + + "PARTITIONED BY DAY\n", + "dst" + ); + + // Submit the task and wait for the datasource to get loaded + StatusResponseHolder statusResponseHolder = msqHelper.submitMsqTask( + new SqlQuery(queryLocal, null, false, false, false, ImmutableMap.of(), ImmutableList.of()), + USER_1, + USER_1_PASSWORD + ); + + Assert.assertEquals(HttpResponseStatus.ACCEPTED, statusResponseHolder.getStatus()); + } + + @Test + public void testExportWithoutPermissions() throws IOException, ExecutionException, InterruptedException + { + // No external write permissions for s3 + List permissions = ImmutableList.of( + new ResourceAction(new Resource(".*", "DATASOURCE"), Action.READ), + new ResourceAction(new Resource("EXTERNAL", "EXTERNAL"), Action.READ), + new ResourceAction(new Resource(S3ExportStorageProvider.TYPE_NAME, "EXTERNAL"), Action.WRITE), + new ResourceAction(new Resource("STATE", "STATE"), Action.READ), + new ResourceAction(new Resource(".*", "DATASOURCE"), Action.WRITE) + ); + securityClient.setPermissionsToRole(ROLE_1, permissions); + + String exportQuery = + StringUtils.format( + "INSERT INTO extern(%s(exportPath => '%s'))\n" + + "AS CSV\n" + + "SELECT page, added, delta\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n", + LocalFileExportStorageProvider.TYPE_NAME, "/shared/export/" + ); + + StatusResponseHolder statusResponseHolder = msqHelper.submitMsqTask( + new SqlQuery(exportQuery, null, false, false, false, ImmutableMap.of(), ImmutableList.of()), + USER_1, + USER_1_PASSWORD + ); + + Assert.assertEquals(HttpResponseStatus.FORBIDDEN, statusResponseHolder.getStatus()); + } + + @Test + public void testExportWithPermissions() throws IOException, ExecutionException, InterruptedException + { + // No external write permissions for s3 + List permissions = ImmutableList.of( + new ResourceAction(new Resource(".*", "DATASOURCE"), Action.READ), + new ResourceAction(new Resource("EXTERNAL", "EXTERNAL"), Action.READ), + new ResourceAction(new Resource(LocalFileExportStorageProvider.TYPE_NAME, "EXTERNAL"), Action.WRITE), + new ResourceAction(new Resource("STATE", "STATE"), Action.READ), + new ResourceAction(new Resource(".*", "DATASOURCE"), Action.WRITE) + ); + securityClient.setPermissionsToRole(ROLE_1, permissions); + + String exportQuery = + StringUtils.format( + "INSERT INTO extern(%s(exportPath => '%s'))\n" + + "AS CSV\n" + + "SELECT page, added, delta\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n", + LocalFileExportStorageProvider.TYPE_NAME, "/shared/export/" + ); + + StatusResponseHolder statusResponseHolder = msqHelper.submitMsqTask( + new SqlQuery(exportQuery, null, false, false, false, ImmutableMap.of(), ImmutableList.of()), + USER_1, + USER_1_PASSWORD + ); + + Assert.assertEquals(HttpResponseStatus.ACCEPTED, statusResponseHolder.getStatus()); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Security.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Security.java new file mode 100644 index 000000000000..07054e1f4204 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Security.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.categories; + +public class Security +{ +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java index 6f1e336c935f..d420b65dd8df 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java @@ -60,7 +60,9 @@ import org.apache.druid.java.util.emitter.core.LoggingEmitter; import org.apache.druid.java.util.emitter.core.LoggingEmitterConfig; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.http.client.CredentialedHttpClient; import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.auth.BasicCredentials; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageProvider; @@ -75,6 +77,7 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.IntegrationTestingConfigProvider; +import org.apache.druid.testing.clients.AdminClient; import org.apache.druid.testing.guice.TestClient; import org.apache.druid.testsEx.cluster.DruidClusterClient; import org.apache.druid.testsEx.cluster.MetastoreClient; @@ -187,6 +190,14 @@ public HttpClient getHttpClient( return delegate; } + @Provides + @AdminClient + public HttpClient getAdminClient(@Client HttpClient delegate) + { + BasicCredentials basicCredentials = new BasicCredentials("admin", "priest"); + return new CredentialedHttpClient(basicCredentials, delegate); + } + @Provides @ManageLifecycle public ServiceEmitter getServiceEmitter(ObjectMapper jsonMapper) diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java index b0080158905e..ad0b764e70fe 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java @@ -106,7 +106,7 @@ public void testMsqIngestionParallelMerging() throws Exception // Submit the task and wait for the datasource to get loaded SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(sqlQuery); if (sqlTaskStatus.getState().isFailure()) { Assert.fail(StringUtils.format( @@ -176,7 +176,7 @@ public void testMsqIngestionSequentialMerging() throws Exception // Submit the task and wait for the datasource to get loaded SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(sqlQuery); if (sqlTaskStatus.getState().isFailure()) { Assert.fail(StringUtils.format( @@ -251,7 +251,7 @@ public void testMsqIngestionSequentialMergingWithEmptyStatistics() throws Except // Submit the task and wait for the datasource to get loaded SqlQuery sqlQuery = new SqlQuery(queryLocal, null, false, false, false, context, null); - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(sqlQuery); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(sqlQuery); if (sqlTaskStatus.getState().isFailure()) { Assert.fail(StringUtils.format( diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java index a8677a53e734..38681c3d63a1 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java @@ -109,7 +109,7 @@ public void testMsqIngestionAndQuerying() throws Exception ); // Submit the task and wait for the datasource to get loaded - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(queryLocal); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryLocal); if (sqlTaskStatus.getState().isFailure()) { Assert.fail(StringUtils.format( @@ -175,7 +175,7 @@ public void testMsqIngestionAndQueryingWithLocalFn() throws Exception ); // Submit the task and wait for the datasource to get loaded - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask(queryLocal); + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryLocal); if (sqlTaskStatus.getState().isFailure()) { Assert.fail(StringUtils.format( @@ -208,7 +208,7 @@ public void testExport() throws Exception LocalFileExportStorageProvider.TYPE_NAME, "/shared/export/" ); - SqlTaskStatus exportTask = msqHelper.submitMsqTask(exportQuery); + SqlTaskStatus exportTask = msqHelper.submitMsqTaskSuccesfully(exportQuery); msqHelper.pollTaskIdForSuccess(exportTask.getTaskId()); @@ -230,7 +230,7 @@ public void testExport() throws Exception + " WHERE delta != 0\n" + " ORDER BY page"); - SqlTaskStatus resultTaskStatus = msqHelper.submitMsqTask(resultQuery); + SqlTaskStatus resultTaskStatus = msqHelper.submitMsqTaskSuccesfully(resultQuery); msqHelper.pollTaskIdForSuccess(resultTaskStatus.getTaskId()); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java index f7e414d7afda..37d99b7d9b93 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java @@ -110,7 +110,7 @@ public void testMsqIngestionAndQuerying() throws Exception ); // Submit the task and wait for the datasource to get loaded - SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTask( + SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully( queryLocal, ImmutableMap.of( MultiStageQueryContext.CTX_FAULT_TOLERANCE, diff --git a/helm/druid/Chart.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Security/docker.yaml similarity index 53% rename from helm/druid/Chart.yaml rename to integration-tests-ex/cases/src/test/resources/cluster/Security/docker.yaml index aeb629d9fcf2..8fab9c0992c8 100644 --- a/helm/druid/Chart.yaml +++ b/integration-tests-ex/cases/src/test/resources/cluster/Security/docker.yaml @@ -12,30 +12,29 @@ # 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. +#------------------------------------------------------------------------- -apiVersion: v2 -appVersion: 24.0.0 -description: Apache Druid is a high performance real-time analytics database. -name: druid -dependencies: - - name: zookeeper - version: 2.1.4 - repository: https://charts.helm.sh/incubator - condition: zookeeper.enabled - - name: mysql - version: 1.6.4 - repository: https://charts.helm.sh/stable - condition: mysql.enabled - - name: postgresql - version: 8.6.4 - repository: https://charts.helm.sh/stable - condition: postgresql.enabled -version: 0.3.5 -home: https://druid.apache.org/ -icon: https://druid.apache.org/img/favicon.png -sources: - - https://github.com/apache/druid -keywords: - - olap - - database - - analytics +# Definition of the multi stage query test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + indexer: + instances: + - port: 8091 diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java index 9b7911cc84f5..c558ff91ca3d 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java @@ -204,10 +204,16 @@ public List> query(String url, QueryType query) } public Future queryAsync(String url, QueryType query) + { + return queryAsync(url, query, null, null); + } + + public Future queryAsync(String url, QueryType query, String username, String password) { try { Request request = new Request(HttpMethod.POST, new URL(url)); request.setContent(MediaType.APPLICATION_JSON, encoderDecoderMap.get(MediaType.APPLICATION_JSON).encode(query)); + request.setBasicAuthentication(username, password); return httpClient.go( request, StatusResponseHandler.getInstance() diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/AdminClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/AdminClient.java new file mode 100644 index 000000000000..750ed0878f26 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/AdminClient.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testing.clients; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface AdminClient +{ +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/SecurityClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/SecurityClient.java new file mode 100644 index 000000000000..5d8075bd696a --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/SecurityClient.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testing.clients; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.testng.Assert; + +import javax.ws.rs.core.MediaType; +import java.io.IOException; +import java.net.URL; +import java.util.List; + +public class SecurityClient +{ + private final ObjectMapper jsonMapper; + private final HttpClient httpClient; + private final String coordinator; + private final StatusResponseHandler responseHandler; + + @Inject + SecurityClient( + ObjectMapper jsonMapper, + @AdminClient HttpClient httpClient, + IntegrationTestingConfig config + ) + { + this.jsonMapper = jsonMapper; + this.httpClient = httpClient; + this.coordinator = config.getCoordinatorUrl(); + this.responseHandler = StatusResponseHandler.getInstance(); + } + + public void createAuthenticationUser(String username) throws IOException + { + final Request request = new Request( + HttpMethod.POST, + new URL( + StringUtils.format( + "%s/users/%s", + getAuthenticatorURL(), + StringUtils.urlEncode(username) + ) + ) + ); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void deleteAuthenticationUser(String username) throws IOException + { + final Request request = new Request( + HttpMethod.DELETE, + new URL( + StringUtils.format( + "%s/users/%s", + getAuthenticatorURL(), + StringUtils.urlEncode(username) + ) + ) + ); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void setUserPassword(String username, String password) throws IOException + { + final Request request = new Request( + HttpMethod.POST, + new URL( + StringUtils.format( + "%s/users/%s/credentials", + getAuthenticatorURL(), + StringUtils.urlEncode(username) + ) + ) + ); + + request.setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(ImmutableMap.of("password", password))); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void createAuthorizerUser(String username) throws IOException + { + final Request request = new Request( + HttpMethod.POST, + new URL( + StringUtils.format( + "%s/users/%s", + getAuthorizerURL(), + StringUtils.urlEncode(username) + ) + ) + ); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void deleteAuthorizerUser(String username) throws IOException + { + final Request request = new Request( + HttpMethod.DELETE, + new URL( + StringUtils.format( + "%s/users/%s", + getAuthorizerURL(), + StringUtils.urlEncode(username) + ) + ) + ); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void createAuthorizerRole(String role) throws IOException + { + final Request request = new Request( + HttpMethod.POST, + new URL( + StringUtils.format( + "%s/roles/%s", + getAuthorizerURL(), + StringUtils.urlEncode(role) + ) + ) + ); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void deleteAuthorizerRole(String role) throws IOException + { + final Request request = new Request( + HttpMethod.DELETE, + new URL( + StringUtils.format( + "%s/roles/%s", + getAuthorizerURL(), + StringUtils.urlEncode(role) + ) + ) + ); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void assignUserToRole(String user, String role) throws IOException + { + final Request request = new Request( + HttpMethod.POST, + new URL( + StringUtils.format( + "%s/users/%s/roles/%s", + getAuthorizerURL(), + StringUtils.urlEncode(user), + StringUtils.urlEncode(role) + ) + ) + ); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + public void setPermissionsToRole(String role, List permissions) throws IOException + { + final Request request = new Request( + HttpMethod.POST, + new URL( + StringUtils.format( + "%s/roles/%s/permissions/", + getAuthorizerURL(), + StringUtils.urlEncode(role) + ) + ) + ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(permissions)); + Assert.assertEquals(HttpResponseStatus.OK, sendRequest(request).getStatus()); + } + + private StatusResponseHolder sendRequest(Request request) + { + try { + final StatusResponseHolder response = httpClient.go( + request, + responseHandler + ).get(); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while creating users status [%s] content [%s]", + response.getStatus(), + response.getContent() + ); + } + + return response; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private String getAuthenticatorURL() + { + return StringUtils.format( + "%s/druid-ext/basic-security/authentication/db/basic", + coordinator + ); + } + + private String getAuthorizerURL() + { + return StringUtils.format( + "%s/druid-ext/basic-security/authorization/db/basic", + coordinator + ); + } +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java index a7510db860f7..3d20e31f566a 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/MsqTestQueryHelper.java @@ -93,17 +93,22 @@ public String getQueryURL(String schemeAndHost) /** * Submits a task to the MSQ API with the given query string, and default headers and parameters */ - public SqlTaskStatus submitMsqTask(String sqlQueryString) throws ExecutionException, InterruptedException + public SqlTaskStatus submitMsqTaskSuccesfully(String sqlQueryString) throws ExecutionException, InterruptedException { - return submitMsqTask(sqlQueryString, ImmutableMap.of()); + return submitMsqTaskSuccesfully(sqlQueryString, ImmutableMap.of()); } /** * Submits a task to the MSQ API with the given query string, and default headers and custom context parameters */ - public SqlTaskStatus submitMsqTask(String sqlQueryString, Map context) throws ExecutionException, InterruptedException + public SqlTaskStatus submitMsqTaskSuccesfully(String sqlQueryString, Map context) throws ExecutionException, InterruptedException { - return submitMsqTask(new SqlQuery(sqlQueryString, null, false, false, false, context, null)); + return submitMsqTaskSuccesfully(new SqlQuery(sqlQueryString, null, false, false, false, context, null), null, null); + } + + public SqlTaskStatus submitMsqTaskSuccesfully(SqlQuery sqlQuery) throws ExecutionException, InterruptedException + { + return submitMsqTaskSuccesfully(sqlQuery, null, null); } // Run the task, wait for it to complete, fetch the reports, verify the results, @@ -112,20 +117,9 @@ public SqlTaskStatus submitMsqTask(String sqlQueryString, Map co * Submits a {@link SqlQuery} to the MSQ API for execution. This method waits for the task to be accepted by the cluster * and returns the status associated with the submitted task */ - public SqlTaskStatus submitMsqTask(SqlQuery sqlQuery) throws ExecutionException, InterruptedException + public SqlTaskStatus submitMsqTaskSuccesfully(SqlQuery sqlQuery, String username, String password) throws ExecutionException, InterruptedException { - String queryUrl = getQueryURL(config.getBrokerUrl()); - Future responseHolderFuture = msqClient.queryAsync(queryUrl, sqlQuery); - // It is okay to block here for the result because MSQ tasks return the task id associated with it, which shouldn't - // consume a lot of time - StatusResponseHolder statusResponseHolder; - try { - statusResponseHolder = responseHolderFuture.get(5, TimeUnit.MINUTES); - } - catch (TimeoutException e) { - throw new ISE(e, "Unable to fetch the task id for the submitted task in time."); - } - + StatusResponseHolder statusResponseHolder = submitMsqTask(sqlQuery, username, password); // Check if the task has been accepted successfully HttpResponseStatus httpResponseStatus = statusResponseHolder.getStatus(); if (!httpResponseStatus.equals(HttpResponseStatus.ACCEPTED)) { @@ -148,6 +142,20 @@ public SqlTaskStatus submitMsqTask(SqlQuery sqlQuery) throws ExecutionException, return sqlTaskStatus; } + public StatusResponseHolder submitMsqTask(SqlQuery sqlQuery, String username, String password) throws ExecutionException, InterruptedException + { + String queryUrl = getQueryURL(config.getBrokerUrl()); + Future responseHolderFuture = msqClient.queryAsync(queryUrl, sqlQuery, username, password); + // It is okay to block here for the result because MSQ tasks return the task id associated with it, which shouldn't + // consume a lot of time + try { + return responseHolderFuture.get(5, TimeUnit.MINUTES); + } + catch (TimeoutException e) { + throw new ISE(e, "Unable to fetch the task id for the submitted task in time."); + } + } + /** * The method retries till the task with taskId gets completed i.e. {@link TaskState#isComplete()}} returns true and * returns the last fetched state {@link TaskState} of the task @@ -253,7 +261,7 @@ public void testQueriesFromFile(String filePath, String fullDatasourcePath) thro for (MsqQueryWithResults queryWithResults : queries) { String queryString = queryWithResults.getQuery(); String queryWithDatasource = StringUtils.replace(queryString, "%%DATASOURCE%%", fullDatasourcePath); - SqlTaskStatus sqlTaskStatus = submitMsqTask(queryWithDatasource); + SqlTaskStatus sqlTaskStatus = submitMsqTaskSuccesfully(queryWithDatasource); if (sqlTaskStatus.getState().isFailure()) { throw new ISE( "Unable to start the task successfully.\nPossible exception: %s", @@ -272,7 +280,7 @@ public void testQueriesFromFile(String filePath, String fullDatasourcePath) thro public void submitMsqTaskAndWaitForCompletion(String sqlQueryString, Map context) throws Exception { - SqlTaskStatus sqlTaskStatus = submitMsqTask(sqlQueryString, context); + SqlTaskStatus sqlTaskStatus = submitMsqTaskSuccesfully(sqlQueryString, context); LOG.info("Sql Task submitted with task Id - %s", sqlTaskStatus.getTaskId()); diff --git a/licenses.yaml b/licenses.yaml index 48df6b1a157c..1d3393058994 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -397,7 +397,7 @@ name: JsonPath license_category: binary module: java-core license_name: Apache License version 2.0 -version: 2.3.0 +version: 2.9.0 libraries: - com.jayway.jsonpath: json-path diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml index cf95771fa14e..2cfbc334edb7 100644 --- a/owasp-dependency-check-suppressions.xml +++ b/owasp-dependency-check-suppressions.xml @@ -21,7 +21,7 @@ CVE-2022-45688 CVE-2023-35116 diff --git a/pom.xml b/pom.xml index c5c2a91ae49c..40051638767b 100644 --- a/pom.xml +++ b/pom.xml @@ -927,7 +927,7 @@ com.jayway.jsonpath json-path - 2.3.0 + 2.9.0 net.thisptr @@ -1262,6 +1262,12 @@ ${hamcrest.version} test + + org.reflections + reflections + 0.9.12 + test + pl.pragmatists JUnitParams diff --git a/processing/pom.xml b/processing/pom.xml index 9abe06439e41..bc1365a6f138 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -380,7 +380,6 @@ org.reflections reflections - 0.9.12 test diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/DoubleArrayFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/DoubleArrayFrameColumnReader.java new file mode 100644 index 000000000000..909f2c5b3641 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/DoubleArrayFrameColumnReader.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.read.columnar; + +import com.google.common.math.LongMath; +import org.apache.datasketches.memory.Memory; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.write.columnar.FrameColumnWriters; +import org.apache.druid.segment.column.ColumnType; + +/** + * Reader for columns written by {@link org.apache.druid.frame.write.columnar.DoubleArrayFrameColumnWriter} + * + * @see NumericArrayFrameColumnReader + * @see org.apache.druid.frame.write.columnar.NumericArrayFrameColumnWriter for column's layout in memory + */ +public class DoubleArrayFrameColumnReader extends NumericArrayFrameColumnReader +{ + public DoubleArrayFrameColumnReader(int columnNumber) + { + super(FrameColumnWriters.TYPE_DOUBLE_ARRAY, ColumnType.DOUBLE_ARRAY, columnNumber); + } + + @Override + NumericArrayFrameColumn column(Frame frame, Memory memory, ColumnType columnType) + { + return new DoubleArrayFrameColumn(frame, memory, columnType); + } + + private static class DoubleArrayFrameColumn extends NumericArrayFrameColumn + { + public DoubleArrayFrameColumn( + Frame frame, + Memory memory, + ColumnType columnType + ) + { + super(frame, memory, columnType); + } + + @Override + Number getElement(Memory memory, long rowDataOffset, int cumulativeIndex) + { + return memory.getDouble(LongMath.checkedAdd(rowDataOffset, (long) cumulativeIndex * Double.BYTES)); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FloatArrayFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FloatArrayFrameColumnReader.java new file mode 100644 index 000000000000..ea1ffbdd060a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FloatArrayFrameColumnReader.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.read.columnar; + +import com.google.common.math.LongMath; +import org.apache.datasketches.memory.Memory; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.write.columnar.FrameColumnWriters; +import org.apache.druid.segment.column.ColumnType; + +/** + * Reader for columns written by {@link org.apache.druid.frame.write.columnar.FloatArrayFrameColumnWriter} + * + * @see NumericArrayFrameColumnReader + * @see org.apache.druid.frame.write.columnar.NumericArrayFrameColumnWriter for column's layout in memory + */ +public class FloatArrayFrameColumnReader extends NumericArrayFrameColumnReader +{ + public FloatArrayFrameColumnReader(int columnNumber) + { + super(FrameColumnWriters.TYPE_FLOAT_ARRAY, ColumnType.FLOAT_ARRAY, columnNumber); + } + + @Override + NumericArrayFrameColumn column(Frame frame, Memory memory, ColumnType columnType) + { + return new FloatArrayFrameColumn(frame, memory, columnType); + } + + private static class FloatArrayFrameColumn extends NumericArrayFrameColumn + { + public FloatArrayFrameColumn( + Frame frame, + Memory memory, + ColumnType columnType + ) + { + super(frame, memory, columnType); + } + + @Override + Number getElement(Memory memory, long rowDataOffset, int cumulativeIndex) + { + return memory.getFloat(LongMath.checkedAdd(rowDataOffset, (long) cumulativeIndex * Float.BYTES)); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaderUtils.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaderUtils.java new file mode 100644 index 000000000000..c0dfdc6a76f1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaderUtils.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.read.columnar; + +import org.apache.datasketches.memory.Memory; + +public class FrameColumnReaderUtils +{ + /** + * Adjusts a negative cumulative row length from {@link #getCumulativeRowLength(Memory, long, int)} to be the actual + * positive length. + */ + public static int adjustCumulativeRowLength(final int cumulativeRowLength) + { + if (cumulativeRowLength < 0) { + return -(cumulativeRowLength + 1); + } else { + return cumulativeRowLength; + } + } + + /** + * Returns cumulative row length, if the row is not null itself, or -(cumulative row length) - 1 if the row is + * null itself. + * + * To check if the return value from this function indicate a null row, use {@link #isNullRow(int)} + * + * To get the actual cumulative row length, use {@link FrameColumnReaderUtils#adjustCumulativeRowLength(int)}. + */ + public static int getCumulativeRowLength(final Memory memory, final long offset, final int physicalRow) + { + // Note: only valid to call this if multiValue = true. + return memory.getInt(offset + (long) Integer.BYTES * physicalRow); + } + + public static int getAdjustedCumulativeRowLength(final Memory memory, final long offset, final int physicalRow) + { + return adjustCumulativeRowLength(getCumulativeRowLength(memory, offset, physicalRow)); + } + + /** + * When given a return value from {@link FrameColumnReaderUtils#getCumulativeRowLength(Memory, long, int)}, returns whether the row is + * null itself (i.e. a null array). + */ + public static boolean isNullRow(final int cumulativeRowLength) + { + return cumulativeRowLength < 0; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java index 98218819ce13..9b4dc85cb1e0 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java @@ -20,7 +20,6 @@ package org.apache.druid.frame.read.columnar; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.ValueType; /** * Creates {@link FrameColumnReader} corresponding to a given column type and number. @@ -58,12 +57,18 @@ public static FrameColumnReader create( return new ComplexFrameColumnReader(columnNumber); case ARRAY: - if (columnType.getElementType().getType() == ValueType.STRING) { - return new StringFrameColumnReader(columnNumber, true); - } else { - return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType); + switch (columnType.getElementType().getType()) { + case STRING: + return new StringFrameColumnReader(columnNumber, true); + case LONG: + return new LongArrayFrameColumnReader(columnNumber); + case FLOAT: + return new FloatArrayFrameColumnReader(columnNumber); + case DOUBLE: + return new DoubleArrayFrameColumnReader(columnNumber); + default: + return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType); } - default: return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType); } diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/LongArrayFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/LongArrayFrameColumnReader.java new file mode 100644 index 000000000000..898e1f1cebb1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/LongArrayFrameColumnReader.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.read.columnar; + +import com.google.common.math.LongMath; +import org.apache.datasketches.memory.Memory; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.write.columnar.FrameColumnWriters; +import org.apache.druid.segment.column.ColumnType; + +/** + * Reader for columns written by {@link org.apache.druid.frame.write.columnar.LongArrayFrameColumnWriter} + * + * @see NumericArrayFrameColumnReader + * @see org.apache.druid.frame.write.columnar.NumericArrayFrameColumnWriter for column's layout in memory + */ +public class LongArrayFrameColumnReader extends NumericArrayFrameColumnReader +{ + public LongArrayFrameColumnReader(int columnNumber) + { + super(FrameColumnWriters.TYPE_LONG_ARRAY, ColumnType.LONG_ARRAY, columnNumber); + } + + @Override + NumericArrayFrameColumn column(Frame frame, Memory memory, ColumnType columnType) + { + return new LongArrayFrameColumn(frame, memory, columnType); + } + + private static class LongArrayFrameColumn extends NumericArrayFrameColumn + { + public LongArrayFrameColumn( + Frame frame, + Memory memory, + ColumnType columnType + ) + { + super(frame, memory, columnType); + } + + @Override + Number getElement(Memory memory, long rowDataOffset, int cumulativeIndex) + { + return memory.getLong(LongMath.checkedAdd(rowDataOffset, (long) cumulativeIndex * Long.BYTES)); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/NumericArrayFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/NumericArrayFrameColumnReader.java new file mode 100644 index 000000000000..986baaa099d9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/NumericArrayFrameColumnReader.java @@ -0,0 +1,360 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.read.columnar; + +import com.google.common.math.LongMath; +import it.unimi.dsi.fastutil.objects.ObjectArrays; +import org.apache.datasketches.memory.Memory; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.write.columnar.NumericArrayFrameColumnWriter; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.column.accessor.ObjectColumnAccessorBase; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.ObjectColumnSelector; +import org.apache.druid.segment.column.BaseColumn; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.vector.ReadableVectorInspector; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.VectorObjectSelector; + +import javax.annotation.Nullable; +import java.util.Comparator; + +/** + * Implementations of this class reads columns written by the corresponding implementations of {@link NumericArrayFrameColumnWriter}. + * + * @see NumericArrayFrameColumnWriter for the column format read + */ +public abstract class NumericArrayFrameColumnReader implements FrameColumnReader +{ + private final byte typeCode; + private final ColumnType columnType; + private final int columnNumber; + + public NumericArrayFrameColumnReader(byte typeCode, ColumnType columnType, int columnNumber) + { + this.typeCode = typeCode; + this.columnType = columnType; + this.columnNumber = columnNumber; + } + + @Override + public Column readRACColumn(Frame frame) + { + final Memory memory = frame.region(columnNumber); + validate(memory); + return new ColumnAccessorBasedColumn(column(frame, memory, columnType)); + } + + @Override + public ColumnPlus readColumn(Frame frame) + { + final Memory memory = frame.region(columnNumber); + validate(memory); + return new ColumnPlus( + column(frame, memory, columnType), + ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(columnType), + frame.numRows() + ); + } + + abstract NumericArrayFrameColumn column(Frame frame, Memory memory, ColumnType columnType); + + /** + * Validates that the written type code is the same as the provided type code. It's a defensive check that prevents + * unexpected results by reading columns of different types + */ + private void validate(final Memory region) + { + if (region.getCapacity() < NumericArrayFrameColumnWriter.DATA_OFFSET) { + throw DruidException.defensive("Column[%s] is not big enough for a header", columnNumber); + } + final byte actualTypeCode = region.getByte(0); + if (actualTypeCode != this.typeCode) { + throw DruidException.defensive( + "Column[%s] does not have the correct type code; expected[%s], got[%s]", + columnNumber, + this.typeCode, + actualTypeCode + ); + } + } + + /** + * Gets the start of the section where cumulative lengths of the array elements are stored (section 1) + */ + private static long getStartOfCumulativeLengthSection() + { + return NumericArrayFrameColumnWriter.DATA_OFFSET; + } + + /** + * Gets the start of the section where information about element's nullity is stored (section 2) + */ + private static long getStartOfRowNullityData(final int numRows) + { + return getStartOfCumulativeLengthSection() + ((long) numRows * Integer.BYTES); + } + + /** + * Gets the start of the section where elements are stored (section 3) + */ + private static long getStartOfRowData(final Memory memory, final int numRows) + { + long nullityDataOffset = + (long) Byte.BYTES * FrameColumnReaderUtils.getAdjustedCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + numRows - 1 + ); + return LongMath.checkedAdd(getStartOfRowNullityData(numRows), nullityDataOffset); + } + + public abstract static class NumericArrayFrameColumn extends ObjectColumnAccessorBase implements BaseColumn + { + private final Frame frame; + private final Memory memory; + private final ColumnType columnType; + + /** + * Cache start of rowNullityDataOffset, as it won't change + */ + private final long rowNullityDataOffset; + + /** + * Cache start of rowDataOffset, as it won't change + */ + private final long rowDataOffset; + + + public NumericArrayFrameColumn(Frame frame, Memory memory, ColumnType columnType) + { + this.frame = frame; + this.memory = memory; + this.columnType = columnType; + + this.rowNullityDataOffset = getStartOfRowNullityData(frame.numRows()); + this.rowDataOffset = getStartOfRowData(memory, frame.numRows()); + } + + @Override + public ColumnType getType() + { + return columnType; + } + + @Override + public int numRows() + { + return frame.numRows(); + } + + @Override + protected Object getVal(int rowNum) + { + return getNumericArray(physicalRow(rowNum)); + } + + @Override + protected Comparator getComparator() + { + return columnType.getNullableStrategy(); + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + // Cache's the row's value before returning + return new ObjectColumnSelector() + { + + // Cached row number + private int cachedLogicalRow = -1; + + // Cached value + @Nullable + private Object[] cachedValue = null; + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } + + @Nullable + @Override + public Object getObject() + { + compute(); + return cachedValue; + } + + @Override + public Class classOfObject() + { + return Object[].class; + } + + /** + * Cache's the row value and the logical row number into the class variables + */ + private void compute() + { + int currentLogicalRow = offset.getOffset(); + if (cachedLogicalRow == currentLogicalRow) { + return; + } + cachedValue = getNumericArray(physicalRow(currentLogicalRow)); + cachedLogicalRow = currentLogicalRow; + } + }; + } + + @Override + public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset) + { + return new VectorObjectSelector() + { + private final Object[] vector = new Object[offset.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + @Override + public Object[] getObjectVector() + { + computeVector(); + return vector; + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + + private void computeVector() + { + if (id == offset.getId()) { + return; + } + + if (offset.isContiguous()) { + // Contiguous offsets can have a cache optimized implementation if 'frame.isPermuted() == false', + // i.e. logicalRow == physicalRow. The implementation can separately fetch out the nullity data, and the + // element data continguously. + final int start = offset.getStartOffset(); + for (int i = 0; i < offset.getCurrentVectorSize(); ++i) { + vector[i] = getNumericArray(physicalRow(start + i)); + } + } else { + final int[] offsets = offset.getOffsets(); + for (int i = 0; i < offset.getCurrentVectorSize(); ++i) { + vector[i] = getNumericArray(physicalRow(offsets[i])); + } + + id = offset.getId(); + } + } + }; + } + + @Override + public void close() + { + // Do nothing + } + + private int physicalRow(int logicalRow) + { + return frame.physicalRow(logicalRow); + } + + /** + * Given the physical row, it fetches the value from the memory + */ + @Nullable + private Object[] getNumericArray(final int physicalRow) + { + final int cumulativeLength = FrameColumnReaderUtils.getCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow + ); + + final int rowLength; + if (FrameColumnReaderUtils.isNullRow(cumulativeLength)) { + return null; + } else if (physicalRow == 0) { + rowLength = cumulativeLength; + } else { + final int previousCumulativeLength = FrameColumnReaderUtils.adjustCumulativeRowLength( + FrameColumnReaderUtils.getCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow - 1 + ) + ); + // cumulativeLength doesn't need to be adjusted, since its greater than 0 or else it would have been a null row, + // which we check for in the first if..else + rowLength = cumulativeLength - previousCumulativeLength; + } + + if (rowLength == 0) { + return ObjectArrays.EMPTY_ARRAY; + } + + final Object[] row = new Object[rowLength]; + for (int i = 0; i < rowLength; ++i) { + final int cumulativeIndex = cumulativeLength - rowLength + i; + row[i] = getElementNullity(cumulativeIndex) ? null : getElement(memory, rowDataOffset, cumulativeIndex); + } + + return row; + } + + /** + * Returns true if element is null, else false + */ + private boolean getElementNullity(final int cumulativeIndex) + { + byte b = memory.getByte(LongMath.checkedAdd(rowNullityDataOffset, (long) cumulativeIndex * Byte.BYTES)); + if (b == NumericArrayFrameColumnWriter.NULL_ELEMENT_MARKER) { + return true; + } + assert b == NumericArrayFrameColumnWriter.NON_NULL_ELEMENT_MARKER; + return false; + } + + /** + * Returns the value of the element of the array in the memory provided, given that the start of the array is + * {@code rowDataOffset} and the index of the element in the array is {@code cumulativeIndex} + */ + abstract Number getElement(Memory memory, long rowDataOffset, int cumulativeIndex); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java index 119dd48a3f17..d9fb9d83a9f4 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java @@ -92,17 +92,18 @@ public Column readRACColumn(Frame frame) final Memory memory = frame.region(columnNumber); validate(memory); - if (isMultiValue(memory)) { - // When we implement handling of multi-value, we should actually make this look like an Array of String instead - // of perpetuating the multi-value idea. Thus, when we add support for Arrays to the RAC stuff, that's when - // we can start supporting multi-value. - throw new ISE("Multivalue not yet handled by RAC"); - } final long positionOfLengths = getStartOfStringLengthSection(frame.numRows(), false); final long positionOfPayloads = getStartOfStringDataSection(memory, frame.numRows(), false); StringFrameColumn frameCol = - new StringFrameColumn(frame, false, memory, positionOfLengths, positionOfPayloads, false); + new StringFrameColumn( + frame, + false, + memory, + positionOfLengths, + positionOfPayloads, + asArray || isMultiValue(memory) // Read MVDs as String arrays + ); return new ColumnAccessorBasedColumn(frameCol); } @@ -174,40 +175,9 @@ private static boolean isMultiValue(final Memory memory) return memory.getByte(1) == 1; } - /** - * Returns cumulative row length, if the row is not null itself, or -(cumulative row length) - 1 if the row is - * null itself. - * - * To check if the return value from this function indicate a null row, use {@link #isNullRow(int)} - * - * To get the actual cumulative row length, use {@link #adjustCumulativeRowLength(int)}. - */ - private static int getCumulativeRowLength(final Memory memory, final int physicalRow) - { - // Note: only valid to call this if multiValue = true. - return memory.getInt(StringFrameColumnWriter.DATA_OFFSET + (long) Integer.BYTES * physicalRow); - } - - /** - * When given a return value from {@link #getCumulativeRowLength(Memory, int)}, returns whether the row is - * null itself (i.e. a null array). - */ - private static boolean isNullRow(final int cumulativeRowLength) + private static long getStartOfCumulativeLengthSection() { - return cumulativeRowLength < 0; - } - - /** - * Adjusts a negative cumulative row length from {@link #getCumulativeRowLength(Memory, int)} to be the actual - * positive length. - */ - private static int adjustCumulativeRowLength(final int cumulativeRowLength) - { - if (cumulativeRowLength < 0) { - return -(cumulativeRowLength + 1); - } else { - return cumulativeRowLength; - } + return StringFrameColumnWriter.DATA_OFFSET; } private static long getStartOfStringLengthSection( @@ -231,7 +201,11 @@ private static long getStartOfStringDataSection( final int totalNumValues; if (multiValue) { - totalNumValues = adjustCumulativeRowLength(getCumulativeRowLength(memory, numRows - 1)); + totalNumValues = FrameColumnReaderUtils.getAdjustedCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + numRows - 1 + ); } else { totalNumValues = numRows; } @@ -489,15 +463,23 @@ private String getString(final int index) private Object getRowAsObject(final int physicalRow, final boolean decode) { if (multiValue) { - final int cumulativeRowLength = getCumulativeRowLength(memory, physicalRow); + final int cumulativeRowLength = FrameColumnReaderUtils.getCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow + ); final int rowLength; - if (isNullRow(cumulativeRowLength)) { + if (FrameColumnReaderUtils.isNullRow(cumulativeRowLength)) { return null; } else if (physicalRow == 0) { rowLength = cumulativeRowLength; } else { - rowLength = cumulativeRowLength - adjustCumulativeRowLength(getCumulativeRowLength(memory, physicalRow - 1)); + rowLength = cumulativeRowLength - FrameColumnReaderUtils.getAdjustedCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow - 1 + ); } if (rowLength == 0) { diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/DoubleArrayFrameColumnWriter.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/DoubleArrayFrameColumnWriter.java new file mode 100644 index 000000000000..80c1b5ebfdf6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/DoubleArrayFrameColumnWriter.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.write.columnar; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.segment.ColumnValueSelector; + +/** + * Columnar frame writer for {@link org.apache.druid.segment.column.ColumnType#DOUBLE_ARRAY} columns + */ +public class DoubleArrayFrameColumnWriter extends NumericArrayFrameColumnWriter +{ + public DoubleArrayFrameColumnWriter( + ColumnValueSelector selector, + MemoryAllocator allocator + ) + { + super(selector, allocator, FrameColumnWriters.TYPE_DOUBLE_ARRAY); + } + + @Override + int elementSizeBytes() + { + return Double.BYTES; + } + + @Override + void putNull(WritableMemory memory, long offset) + { + memory.putDouble(offset, 0d); + } + + @Override + void putArrayElement(WritableMemory memory, long offset, Number element) + { + // The element is of type Double, and non-null, therefore it can be cast safely + memory.putDouble(offset, (double) element); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/FloatArrayFrameColumnWriter.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/FloatArrayFrameColumnWriter.java new file mode 100644 index 000000000000..47c492c28bba --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/FloatArrayFrameColumnWriter.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.write.columnar; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.segment.ColumnValueSelector; + +/** + * Columnar frame writer for {@link org.apache.druid.segment.column.ColumnType#FLOAT_ARRAY} columns + */ +public class FloatArrayFrameColumnWriter extends NumericArrayFrameColumnWriter +{ + public FloatArrayFrameColumnWriter( + ColumnValueSelector selector, + MemoryAllocator allocator + ) + { + super(selector, allocator, FrameColumnWriters.TYPE_FLOAT_ARRAY); + } + + @Override + int elementSizeBytes() + { + return Float.BYTES; + } + + @Override + void putNull(WritableMemory memory, long offset) + { + memory.putFloat(offset, 0f); + } + + @Override + void putArrayElement(WritableMemory memory, long offset, Number element) + { + // The element is of type Float, and non-null, therefore it can be cast safely + memory.putFloat(offset, (float) element); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriter.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriter.java index bb954d6c366a..0a91a58bd830 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriter.java @@ -23,14 +23,34 @@ import java.io.Closeable; +/** + * Represent writers for the columnar frames. + * + * The class objects must be provided with information on what values to write, usually provided as a + * {@link org.apache.druid.segment.ColumnValueSelector} and where to write to, usually temporary growable memory + * {@link #addSelection()} will be called repeatedly, as the current value to write gets updated. For the final write, + * call {@link #writeTo}, which will copy the values we have added so far to the destination memory. + */ public interface FrameColumnWriter extends Closeable { + /** + * Adds the current value to the writer + */ boolean addSelection(); + /** + * Reverts the last added value. Undo calls cannot be called in successsion + */ void undo(); + /** + * Size (in bytes) of the column data that will get written when {@link #writeTo} will be called + */ long size(); + /** + * Writes the value of the column to the provided memory at the given position + */ long writeTo(WritableMemory memory, long position); @Override diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java index 8c5dbe758532..93f0c12bae6f 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java @@ -41,6 +41,9 @@ public class FrameColumnWriters public static final byte TYPE_STRING = 4; public static final byte TYPE_COMPLEX = 5; public static final byte TYPE_STRING_ARRAY = 6; + public static final byte TYPE_LONG_ARRAY = 7; + public static final byte TYPE_FLOAT_ARRAY = 8; + public static final byte TYPE_DOUBLE_ARRAY = 9; private FrameColumnWriters() { @@ -76,6 +79,12 @@ static FrameColumnWriter create( switch (type.getElementType().getType()) { case STRING: return makeStringArrayWriter(columnSelectorFactory, allocator, column); + case LONG: + return makeLongArrayWriter(columnSelectorFactory, allocator, column); + case FLOAT: + return makeFloatArrayWriter(columnSelectorFactory, allocator, column); + case DOUBLE: + return makeDoubleArrayWriter(columnSelectorFactory, allocator, column); default: throw new UnsupportedColumnTypeException(column, type); } @@ -144,6 +153,36 @@ private static StringFrameColumnWriter makeStringArrayWriter( return new StringArrayFrameColumnWriterImpl(selector, allocator); } + private static NumericArrayFrameColumnWriter makeLongArrayWriter( + final ColumnSelectorFactory selectorFactory, + final MemoryAllocator allocator, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return new LongArrayFrameColumnWriter(selector, allocator); + } + + private static NumericArrayFrameColumnWriter makeFloatArrayWriter( + final ColumnSelectorFactory selectorFactory, + final MemoryAllocator allocator, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return new FloatArrayFrameColumnWriter(selector, allocator); + } + + private static NumericArrayFrameColumnWriter makeDoubleArrayWriter( + final ColumnSelectorFactory selectorFactory, + final MemoryAllocator allocator, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return new DoubleArrayFrameColumnWriter(selector, allocator); + } + private static ComplexFrameColumnWriter makeComplexWriter( final ColumnSelectorFactory selectorFactory, final MemoryAllocator allocator, diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/LongArrayFrameColumnWriter.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/LongArrayFrameColumnWriter.java new file mode 100644 index 000000000000..cc26fd3a36a1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/LongArrayFrameColumnWriter.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.write.columnar; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.segment.ColumnValueSelector; + +/** + * Columnar frame writer for {@link org.apache.druid.segment.column.ColumnType#LONG_ARRAY} columns + */ +public class LongArrayFrameColumnWriter extends NumericArrayFrameColumnWriter +{ + public LongArrayFrameColumnWriter( + ColumnValueSelector selector, + MemoryAllocator allocator + ) + { + super(selector, allocator, FrameColumnWriters.TYPE_LONG_ARRAY); + } + + @Override + int elementSizeBytes() + { + return Long.BYTES; + } + + @Override + void putNull(WritableMemory memory, long offset) + { + memory.putLong(offset, 0L); + } + + @Override + void putArrayElement(WritableMemory memory, long offset, Number element) + { + // The element is of type Long, and non-null, therefore it can be casted safely + memory.putLong(offset, (long) element); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/NumericArrayFrameColumnWriter.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/NumericArrayFrameColumnWriter.java new file mode 100644 index 000000000000..619bf53b8d3d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/NumericArrayFrameColumnWriter.java @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.frame.write.columnar; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.allocation.AppendableMemory; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.allocation.MemoryRange; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.segment.ColumnValueSelector; + +import java.util.List; + +/** + * Parent class for the family of writers writing numeric arrays in columnar frames. Since the numeric primitives are + * fixed width, we don't need to store the width of each element. The memory layout of a column written by this writer + * is as follows: + * + * n : Total number of rows + * k : Total number of elements in all the rows, cumulative + * + * | Section | Length of the section | Denotion | + * |---------|-----------------------|--------------------------------------------------------------------------------------| + * | 0 | 1 | typeCode | + * | 1 | n * Integer.BYTES | n integers, where i-th integer represents the cumulative length of the array | + * | 2 | k * Byte.BYTES | k bytes, where i-th byte represent whether the i-th value from the start is null | + * | 3 | k * ELEMENT_SIZE | k values, each representing the element, or null equivalent value (e.g 0 for double) | + * + * Note on cumulative lengths stored in section 1: Cumulative lengths are stored so that its fast to offset into the + * elements of the array. We also use negative cumulative length to denote that the array itself is null (as opposed to + * individual elements being null, which we store in section 2) + */ +public abstract class NumericArrayFrameColumnWriter implements FrameColumnWriter +{ + /** + * Equivalent to {@link AppendableMemory#DEFAULT_INITIAL_ALLOCATION_SIZE} / 3, since the memory would be further split + * up into three regions + */ + private static final int INITIAL_ALLOCATION_SIZE = 120; + + public static final byte NULL_ELEMENT_MARKER = 0x00; + public static final byte NON_NULL_ELEMENT_MARKER = 0x01; + + /** + * A byte required at the beginning for type code + */ + public static final long DATA_OFFSET = 1; + + final ColumnValueSelector selector; + final byte typeCode; + + /** + * Row lengths: one int per row with the number of values contained by that row and all previous rows. + * Only written for multi-value and array columns. When the corresponding row is null itself, the length is + * written as -(actual length) - 1. (Guaranteed to be a negative number even if "actual length" is zero.) + */ + private final AppendableMemory cumulativeRowLengths; + + /** + * Denotes if the element of the row is null or not + */ + private final AppendableMemory rowNullityData; + + /** + * Row data. + */ + private final AppendableMemory rowData; + + private int lastCumulativeRowLength = 0; + private int lastRowLength = -1; + + + public NumericArrayFrameColumnWriter( + final ColumnValueSelector selector, + final MemoryAllocator allocator, + final byte typeCode + ) + { + this.selector = selector; + this.typeCode = typeCode; + this.cumulativeRowLengths = AppendableMemory.create(allocator, INITIAL_ALLOCATION_SIZE); + this.rowNullityData = AppendableMemory.create(allocator, INITIAL_ALLOCATION_SIZE); + this.rowData = AppendableMemory.create(allocator, INITIAL_ALLOCATION_SIZE); + } + + /** + * Returns the size of the elements of the array + */ + abstract int elementSizeBytes(); + + /** + * Inserts default null value in the given memory location at the provided offset. + */ + abstract void putNull(WritableMemory memory, long offset); + + /** + * Inserts the element value in the given memory location at the provided offset. + */ + abstract void putArrayElement(WritableMemory memory, long offset, Number element); + + @Override + public boolean addSelection() + { + List numericArray = FrameWriterUtils.getNumericArrayFromObject(selector.getObject()); + int rowLength = numericArray == null ? 0 : numericArray.size(); + + // Begin memory allocations before writing + if ((long) lastCumulativeRowLength + rowLength > Integer.MAX_VALUE) { + return false; + } + + if (!cumulativeRowLengths.reserveAdditional(Integer.BYTES)) { + return false; + } + + if (!rowNullityData.reserveAdditional(rowLength * Byte.BYTES)) { + return false; + } + + if (!rowData.reserveAdditional(rowLength * elementSizeBytes())) { + return false; + } + // Memory allocations completed + + final MemoryRange rowLengthsCursor = cumulativeRowLengths.cursor(); + + if (numericArray == null) { + rowLengthsCursor.memory().putInt(rowLengthsCursor.start(), -(lastCumulativeRowLength + rowLength) - 1); + } else { + rowLengthsCursor.memory().putInt(rowLengthsCursor.start(), lastCumulativeRowLength + rowLength); + } + cumulativeRowLengths.advanceCursor(Integer.BYTES); + lastRowLength = rowLength; + lastCumulativeRowLength += rowLength; + + final MemoryRange rowNullityDataCursor = rowLength > 0 ? rowNullityData.cursor() : null; + final MemoryRange rowDataCursor = rowLength > 0 ? rowData.cursor() : null; + + for (int i = 0; i < rowLength; ++i) { + final Number element = numericArray.get(i); + final long memoryOffset = rowDataCursor.start() + ((long) elementSizeBytes() * i); + if (element == null) { + rowNullityDataCursor.memory() + .putByte(rowNullityDataCursor.start() + (long) Byte.BYTES * i, NULL_ELEMENT_MARKER); + putNull(rowDataCursor.memory(), memoryOffset); + } else { + rowNullityDataCursor.memory() + .putByte(rowNullityDataCursor.start() + (long) Byte.BYTES * i, NON_NULL_ELEMENT_MARKER); + putArrayElement(rowDataCursor.memory(), memoryOffset, element); + } + } + + if (rowLength > 0) { + rowNullityData.advanceCursor(Byte.BYTES * rowLength); + rowData.advanceCursor(elementSizeBytes() * rowLength); + } + + return true; + } + + @Override + public void undo() + { + if (lastRowLength == -1) { + throw DruidException.defensive("Nothing written to undo()"); + } + + cumulativeRowLengths.rewindCursor(Integer.BYTES); + rowNullityData.rewindCursor(lastRowLength * Byte.BYTES); + rowData.rewindCursor(lastRowLength * elementSizeBytes()); + + lastCumulativeRowLength -= lastRowLength; + // Multiple undo calls cannot be chained together + lastRowLength = -1; + } + + @Override + public long size() + { + return DATA_OFFSET + cumulativeRowLengths.size() + rowNullityData.size() + rowData.size(); + } + + @Override + public long writeTo(final WritableMemory memory, final long startPosition) + { + long currentPosition = startPosition; + + memory.putByte(currentPosition, typeCode); + ++currentPosition; + + currentPosition += cumulativeRowLengths.writeTo(memory, currentPosition); + currentPosition += rowNullityData.writeTo(memory, currentPosition); + currentPosition += rowData.writeTo(memory, currentPosition); + + return currentPosition - startPosition; + } + + @Override + public void close() + { + cumulativeRowLengths.close(); + rowNullityData.close(); + rowData.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java b/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java index 1a6186bbc8a8..4b0d8abb23f5 100644 --- a/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java +++ b/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java @@ -20,6 +20,7 @@ package org.apache.druid.jackson; import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.JavaType; @@ -35,7 +36,6 @@ import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; - import java.io.IOException; /** @@ -81,6 +81,10 @@ public DefaultObjectMapper(JsonFactory factory, @Nullable String serviceName) configure(SerializationFeature.INDENT_OUTPUT, false); configure(SerializationFeature.FLUSH_AFTER_WRITE_VALUE, false); + // Disable automatic JSON termination, so readers can detect truncated responses when a JsonGenerator is + // closed after an exception is thrown while writing. + configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); + addHandler(new DefaultDeserializationProblemHandler(serviceName)); } diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index 16a1b6675567..61b01ad0646e 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -244,27 +244,18 @@ public void test_long() @Test public void test_arrayLong() { - // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those - // parameters - Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); testWithDataset(FrameWriterTestData.TEST_ARRAYS_LONG); } @Test public void test_arrayFloat() { - // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those - // parameters - Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); testWithDataset(FrameWriterTestData.TEST_ARRAYS_FLOAT); } @Test public void test_arrayDouble() { - // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those - // parameters - Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); testWithDataset(FrameWriterTestData.TEST_ARRAYS_DOUBLE); } diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java index 9d359eed05e0..71c67deadb02 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java @@ -67,7 +67,16 @@ public void test_columnar() final FrameWriterFactory factory = FrameWriters.makeFrameWriterFactory( FrameType.COLUMNAR, new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY), - RowSignature.builder().add("x", ColumnType.LONG).build(), + RowSignature.builder() + .add("a", ColumnType.LONG) + .add("b", ColumnType.FLOAT) + .add("c", ColumnType.DOUBLE) + .add("d", ColumnType.STRING) + .add("e", ColumnType.LONG_ARRAY) + .add("f", ColumnType.FLOAT_ARRAY) + .add("g", ColumnType.DOUBLE_ARRAY) + .add("h", ColumnType.STRING_ARRAY) + .build(), Collections.emptyList() ); @@ -81,7 +90,7 @@ public void test_columnar_unsupportedColumnType() final FrameWriterFactory factory = FrameWriters.makeFrameWriterFactory( FrameType.COLUMNAR, new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY), - RowSignature.builder().add("x", ColumnType.LONG_ARRAY).build(), + RowSignature.builder().add("x", ColumnType.ofArray(ColumnType.LONG_ARRAY)).build(), Collections.emptyList() ); @@ -91,7 +100,7 @@ public void test_columnar_unsupportedColumnType() ); Assert.assertEquals("x", e.getColumnName()); - Assert.assertEquals(ColumnType.LONG_ARRAY, e.getColumnType()); + Assert.assertEquals(ColumnType.ofArray(ColumnType.LONG_ARRAY), e.getColumnType()); } @Test diff --git a/sql/pom.xml b/sql/pom.xml index a098c3d15bfb..d1d4e8afd8b9 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -262,6 +262,11 @@ jdbi test + + org.reflections + reflections + test + diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java index 53b7b7902563..762ee43496ac 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java @@ -38,7 +38,7 @@ public class ParseLongOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(NAME) .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) - .returnTypeCascadeNullable(SqlTypeName.BIGINT) + .returnTypeNullable(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.STRING) .requiredOperandCount(1) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java index b15bb38fdcce..36279bcd8d17 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java @@ -248,20 +248,21 @@ public List programs(final PlannerContext plannerContext) ), Programs.sequence( druidPreProgram, - buildBaseRuleSetProgram(plannerContext), - new LoggingProgram("After baseRuleSet program", isDebug), + buildDecoupledLogicalOptimizationProgram(plannerContext), + new LoggingProgram("After DecoupledLogicalOptimizationProgram program", isDebug), Programs.ofRules(logicalConventionRuleSet(plannerContext)), new LoggingProgram("After logical volcano planner program", isDebug) ) ); } - private Program buildBaseRuleSetProgram(PlannerContext plannerContext) + private Program buildDecoupledLogicalOptimizationProgram(PlannerContext plannerContext) { final HepProgramBuilder builder = HepProgram.builder(); builder.addMatchLimit(CalciteRulesManager.HEP_DEFAULT_MATCH_LIMIT); builder.addGroupBegin(); builder.addRuleCollection(baseRuleSet(plannerContext)); + builder.addRuleInstance(CoreRules.UNION_MERGE); builder.addGroupEnd(); return Programs.of(builder.build(), true, DefaultRelMetadataProvider.INSTANCE); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidQueryGenerator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidQueryGenerator.java deleted file mode 100644 index e9d6d62f72b3..000000000000 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidQueryGenerator.java +++ /dev/null @@ -1,354 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.sql.calcite.planner; - -import com.google.common.collect.ImmutableList; -import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.RelShuttleImpl; -import org.apache.calcite.rel.core.Aggregate; -import org.apache.calcite.rel.core.Filter; -import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.core.Sort; -import org.apache.calcite.rel.core.TableFunctionScan; -import org.apache.calcite.rel.core.TableScan; -import org.apache.calcite.rel.core.Window; -import org.apache.calcite.rel.logical.LogicalAggregate; -import org.apache.calcite.rel.logical.LogicalCorrelate; -import org.apache.calcite.rel.logical.LogicalExchange; -import org.apache.calcite.rel.logical.LogicalFilter; -import org.apache.calcite.rel.logical.LogicalIntersect; -import org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.calcite.rel.logical.LogicalMatch; -import org.apache.calcite.rel.logical.LogicalMinus; -import org.apache.calcite.rel.logical.LogicalProject; -import org.apache.calcite.rel.logical.LogicalSort; -import org.apache.calcite.rel.logical.LogicalUnion; -import org.apache.calcite.rel.logical.LogicalValues; -import org.apache.calcite.rex.RexLiteral; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.query.InlineDataSource; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.sql.calcite.rel.PartialDruidQuery; -import org.apache.druid.sql.calcite.rel.logical.DruidTableScan; -import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule; -import org.apache.druid.sql.calcite.table.DruidTable; -import org.apache.druid.sql.calcite.table.InlineTable; -import org.apache.druid.sql.calcite.table.RowSignatures; - -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; - - -/** - * Converts a DAG of {@link org.apache.druid.sql.calcite.rel.logical.DruidLogicalNode} convention to a native - * Druid query for execution. The convertion is done via a {@link org.apache.calcite.rel.RelShuttle} visitor - * implementation. - */ -public class DruidQueryGenerator extends RelShuttleImpl -{ - private final List queryList = new ArrayList<>(); - private final List queryTables = new ArrayList<>(); - private final PlannerContext plannerContext; - private PartialDruidQuery partialDruidQuery; - private PartialDruidQuery.Stage currentStage = null; - private DruidTable currentTable = null; - private boolean isRoot = true; - - public DruidQueryGenerator(PlannerContext plannerContext) - { - this.plannerContext = plannerContext; - } - - @Override - public RelNode visit(TableScan scan) - { - if (!(scan instanceof DruidTableScan)) { - throw new ISE("Planning hasn't converted logical table scan to druid convention"); - } - DruidTableScan druidTableScan = (DruidTableScan) scan; - isRoot = false; - RelNode result = super.visit(scan); - partialDruidQuery = PartialDruidQuery.create(scan); - currentStage = PartialDruidQuery.Stage.SCAN; - final RelOptTable table = scan.getTable(); - final DruidTable druidTable = table.unwrap(DruidTable.class); - if (druidTable != null) { - currentTable = druidTable; - } - if (druidTableScan.getProject() != null) { - partialDruidQuery = partialDruidQuery.withSelectProject(druidTableScan.getProject()); - currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; - } - return result; - } - - @Override - public RelNode visit(TableFunctionScan scan) - { - return null; - } - - @Override - public RelNode visit(LogicalValues values) - { - isRoot = false; - RelNode result = super.visit(values); - final List> tuples = values.getTuples(); - final List objectTuples = tuples - .stream() - .map(tuple -> tuple - .stream() - .map(v -> DruidLogicalValuesRule.getValueFromLiteral(v, plannerContext)) - .collect(Collectors.toList()) - .toArray(new Object[0]) - ) - .collect(Collectors.toList()); - RowSignature rowSignature = RowSignatures.fromRelDataType( - values.getRowType().getFieldNames(), - values.getRowType() - ); - currentTable = new InlineTable(InlineDataSource.fromIterable(objectTuples, rowSignature)); - if (currentStage == null) { - partialDruidQuery = PartialDruidQuery.create(values); - currentStage = PartialDruidQuery.Stage.SCAN; - } else { - throw new ISE("Values node found at non leaf node in the plan"); - } - return result; - } - - @Override - public RelNode visit(LogicalFilter filter) - { - return visitFilter(filter); - } - - public RelNode visitFilter(Filter filter) - { - isRoot = false; - RelNode result = super.visit(filter); - if (currentStage == PartialDruidQuery.Stage.AGGREGATE) { - partialDruidQuery = partialDruidQuery.withHavingFilter(filter); - currentStage = PartialDruidQuery.Stage.HAVING_FILTER; - } else if (currentStage == PartialDruidQuery.Stage.SCAN) { - partialDruidQuery = partialDruidQuery.withWhereFilter(filter); - currentStage = PartialDruidQuery.Stage.WHERE_FILTER; - } else if (currentStage == PartialDruidQuery.Stage.SELECT_PROJECT) { - PartialDruidQuery old = partialDruidQuery; - partialDruidQuery = PartialDruidQuery.create(old.getScan()); - partialDruidQuery = partialDruidQuery.withWhereFilter(filter); - partialDruidQuery = partialDruidQuery.withSelectProject(old.getSelectProject()); - currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; - } else { - queryList.add(partialDruidQuery); - queryTables.add(currentTable); - partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withWhereFilter(filter); - currentStage = PartialDruidQuery.Stage.WHERE_FILTER; - } - return result; - } - - @Override - public RelNode visit(LogicalProject project) - { - return visitProject(project); - } - - @Override - public RelNode visit(LogicalJoin join) - { - throw new UnsupportedOperationException("Found join"); - } - - @Override - public RelNode visit(LogicalCorrelate correlate) - { - return null; - } - - @Override - public RelNode visit(LogicalUnion union) - { - throw new UnsupportedOperationException("Found union"); - } - - @Override - public RelNode visit(LogicalIntersect intersect) - { - return null; - } - - @Override - public RelNode visit(LogicalMinus minus) - { - return null; - } - - @Override - public RelNode visit(LogicalAggregate aggregate) - { - isRoot = false; - RelNode result = super.visit(aggregate); - if (PartialDruidQuery.Stage.AGGREGATE.canFollow(currentStage)) { - partialDruidQuery = partialDruidQuery.withAggregate(aggregate); - } else { - queryList.add(partialDruidQuery); - queryTables.add(currentTable); - partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withAggregate(aggregate); - } - currentStage = PartialDruidQuery.Stage.AGGREGATE; - return result; - } - - @Override - public RelNode visit(LogicalMatch match) - { - return null; - } - - @Override - public RelNode visit(LogicalSort sort) - { - return visitSort(sort); - } - - @Override - public RelNode visit(LogicalExchange exchange) - { - return null; - } - - private RelNode visitProject(Project project) - { - boolean rootForReal = isRoot; - isRoot = false; - RelNode result = super.visit(project); - if (rootForReal && (currentStage == PartialDruidQuery.Stage.AGGREGATE - || currentStage == PartialDruidQuery.Stage.HAVING_FILTER)) { - partialDruidQuery = partialDruidQuery.withAggregateProject(project); - currentStage = PartialDruidQuery.Stage.AGGREGATE_PROJECT; - } else if (currentStage == PartialDruidQuery.Stage.SCAN || currentStage == PartialDruidQuery.Stage.WHERE_FILTER) { - partialDruidQuery = partialDruidQuery.withSelectProject(project); - currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; - } else if (currentStage == PartialDruidQuery.Stage.SELECT_PROJECT) { - partialDruidQuery = partialDruidQuery.mergeProject(project); - currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; - } else if (currentStage == PartialDruidQuery.Stage.SORT) { - partialDruidQuery = partialDruidQuery.withSortProject(project); - currentStage = PartialDruidQuery.Stage.SORT_PROJECT; - } else { - queryList.add(partialDruidQuery); - queryTables.add(currentTable); - partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withSelectProject(project); - currentStage = PartialDruidQuery.Stage.SELECT_PROJECT; - } - return result; - } - - private RelNode visitSort(Sort sort) - { - isRoot = false; - RelNode result = super.visit(sort); - if (PartialDruidQuery.Stage.SORT.canFollow(currentStage)) { - partialDruidQuery = partialDruidQuery.withSort(sort); - } else { - queryList.add(partialDruidQuery); - queryTables.add(currentTable); - partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withSort(sort); - } - currentStage = PartialDruidQuery.Stage.SORT; - return result; - } - - private RelNode visitAggregate(Aggregate aggregate) - { - isRoot = false; - RelNode result = super.visit(aggregate); - if (PartialDruidQuery.Stage.AGGREGATE.canFollow(currentStage)) { - partialDruidQuery = partialDruidQuery.withAggregate(aggregate); - } else { - queryList.add(partialDruidQuery); - queryTables.add(currentTable); - partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery).withAggregate(aggregate); - } - currentStage = PartialDruidQuery.Stage.AGGREGATE; - return result; - } - - @Override - public RelNode visit(RelNode other) - { - if (other instanceof TableScan) { - return visit((TableScan) other); - } else if (other instanceof Project) { - return visitProject((Project) other); - } else if (other instanceof Sort) { - return visitSort((Sort) other); - } else if (other instanceof Aggregate) { - return visitAggregate((Aggregate) other); - } else if (other instanceof Filter) { - return visitFilter((Filter) other); - } else if (other instanceof LogicalValues) { - return visit((LogicalValues) other); - } else if (other instanceof Window) { - return visitWindow((Window) other); - } - - throw new UOE("Found unsupported RelNode [%s]", other.getClass().getSimpleName()); - } - - private RelNode visitWindow(Window other) - { - RelNode result = super.visit(other); - if (!PartialDruidQuery.Stage.WINDOW.canFollow(currentStage)) { - queryList.add(partialDruidQuery); - queryTables.add(currentTable); - partialDruidQuery = PartialDruidQuery.createOuterQuery(partialDruidQuery); - } - partialDruidQuery = partialDruidQuery.withWindow((Window) result); - currentStage = PartialDruidQuery.Stage.WINDOW; - - return result; - } - - public PartialDruidQuery getPartialDruidQuery() - { - return partialDruidQuery; - } - - public List getQueryList() - { - return queryList; - } - - public List getQueryTables() - { - return queryTables; - } - - public DruidTable getCurrentTable() - { - return currentTable; - } - -} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index 0a570efa32f1..57d4e08b0719 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -61,11 +61,11 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryDataSource; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.sql.calcite.planner.querygen.DruidQueryGenerator; import org.apache.druid.sql.calcite.rel.DruidConvention; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.sql.calcite.rel.DruidRel; @@ -241,7 +241,7 @@ public PlannerResult plan() } // Exceptions during rule evaluations could be wrapped inside a RuntimeException by VolcanoRuleCall class. - // This block will extract a user-friendly message from the exception chain. + // This block will extract a user-friendly message from the exception chain. if (e.getMessage() != null && e.getCause() != null && e.getCause().getMessage() != null @@ -559,36 +559,14 @@ protected PlannerResult planWithDruidConvention() throws ValidationException .plus(DruidLogicalConvention.instance()), newRoot ); - DruidQueryGenerator shuttle = new DruidQueryGenerator(plannerContext); - newRoot.accept(shuttle); - log.info("PartialDruidQuery : " + shuttle.getPartialDruidQuery()); - shuttle.getQueryList().add(shuttle.getPartialDruidQuery()); // add topmost query to the list - shuttle.getQueryTables().add(shuttle.getCurrentTable()); - assert !shuttle.getQueryList().isEmpty(); - log.info("query list size " + shuttle.getQueryList().size()); - log.info("query tables size " + shuttle.getQueryTables().size()); - // build bottom-most query - DruidQuery baseQuery = shuttle.getQueryList().get(0).build( - shuttle.getQueryTables().get(0).getDataSource(), - shuttle.getQueryTables().get(0).getRowSignature(), - plannerContext, - rexBuilder, - shuttle.getQueryList().size() != 1, - null - ); - // build outer queries - for (int i = 1; i < shuttle.getQueryList().size(); i++) { - baseQuery = shuttle.getQueryList().get(i).build( - new QueryDataSource(baseQuery.getQuery()), - baseQuery.getOutputRowSignature(), - plannerContext, - rexBuilder, - false - ); - } + + DruidQueryGenerator generator = new DruidQueryGenerator(plannerContext, newRoot, rexBuilder); + DruidQuery baseQuery = generator.buildQuery(); try { - log.info("final query : " + - new DefaultObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(baseQuery.getQuery())); + log.info( + "final query : " + + new DefaultObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(baseQuery.getQuery()) + ); } catch (JsonProcessingException e) { throw new RuntimeException(e); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java new file mode 100644 index 000000000000..d10c9d3a65b7 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite.planner.querygen; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.Window; +import org.apache.calcite.rex.RexBuilder; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.planner.querygen.DruidQueryGenerator.PDQVertexFactory.PDQVertex; +import org.apache.druid.sql.calcite.planner.querygen.InputDescProducer.InputDesc; +import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.sql.calcite.rel.PartialDruidQuery; +import org.apache.druid.sql.calcite.rel.PartialDruidQuery.Stage; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +/** + * Converts a DAG of {@link org.apache.druid.sql.calcite.rel.logical.DruidLogicalNode} convention to a native + * {@link DruidQuery} for execution. + */ +public class DruidQueryGenerator +{ + private final RelNode relRoot; + private final PDQVertexFactory vertexFactory; + + public DruidQueryGenerator(PlannerContext plannerContext, RelNode relRoot, RexBuilder rexBuilder) + { + this.relRoot = relRoot; + this.vertexFactory = new PDQVertexFactory(plannerContext, rexBuilder); + } + + public DruidQuery buildQuery() + { + Vertex vertex = buildVertexFor(relRoot, true); + return vertex.buildQuery(true); + } + + private Vertex buildVertexFor(RelNode node, boolean isRoot) + { + List newInputs = new ArrayList<>(); + for (RelNode input : node.getInputs()) { + newInputs.add(buildVertexFor(input, false)); + } + Vertex vertex = processNodeWithInputs(node, newInputs, isRoot); + return vertex; + } + + private Vertex processNodeWithInputs(RelNode node, List newInputs, boolean isRoot) + { + if (node instanceof InputDescProducer) { + return vertexFactory.createVertex(PartialDruidQuery.create(node), newInputs); + } + if (newInputs.size() == 1) { + Vertex inputVertex = newInputs.get(0); + Optional newVertex = inputVertex.extendWith(node, isRoot); + if (newVertex.isPresent()) { + return newVertex.get(); + } + inputVertex = vertexFactory.createVertex( + PartialDruidQuery.createOuterQuery(((PDQVertex) inputVertex).partialDruidQuery), + ImmutableList.of(inputVertex) + ); + newVertex = inputVertex.extendWith(node, false); + if (newVertex.isPresent()) { + return newVertex.get(); + } + } + throw DruidException.defensive().build("Unable to process relNode[%s]", node); + } + + /** + * Execution dag vertex - encapsulates a list of operators. + */ + private interface Vertex + { + /** + * Builds the query. + */ + DruidQuery buildQuery(boolean isRoot); + + /** + * Extends the current vertex to include the specified parent. + */ + Optional extendWith(RelNode parentNode, boolean isRoot); + + /** + * Decides wether this {@link Vertex} can be unwrapped into an {@link InputDesc}. + */ + boolean canUnwrapInput(); + + /** + * Unwraps this {@link Vertex} into an {@link InputDesc}. + * + * Unwraps the input of this vertex - if it doesn't do anything beyond reading its input. + * + * @throws DruidException if unwrap is not possible. + */ + InputDesc unwrapInputDesc(); + } + + /** + * {@link PartialDruidQuery} based {@link Vertex} factory. + */ + protected static class PDQVertexFactory + { + private final PlannerContext plannerContext; + private final RexBuilder rexBuilder; + + public PDQVertexFactory(PlannerContext plannerContext, RexBuilder rexBuilder) + { + this.plannerContext = plannerContext; + this.rexBuilder = rexBuilder; + } + + Vertex createVertex(PartialDruidQuery partialDruidQuery, List inputs) + { + return new PDQVertex(partialDruidQuery, inputs); + } + + public class PDQVertex implements Vertex + { + final PartialDruidQuery partialDruidQuery; + final List inputs; + + public PDQVertex(PartialDruidQuery partialDruidQuery, List inputs) + { + this.partialDruidQuery = partialDruidQuery; + this.inputs = inputs; + } + + @Override + public DruidQuery buildQuery(boolean topLevel) + { + InputDesc input = getInput(); + return partialDruidQuery.build( + input.dataSource, + input.rowSignature, + plannerContext, + rexBuilder, + !topLevel + ); + } + + /** + * Creates the {@link InputDesc} for the current {@link Vertex}. + */ + private InputDesc getInput() + { + List inputDescs = new ArrayList<>(); + for (Vertex inputVertex : inputs) { + final InputDesc desc; + if (inputVertex.canUnwrapInput()) { + desc = inputVertex.unwrapInputDesc(); + } else { + DruidQuery inputQuery = inputVertex.buildQuery(false); + desc = new InputDesc(new QueryDataSource(inputQuery.getQuery()), inputQuery.getOutputRowSignature()); + } + inputDescs.add(desc); + } + RelNode scan = partialDruidQuery.getScan(); + if (scan instanceof InputDescProducer) { + InputDescProducer inp = (InputDescProducer) scan; + return inp.getInputDesc(plannerContext, inputDescs); + } + if (inputs.size() == 1) { + return inputDescs.get(0); + } + throw DruidException.defensive("Unable to create InputDesc for Operator [%s]", scan); + } + + /** + * Extends the the current partial query with the new parent if possible. + */ + @Override + public Optional extendWith(RelNode parentNode, boolean isRoot) + { + Optional newPartialQuery = extendPartialDruidQuery(parentNode, isRoot); + if (!newPartialQuery.isPresent()) { + return Optional.empty(); + } + return Optional.of(createVertex(newPartialQuery.get(), inputs)); + } + + /** + * Merges the given {@link RelNode} into the current {@link PartialDruidQuery}. + */ + private Optional extendPartialDruidQuery(RelNode parentNode, boolean isRoot) + { + if (accepts(parentNode, Stage.WHERE_FILTER, Filter.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withWhereFilter((Filter) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.SELECT_PROJECT, Project.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withSelectProject((Project) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.AGGREGATE, Aggregate.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withAggregate((Aggregate) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.AGGREGATE_PROJECT, Project.class) && isRoot) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withAggregateProject((Project) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.HAVING_FILTER, Filter.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withHavingFilter((Filter) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.SORT, Sort.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withSort((Sort) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.SORT_PROJECT, Project.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withSortProject((Project) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.WINDOW, Window.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withWindow((Window) parentNode); + return Optional.of(newPartialQuery); + } + if (accepts(parentNode, Stage.WINDOW_PROJECT, Project.class)) { + PartialDruidQuery newPartialQuery = partialDruidQuery.withWindowProject((Project) parentNode); + return Optional.of(newPartialQuery); + } + return Optional.empty(); + } + + private boolean accepts(RelNode node, Stage whereFilter, Class class1) + { + return partialDruidQuery.canAccept(whereFilter) && class1.isInstance(node); + } + + @Override + public InputDesc unwrapInputDesc() + { + if (canUnwrapInput()) { + DruidQuery q = buildQuery(false); + InputDesc origInput = getInput(); + return new InputDesc(origInput.dataSource, q.getOutputRowSignature()); + } + throw DruidException.defensive("Can't unwrap input of vertex[%s]", partialDruidQuery); + } + + @Override + public boolean canUnwrapInput() + { + if (partialDruidQuery.stage() == Stage.SCAN) { + return true; + } + if (partialDruidQuery.stage() == PartialDruidQuery.Stage.SELECT_PROJECT && + partialDruidQuery.getWhereFilter() == null && + partialDruidQuery.getSelectProject().isMapping()) { + return true; + } + return false; + } + } + + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/InputDescProducer.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/InputDescProducer.java new file mode 100644 index 000000000000..412ac4d1a28e --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/InputDescProducer.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite.planner.querygen; + +import org.apache.druid.query.DataSource; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.planner.PlannerContext; + +import java.util.List; + +/** + * Abstracts away non-trivial input operation handlings between {@link DataSource}s. + * + * Example: TableScan ; Union; Join. + */ +public interface InputDescProducer +{ + /** + * Utility class to input related things details. + * + * Main reason to have this was that {@link DataSource} doesn't contain the {@link RowSignature}. + */ + class InputDesc + { + public DataSource dataSource; + public RowSignature rowSignature; + + public InputDesc(DataSource dataSource, RowSignature rowSignature) + { + this.dataSource = dataSource; + this.rowSignature = rowSignature; + } + } + + InputDesc getInputDesc(PlannerContext plannerContext, List inputs); +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java index d601aa4d2f78..b3bc5ba782aa 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidTableScan.java @@ -19,6 +19,7 @@ package org.apache.druid.sql.calcite.rel.logical; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptCost; @@ -28,37 +29,35 @@ import org.apache.calcite.rel.RelCollationTraitDef; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelWriter; -import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.schema.Table; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.planner.querygen.InputDescProducer; +import org.apache.druid.sql.calcite.table.DruidTable; import java.util.List; /** * {@link DruidLogicalNode} convention node for {@link TableScan} plan node. */ -public class DruidTableScan extends TableScan implements DruidLogicalNode +public class DruidTableScan extends TableScan implements DruidLogicalNode, InputDescProducer { - private final Project project; - public DruidTableScan( RelOptCluster cluster, RelTraitSet traitSet, - RelOptTable table, - Project project + RelOptTable table ) { super(cluster, traitSet, table); - this.project = project; assert getConvention() instanceof DruidLogicalConvention; } @Override public RelNode copy(RelTraitSet traitSet, List inputs) { - return new DruidTableScan(getCluster(), traitSet, table, project); + return new DruidTableScan(getCluster(), traitSet, table); } @Override @@ -76,26 +75,15 @@ public double estimateRowCount(RelMetadataQuery mq) @Override public RelWriter explainTerms(RelWriter pw) { - if (project != null) { - project.explainTerms(pw); - } return super.explainTerms(pw).item("druid", "logical"); } @Override public RelDataType deriveRowType() { - if (project != null) { - return project.getRowType(); - } return super.deriveRowType(); } - public Project getProject() - { - return project; - } - public static DruidTableScan create(RelOptCluster cluster, final RelOptTable relOptTable) { final Table table = relOptTable.unwrap(Table.class); @@ -106,6 +94,21 @@ public static DruidTableScan create(RelOptCluster cluster, final RelOptTable rel } return ImmutableList.of(); }); - return new DruidTableScan(cluster, traitSet, relOptTable, null); + return new DruidTableScan(cluster, traitSet, relOptTable); + } + + @Override + public InputDesc getInputDesc(PlannerContext plannerContext, List inputs) + { + final DruidTable druidTable = getDruidTable(); + return new InputDesc(druidTable.getDataSource(), druidTable.getRowSignature()); + } + + private DruidTable getDruidTable() + { + final RelOptTable table = getTable(); + final DruidTable druidTable = table.unwrap(DruidTable.class); + Preconditions.checkNotNull(druidTable, "DruidTable may not be null"); + return druidTable; } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidUnion.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidUnion.java new file mode 100644 index 000000000000..daab1708cb0a --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidUnion.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite.rel.logical; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.SetOp; +import org.apache.calcite.rel.core.Union; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.UnionDataSource; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.planner.querygen.InputDescProducer; +import java.util.ArrayList; +import java.util.List; + +public class DruidUnion extends Union implements DruidLogicalNode, InputDescProducer +{ + public DruidUnion( + RelOptCluster cluster, + RelTraitSet traits, + List hints, + List inputs, + boolean all) + { + super(cluster, traits, hints, inputs, all); + } + + @Override + public SetOp copy(RelTraitSet traitSet, List inputs, boolean all) + { + return new DruidUnion(getCluster(), traitSet, hints, inputs, all); + } + + @Override + public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) + { + return planner.getCostFactory().makeCost(mq.getRowCount(this), 0, 0); + } + + @Override + public InputDesc getInputDesc(PlannerContext plannerContext, List inputs) + { + List dataSources = new ArrayList<>(); + RowSignature signature = null; + for (InputDesc inputDesc : inputs) { + checkDataSourceSupported(inputDesc.dataSource); + dataSources.add(inputDesc.dataSource); + if (signature == null) { + signature = inputDesc.rowSignature; + } else { + if (!signature.equals(inputDesc.rowSignature)) { + throw DruidException.defensive( + "Row signature mismatch in Union inputs [%s] and [%s]", + signature, + inputDesc.rowSignature + ); + } + } + } + return new InputDesc(new UnionDataSource(dataSources), signature); + } + + private void checkDataSourceSupported(DataSource dataSource) + { + if (dataSource instanceof TableDataSource || dataSource instanceof InlineDataSource) { + return; + } + throw DruidException.defensive("Only Table and Values are supported as inputs for Union [%s]", dataSource); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java index d6a8ca98a22d..fea4e5f610dc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidValues.java @@ -29,22 +29,31 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexLiteral; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.planner.querygen.InputDescProducer; import org.apache.druid.sql.calcite.rel.CostEstimates; +import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule; +import org.apache.druid.sql.calcite.table.InlineTable; +import org.apache.druid.sql.calcite.table.RowSignatures; import java.util.List; +import java.util.stream.Collectors; /** * {@link DruidLogicalNode} convention node for {@link LogicalValues} plan node. */ -public class DruidValues extends LogicalValues implements DruidLogicalNode +public class DruidValues extends LogicalValues implements DruidLogicalNode, InputDescProducer { + private InlineTable inlineTable; + public DruidValues( RelOptCluster cluster, RelTraitSet traitSet, RelDataType rowType, - ImmutableList> tuples - ) + ImmutableList> tuples) { super(cluster, traitSet, rowType, tuples); assert getConvention() instanceof DruidLogicalConvention; @@ -61,4 +70,36 @@ public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) { return planner.getCostFactory().makeCost(CostEstimates.COST_BASE, 0, 0); } + + @Override + public InputDesc getInputDesc(PlannerContext plannerContext, List inputs) + { + if (inlineTable == null) { + inlineTable = buildInlineTable(plannerContext); + } + return new InputDesc(inlineTable.getDataSource(), inlineTable.getRowSignature()); + } + + private InlineTable buildInlineTable(PlannerContext plannerContext) + { + + final List> tuples = getTuples(); + final List objectTuples = tuples + .stream() + .map( + tuple -> tuple + .stream() + .map(v -> DruidLogicalValuesRule.getValueFromLiteral(v, plannerContext)) + .collect(Collectors.toList()) + .toArray(new Object[0]) + ) + .collect(Collectors.toList()); + RowSignature rowSignature = RowSignatures.fromRelDataType( + getRowType().getFieldNames(), + getRowType() + ); + InlineTable inlineTable = new InlineTable(InlineDataSource.fromIterable(objectTuples, rowSignature)); + + return inlineTable; + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidSortUnionRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidSortUnionRule.java index d06c39d72b5b..656310669ba3 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidSortUnionRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidSortUnionRule.java @@ -34,7 +34,7 @@ public class DruidSortUnionRule extends RelOptRule { private static final DruidSortUnionRule INSTANCE = new DruidSortUnionRule(); - + private DruidSortUnionRule() { super(operand(Sort.class, operand(DruidUnionRel.class, any()))); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java index 7bed39fc9ac1..5fe939d3e7c9 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidLogicalRules.java @@ -28,8 +28,8 @@ import org.apache.calcite.rel.logical.LogicalProject; import org.apache.calcite.rel.logical.LogicalSort; import org.apache.calcite.rel.logical.LogicalTableScan; +import org.apache.calcite.rel.logical.LogicalUnion; import org.apache.calcite.rel.logical.LogicalValues; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; @@ -51,8 +51,10 @@ public List rules() return new ArrayList<>( ImmutableList.of( new DruidTableScanRule( - RelOptRule.operand(LogicalTableScan.class, null, RelOptRule.any()), - StringUtils.format("%s", DruidTableScanRule.class.getSimpleName()) + LogicalTableScan.class, + Convention.NONE, + DruidLogicalConvention.instance(), + DruidTableScanRule.class.getSimpleName() ), new DruidAggregateRule( LogicalAggregate.class, @@ -90,6 +92,12 @@ public List rules() Convention.NONE, DruidLogicalConvention.instance(), DruidWindowRule.class.getSimpleName() + ), + new DruidUnionRule( + LogicalUnion.class, + Convention.NONE, + DruidLogicalConvention.instance(), + DruidUnionRule.class.getSimpleName() ) ) ); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java index 517e93f2dc33..d72d4457fe11 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidTableScanRule.java @@ -19,36 +19,38 @@ package org.apache.druid.sql.calcite.rule.logical; -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.plan.RelOptRuleOperand; +import org.apache.calcite.plan.RelTrait; import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; import org.apache.calcite.rel.logical.LogicalTableScan; import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; import org.apache.druid.sql.calcite.rel.logical.DruidTableScan; +import org.checkerframework.checker.nullness.qual.Nullable; /** * {@link ConverterRule} to convert {@link org.apache.calcite.rel.core.TableScan} to {@link DruidTableScan} */ -public class DruidTableScanRule extends RelOptRule +public class DruidTableScanRule extends ConverterRule { - public DruidTableScanRule(RelOptRuleOperand operand, String description) + public DruidTableScanRule(Class clazz, RelTrait in, RelTrait out, String descriptionPrefix) { - super(operand, description); + super( + Config.INSTANCE + .withConversion(clazz, in, out, descriptionPrefix) + ); } @Override - public void onMatch(RelOptRuleCall call) + public @Nullable RelNode convert(RelNode rel) { - LogicalTableScan tableScan = call.rel(0); + LogicalTableScan tableScan = (LogicalTableScan) rel; RelTraitSet newTrait = tableScan.getTraitSet().replace(DruidLogicalConvention.instance()); DruidTableScan druidTableScan = new DruidTableScan( tableScan.getCluster(), newTrait, - tableScan.getTable(), - null + tableScan.getTable() ); - call.transformTo(druidTableScan); + return druidTableScan; } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidUnionRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidUnionRule.java new file mode 100644 index 000000000000..2e281c909905 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidUnionRule.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite.rule.logical; + +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.Union; +import org.apache.druid.sql.calcite.rel.logical.DruidLogicalConvention; +import org.apache.druid.sql.calcite.rel.logical.DruidUnion; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class DruidUnionRule extends ConverterRule +{ + + public DruidUnionRule(Class clazz, RelTrait in, RelTrait out, String descriptionPrefix) + { + super( + Config.INSTANCE + .withConversion(clazz, in, out, descriptionPrefix) + ); + } + + @Override + public @Nullable RelNode convert(RelNode rel) + { + Union w = (Union) rel; + RelTraitSet newTrait = w.getTraitSet().replace(DruidLogicalConvention.instance()); + + return new DruidUnion( + w.getCluster(), + newTrait, + w.getHints(), + convertList( + w.getInputs(), + DruidLogicalConvention.instance() + ), + w.all + ); + } + +} diff --git a/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java b/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java index e70f7ecfdf65..4726e70a0e6b 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java +++ b/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java @@ -43,9 +43,6 @@ public ArrayWriter(final OutputStream outputStream, final ObjectMapper jsonMappe this.serializers = jsonMapper.getSerializerProviderInstance(); this.jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream); this.outputStream = outputStream; - - // Disable automatic JSON termination, so clients can detect truncated responses. - jsonGenerator.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java b/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java index 6545ce80eacb..27a027d0abd6 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java +++ b/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java @@ -46,9 +46,6 @@ public ObjectWriter(final OutputStream outputStream, final ObjectMapper jsonMapp this.serializers = jsonMapper.getSerializerProviderInstance(); this.jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream); this.outputStream = outputStream; - - // Disable automatic JSON termination, so clients can detect truncated responses. - jsonGenerator.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); } @Override diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 2ce2fff0431a..1233e58ea6c9 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -2939,7 +2939,7 @@ public void testTopNWithSelectAndOrderByProjections() ); } - @NotYetSupported(Modes.MISSING_UNION_CONVERSION) + @NotYetSupported(Modes.UNION_WITH_COMPLEX_OPERAND) @Test public void testUnionAllQueries() { @@ -2973,7 +2973,7 @@ public void testUnionAllQueries() ); } - @NotYetSupported(Modes.MISSING_UNION_CONVERSION) + @NotYetSupported(Modes.UNION_WITH_COMPLEX_OPERAND) @Test public void testUnionAllQueriesWithLimit() { @@ -3413,7 +3413,6 @@ public void testNullFloatFilter() * doesn't reset framework once the merge buffers */ @SqlTestFrameworkConfig(numMergeBuffers = 3) - @NotYetSupported(Modes.MISSING_UNION_CONVERSION) @Test public void testUnionAllSameTableThreeTimes() { @@ -3458,7 +3457,6 @@ public void testUnionAllSameTableThreeTimes() } @SqlTestFrameworkConfig(numMergeBuffers = 3) - @NotYetSupported(Modes.MISSING_UNION_CONVERSION) @Test public void testExactCountDistinctUsingSubqueryOnUnionAllTables() { @@ -12645,7 +12643,7 @@ public void testNvlColumns() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EXPR_POSTAGG) + @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN) @Test public void testGroupByWithLiteralInSubqueryGrouping() { @@ -12834,7 +12832,7 @@ public void testQueryContextOuterLimit() ); } - @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.EXPR_POSTAGG) + @DecoupledTestConfig(nativeQueryIgnore = NativeQueryIgnore.IMPROVED_PLAN) @Test public void testRepeatedIdenticalVirtualExpressionGrouping() { @@ -15292,4 +15290,29 @@ public void testWindowingWithOrderBy() ) .run(); } + + @Test + public void testFilterParseLongNullable() + { + testQuery( + "select count(*) from druid.foo where parse_long(dim1, 10) is null", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + expressionVirtualColumn( + "v0", + "parse_long(\"dim1\",10)", + ColumnType.LONG) + ) + .filters(isNull("v0")) + .granularity(Granularities.ALL) + .aggregators(new CountAggregatorFactory("a0")) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of(new Object[]{NullHandling.sqlCompatible() ? 4L : 0L}) + ); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java index 773e1776857d..af0066bba051 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java @@ -28,6 +28,7 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.sql.calcite.NotYetSupported.Modes; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; import org.hamcrest.MatcherAssert; @@ -140,6 +141,7 @@ public void testUnionAllTablesColumnCountMismatch() } } + @NotYetSupported(Modes.UNION_MORE_STRICT_ROWTYPE_CHECK) @Test public void testUnionAllTablesColumnTypeMismatchFloatLong() { @@ -186,6 +188,7 @@ public void testUnionAllTablesColumnTypeMismatchFloatLong() ); } + @NotYetSupported(Modes.ERROR_HANDLING) @Test public void testUnionAllTablesColumnTypeMismatchStringLong() { @@ -203,6 +206,7 @@ public void testUnionAllTablesColumnTypeMismatchStringLong() ); } + @NotYetSupported(Modes.ERROR_HANDLING) @Test public void testUnionAllTablesWhenMappingIsRequired() { @@ -219,6 +223,7 @@ public void testUnionAllTablesWhenMappingIsRequired() ); } + @NotYetSupported(Modes.ERROR_HANDLING) @Test public void testUnionIsUnplannable() { @@ -229,6 +234,7 @@ public void testUnionIsUnplannable() ); } + @NotYetSupported(Modes.ERROR_HANDLING) @Test public void testUnionAllTablesWhenCastAndMappingIsRequired() { @@ -328,6 +334,7 @@ public void testUnionAllSameTableTwiceWithSameMapping() ); } + @NotYetSupported(Modes.ERROR_HANDLING) @Test public void testUnionAllSameTableTwiceWithDifferentMapping() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java index ed410ae89c1e..57514ad0e026 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java @@ -25,6 +25,7 @@ import org.apache.druid.sql.calcite.NotYetSupported.NotYetSupportedProcessor; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier; import org.junit.Rule; public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest @@ -41,14 +42,14 @@ public class DecoupledPlanningCalciteQueryTest extends CalciteQueryTest @Override protected QueryTestBuilder testBuilder() { - + PlannerComponentSupplier componentSupplier = this; CalciteTestConfig testConfig = new CalciteTestConfig(CONTEXT_OVERRIDES) { @Override public SqlTestFramework.PlannerFixture plannerFixture(PlannerConfig plannerConfig, AuthConfig authConfig) { plannerConfig = plannerConfig.withOverrides(CONTEXT_OVERRIDES); - return queryFramework().plannerFixture(DecoupledPlanningCalciteQueryTest.this, plannerConfig, authConfig); + return queryFramework().plannerFixture(componentSupplier, plannerConfig, authConfig); } }; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteUnionQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteUnionQueryTest.java new file mode 100644 index 000000000000..9df34485d258 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteUnionQueryTest.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.sql.calcite.NotYetSupported.NotYetSupportedProcessor; +import org.apache.druid.sql.calcite.planner.PlannerConfig; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier; +import org.junit.Rule; + +public class DecoupledPlanningCalciteUnionQueryTest extends CalciteUnionQueryTest +{ + + @Rule(order = 0) + public NotYetSupportedProcessor decoupledIgnoreProcessor = new NotYetSupportedProcessor(); + + private static final ImmutableMap CONTEXT_OVERRIDES = ImmutableMap.of( + PlannerConfig.CTX_NATIVE_QUERY_SQL_PLANNING_MODE, PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED, + QueryContexts.ENABLE_DEBUG, true + ); + + @Override + protected QueryTestBuilder testBuilder() + { + PlannerComponentSupplier componentSupplier = this; + CalciteTestConfig testConfig = new CalciteTestConfig(CONTEXT_OVERRIDES) + { + @Override + public SqlTestFramework.PlannerFixture plannerFixture(PlannerConfig plannerConfig, AuthConfig authConfig) + { + plannerConfig = plannerConfig.withOverrides(CONTEXT_OVERRIDES); + return queryFramework().plannerFixture(componentSupplier, plannerConfig, authConfig); + } + }; + + QueryTestBuilder builder = new QueryTestBuilder(testConfig) + .cannotVectorize(cannotVectorize) + .skipVectorize(skipVectorize); + + DecoupledTestConfig decTestConfig = queryFrameworkRule.getDescription().getAnnotation(DecoupledTestConfig.class); + + if (decTestConfig != null && decTestConfig.nativeQueryIgnore().isPresent()) { + builder.verifyNativeQueries(x -> false); + } + + return builder; + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java index 1feda3cad7a4..bc1bb9362f2d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java @@ -43,11 +43,6 @@ enum NativeQueryIgnore { NONE, - /** - * Decoupled has moved virtualcolumn to postagg (improved plan) - * caused by: {@link CoreRules#AGGREGATE_ANY_PULL_UP_CONSTANTS} - */ - EXPR_POSTAGG, /** * Aggregate column order changes. * @@ -61,7 +56,8 @@ enum NativeQueryIgnore /** * Improved plan * - * Seen that it was induced by {@link CoreRules#AGGREGATE_ANY_PULL_UP_CONSTANTS} + * Seen that some are induced by {@link CoreRules#AGGREGATE_ANY_PULL_UP_CONSTANTS} + * And in some cases decoupled has moved virtualcolumn to postagg */ IMPROVED_PLAN, /** diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index 684d81b4ee2b..7186576ff978 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -70,16 +70,13 @@ enum Modes { PLAN_MISMATCH(AssertionError.class, "AssertionError: query #"), NOT_ENOUGH_RULES(DruidException.class, "not enough rules"), - CANNOT_CONVERT(DruidException.class, "Cannot convert query parts"), - ERROR_HANDLING(AssertionError.class, "(is was |is was |with message a string containing)"), + ERROR_HANDLING(AssertionError.class, "(is was <(OPERATOR|DEVELOPER)>|is was |with message a string containing)"), EXPRESSION_NOT_GROUPED(DruidException.class, "Expression '[a-z]+' is not being grouped"), COLUMN_NOT_FOUND(DruidException.class, "CalciteContextException.*Column.*not found in any table"), NULLS_FIRST_LAST(DruidException.class, "NULLS (FIRST|LAST)"), BIGINT_TO_DATE(DruidException.class, "BIGINT to type (DATE|TIME)"), - NPE_PLAIN(NullPointerException.class, "java.lang.NullPointerException"), NPE(DruidException.class, "java.lang.NullPointerException"), AGGREGATION_NOT_SUPPORT_TYPE(DruidException.class, "Aggregation \\[(MIN|MAX)\\] does not support type \\[STRING\\]"), - MISSING_DESC(DruidException.class, "function signature DESC"), RESULT_COUNT_MISMATCH(AssertionError.class, "result count:"), ALLDATA_CSV(DruidException.class, "allData.csv"), BIGINT_TIME_COMPARE(DruidException.class, "Cannot apply '.' to arguments of type"), @@ -88,10 +85,9 @@ enum Modes T_ALLTYPES_ISSUES(AssertionError.class, "(t_alltype|allTypsUniq|fewRowsAllData).parquet.*Verifier.verify"), RESULT_MISMATCH(AssertionError.class, "(assertResultsEquals|AssertionError: column content mismatch)"), UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"), - CANNOT_TRANSLATE(DruidException.class, "Cannot translate reference"), - MISSING_UNION_CONVERSION(DruidException.class, "Missing conversions? (is|are) LogicalUnion"), - MISSING_WINDOW_CONVERSION(DruidException.class, "Missing conversions? is Window"), - MISSING_JOIN_CONVERSION(DruidException.class, "Missing conversions? is (Logical)?Join"); + MISSING_JOIN_CONVERSION(DruidException.class, "Missing conversions? is (Logical)?Join"), + UNION_WITH_COMPLEX_OPERAND(DruidException.class, "Only Table and Values are supported as inputs for Union"), + UNION_MORE_STRICT_ROWTYPE_CHECK(DruidException.class, "Row signature mismatch in Union inputs"); public Class throwableClass; public String regex; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupportedUsageTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupportedUsageTest.java new file mode 100644 index 000000000000..874f6531994f --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupportedUsageTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.calcite; + +import org.junit.Test; +import org.reflections.Reflections; +import org.reflections.scanners.MethodAnnotationsScanner; + +import java.lang.reflect.Method; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import static org.junit.Assert.assertEquals; + +public class NotYetSupportedUsageTest +{ + @Test + public void ensureAllModesUsed() + { + Set methodsAnnotatedWith = new Reflections("org.apache.druid.sql", new MethodAnnotationsScanner()) + .getMethodsAnnotatedWith(NotYetSupported.class); + + Set modes = new HashSet<>(Arrays.asList(NotYetSupported.Modes.values())); + for (Method method : methodsAnnotatedWith) { + NotYetSupported annot = method.getAnnotation(NotYetSupported.class); + modes.remove(annot.value()); + } + + assertEquals("There are unused modes which should be removed", Collections.emptySet(), modes); + } +} diff --git a/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-dialog.tsx b/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-dialog.tsx index 4a91337d32e1..59f73c27a110 100644 --- a/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-dialog.tsx +++ b/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-dialog.tsx @@ -46,7 +46,7 @@ const COMPACTION_DYNAMIC_CONFIG_FIELDS: Field[] = [ type: 'number', defaultValue: DEFAULT_MAX, info: <>The maximum number of task slots for compaction tasks, - min: 1, + min: 0, }, ];