emitterList : feedToEmitters.values()) {
+ for (Emitter emitter : emitterList) {
+ emittersSetBuilder.add(emitter);
+ }
+ }
+ this.knownEmitters = emittersSetBuilder.build();
+ }
+
+ /**
+ * Start the emitter. This will start all the emitters the SwitchingEmitter uses.
+ */
+ @Override
+ @LifecycleStart
+ public void start()
+ {
+ log.info("Starting Switching Emitter.");
+
+ for (Emitter e : knownEmitters) {
+ log.info("Starting emitter %s.", e.getClass().getName());
+ e.start();
+ }
+ }
+
+ /**
+ * Emit an event. This method must not throw exceptions or block. The emitters that this uses must also not throw
+ * exceptions or block.
+ *
+ * This emitter will direct events based on feed to a list of emitters specified. If there is no match the event will
+ * use a list of default emitters instead.
+ *
+ * Emitters that this emitter uses that receive too many events and internal queues fill up, should drop events rather
+ * than blocking or consuming excessive memory.
+ *
+ * If an emitter that this emitter uses receives input it considers to be invalid, or has an internal problem, it
+ * should deal with that by logging a warning rather than throwing an exception. Emitters that log warnings
+ * should consider throttling warnings to avoid excessive logs, since a busy Druid cluster can emit a high volume of
+ * events.
+ *
+ * @param event The event that will be emitted.
+ */
+ @Override
+ public void emit(Event event)
+ {
+ // linear search is likely faster than hashed lookup
+ for (Map.Entry> feedToEmitters : feedToEmitters.entrySet()) {
+ if (feedToEmitters.getKey().equals(event.getFeed())) {
+ for (Emitter emitter : feedToEmitters.getValue()) {
+ emitter.emit(event);
+ }
+ return;
+ }
+ }
+ for (Emitter emitter : defaultEmitters) {
+ emitter.emit(event);
+ }
+ }
+
+ /**
+ * Triggers this emitter to tell all emitters that this uses to flush.
+ * @throws IOException
+ */
+ @Override
+ public void flush() throws IOException
+ {
+ boolean fail = false;
+ log.info("Flushing Switching Emitter.");
+
+ for (Emitter e : knownEmitters) {
+ try {
+ log.info("Flushing emitter %s.", e.getClass().getName());
+ e.flush();
+ }
+ catch (IOException ex) {
+ log.error(ex, "Failed to flush emitter [%s]", e.getClass().getName());
+ fail = true;
+ }
+ }
+
+ if (fail) {
+ throw new IOException("failed to flush one or more emitters");
+ }
+ }
+
+ /**
+ * Closes all emitters that the SwitchingEmitter uses
+ * @throws IOException
+ */
+ @Override
+ @LifecycleStop
+ public void close() throws IOException
+ {
+ boolean fail = false;
+ log.info("Closing Switching Emitter.");
+
+ for (Emitter e : knownEmitters) {
+ try {
+ log.info("Closing emitter %s.", e.getClass().getName());
+ e.close();
+ }
+ catch (IOException ex) {
+ log.error(ex, "Failed to close emitter [%s]", e.getClass().getName());
+ fail = true;
+ }
+ }
+
+ if (fail) {
+ throw new IOException("failed to close one or more emitters");
+ }
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/segment/column/TypeStrategies.java b/core/src/main/java/org/apache/druid/segment/column/TypeStrategies.java
index 54a15b1dbdf3..20507c597b0f 100644
--- a/core/src/main/java/org/apache/druid/segment/column/TypeStrategies.java
+++ b/core/src/main/java/org/apache/druid/segment/column/TypeStrategies.java
@@ -236,8 +236,6 @@ public static void checkMaxSize(int available, int maxSizeBytes, TypeSignature
*/
public static final class LongTypeStrategy implements TypeStrategy
{
- private static final Comparator COMPARATOR = Longs::compare;
-
@Override
public int estimateSizeBytes(Long value)
{
@@ -276,9 +274,9 @@ public int write(ByteBuffer buffer, Long value, int maxSizeBytes)
}
@Override
- public int compare(Long o1, Long o2)
+ public int compare(Object o1, Object o2)
{
- return COMPARATOR.compare(o1, o2);
+ return Longs.compare(((Number) o1).longValue(), ((Number) o2).longValue());
}
}
@@ -289,8 +287,6 @@ public int compare(Long o1, Long o2)
*/
public static final class FloatTypeStrategy implements TypeStrategy
{
- private static final Comparator COMPARATOR = Floats::compare;
-
@Override
public int estimateSizeBytes(Float value)
{
@@ -329,9 +325,9 @@ public int write(ByteBuffer buffer, Float value, int maxSizeBytes)
}
@Override
- public int compare(Float o1, Float o2)
+ public int compare(Object o1, Object o2)
{
- return COMPARATOR.compare(o1, o2);
+ return Floats.compare(((Number) o1).floatValue(), ((Number) o2).floatValue());
}
}
@@ -342,7 +338,6 @@ public int compare(Float o1, Float o2)
*/
public static final class DoubleTypeStrategy implements TypeStrategy
{
- private static final Comparator COMPARATOR = Double::compare;
@Override
public int estimateSizeBytes(Double value)
@@ -382,9 +377,9 @@ public int write(ByteBuffer buffer, Double value, int maxSizeBytes)
}
@Override
- public int compare(Double o1, Double o2)
+ public int compare(Object o1, Object o2)
{
- return COMPARATOR.compare(o1, o2);
+ return Double.compare(((Number) o1).doubleValue(), ((Number) o2).doubleValue());
}
}
@@ -437,7 +432,7 @@ public int write(ByteBuffer buffer, String value, int maxSizeBytes)
}
@Override
- public int compare(String s, String s2)
+ public int compare(Object s, Object s2)
{
// copy of lexicographical string comparator in druid processing
// Avoid comparisons for equal references
@@ -447,7 +442,7 @@ public int compare(String s, String s2)
return 0;
}
- return ORDERING.compare(s, s2);
+ return ORDERING.compare((String) s, (String) s2);
}
}
@@ -521,8 +516,11 @@ public int write(ByteBuffer buffer, Object[] value, int maxSizeBytes)
}
@Override
- public int compare(@Nullable Object[] o1, @Nullable Object[] o2)
+ public int compare(@Nullable Object o1Obj, @Nullable Object o2Obj)
{
+ Object[] o1 = (Object[]) o1Obj;
+ Object[] o2 = (Object[]) o2Obj;
+
//noinspection ArrayEquality
if (o1 == o2) {
return 0;
diff --git a/core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java b/core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
index 8a97882d54df..e4856f889714 100644
--- a/core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
+++ b/core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
@@ -55,8 +55,15 @@
* Implementations of this interface should be thread safe, but may not use {@link ByteBuffer} in a thread safe manner,
* potentially modifying positions and limits, either temporarily or permanently depending on which set of methods is
* called.
+ *
+ * This interface extends {@code Comparator} instead of {@code Comparator} because trying to specialize the
+ * type of the comparison method can run into issues for comparators of objects that can sometimes be of a different
+ * java class type. For example, {@code Comparator} cannot accept Integer objects in its comparison method
+ * and there is no easy way for this interface definition to allow {@code TypeStrategy} to actually be a
+ * {@code Comparator}. So, we fall back to effectively erasing the generic type and having them all be
+ * {@code Comparator}.
*/
-public interface TypeStrategy extends Comparator
+public interface TypeStrategy extends Comparator
{
/**
* Estimate the size in bytes that writing this value to memory would require. This method is not required to be
diff --git a/core/src/main/java/org/apache/druid/utils/RuntimeInfo.java b/core/src/main/java/org/apache/druid/utils/RuntimeInfo.java
index c8dadfd44d19..86571eff9456 100644
--- a/core/src/main/java/org/apache/druid/utils/RuntimeInfo.java
+++ b/core/src/main/java/org/apache/druid/utils/RuntimeInfo.java
@@ -19,10 +19,12 @@
package org.apache.druid.utils;
+import org.apache.druid.guice.LazySingleton;
import org.apache.druid.java.util.common.UOE;
import java.lang.reflect.InvocationTargetException;
+@LazySingleton
public class RuntimeInfo
{
public int getAvailableProcessors()
diff --git a/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java
new file mode 100644
index 000000000000..59eac8d5a991
--- /dev/null
+++ b/core/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.java.util.common;
+
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.joda.time.Interval;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class IntervalsTest
+{
+
+ @Test
+ public void testFindOverlappingInterval()
+ {
+ final Interval[] sortedIntervals = new Interval[]{
+ Intervals.of("2019/2020"),
+ Intervals.of("2021/2022"),
+ Intervals.of("2021-04-01/2021-05-01"),
+ Intervals.of("2022/2023")
+ };
+ Arrays.sort(sortedIntervals, Comparators.intervalsByStartThenEnd());
+
+ // Search interval outside the bounds of the sorted intervals
+ Assert.assertNull(
+ Intervals.findOverlappingInterval(Intervals.of("2018/2019"), sortedIntervals)
+ );
+ Assert.assertNull(
+ Intervals.findOverlappingInterval(Intervals.of("2023/2024"), sortedIntervals)
+ );
+
+ // Search interval within bounds, overlap exists
+ // Fully overlapping interval
+ Assert.assertEquals(
+ Intervals.of("2021/2022"),
+ Intervals.findOverlappingInterval(Intervals.of("2021/2022"), sortedIntervals)
+ );
+
+ // Partially overlapping interval
+ Assert.assertEquals(
+ Intervals.of("2022/2023"),
+ Intervals.findOverlappingInterval(Intervals.of("2022-01-01/2022-01-02"), sortedIntervals)
+ );
+
+ Assert.assertEquals(
+ Intervals.of("2021/2022"),
+ Intervals.findOverlappingInterval(Intervals.of("2021-06-01/2021-07-01"), sortedIntervals)
+ );
+
+ // Overlap with multiple intervals, "smallest" one is returned
+ Assert.assertEquals(
+ Intervals.of("2021/2022"),
+ Intervals.findOverlappingInterval(Intervals.of("2021-03-01/2021-04-01"), sortedIntervals)
+ );
+
+ // Search interval within bounds, no overlap
+ Assert.assertNull(
+ Intervals.findOverlappingInterval(Intervals.of("2020-01-02/2020-03-03"), sortedIntervals)
+ );
+ }
+
+}
diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java
index 2b610690db0a..e0b0fbcc510c 100644
--- a/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/ObjectFlattenersTest.java
@@ -23,6 +23,7 @@
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.junit.Assert;
import org.junit.Test;
@@ -32,12 +33,14 @@
public class ObjectFlattenersTest
{
private static final String SOME_JSON = "{\"foo\": null, \"bar\": 1}";
+
+ private static final ObjectFlatteners.FlattenerMaker FLATTENER_MAKER = new JSONFlattenerMaker(true);
private static final ObjectFlattener FLATTENER = ObjectFlatteners.create(
new JSONPathSpec(
true,
ImmutableList.of(new JSONPathFieldSpec(JSONPathFieldType.PATH, "extract", "$.bar"))
),
- new JSONFlattenerMaker(true)
+ FLATTENER_MAKER
);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
@@ -62,4 +65,13 @@ public void testToMap() throws JsonProcessingException
Assert.assertNull(flat.get("foo"));
Assert.assertEquals(1, flat.get("bar"));
}
+
+ @Test
+ public void testToMapNull() throws JsonProcessingException
+ {
+ JsonNode node = OBJECT_MAPPER.readTree("null");
+ Map flat = FLATTENER.toMap(node);
+ Assert.assertNull(FLATTENER_MAKER.toPlainJavaType(node));
+ Assert.assertEquals(ImmutableMap.of(), flat);
+ }
}
diff --git a/core/src/test/java/org/apache/druid/java/util/emitter/core/SwitchingEmitterTest.java b/core/src/test/java/org/apache/druid/java/util/emitter/core/SwitchingEmitterTest.java
new file mode 100644
index 000000000000..5183b47947af
--- /dev/null
+++ b/core/src/test/java/org/apache/druid/java/util/emitter/core/SwitchingEmitterTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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.java.util.emitter.core;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.easymock.EasyMock;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class SwitchingEmitterTest
+{
+
+ private static final String FEED_1 = "feed1";
+ private static final String FEED_2 = "feed2";
+ private static final String FEED_3 = "feed3";
+ private SwitchingEmitter switchingEmitter;
+
+ private Map> emitters;
+ private List defaultEmitters;
+
+ private Emitter feed1Emitter1;
+ private Emitter feed1Emitter2;
+ private Emitter feed2Emitter1;
+ private Emitter feed1AndFeed3Emitter;
+
+ private Set allEmitters;
+
+ @Before
+ public void setup()
+ {
+ this.defaultEmitters = ImmutableList.of(
+ EasyMock.createMock(Emitter.class),
+ EasyMock.createMock(Emitter.class)
+ );
+ this.feed1Emitter1 = EasyMock.createMock(Emitter.class);
+ this.feed1Emitter2 = EasyMock.createMock(Emitter.class);
+ this.feed2Emitter1 = EasyMock.createMock(Emitter.class);
+ this.feed1AndFeed3Emitter = EasyMock.createMock(Emitter.class);
+ this.emitters = ImmutableMap.of(FEED_1, ImmutableList.of(feed1Emitter1, feed1Emitter2, feed1AndFeed3Emitter),
+ FEED_2, ImmutableList.of(feed2Emitter1),
+ FEED_3, ImmutableList.of(feed1AndFeed3Emitter));
+
+ allEmitters = new HashSet<>();
+ allEmitters.addAll(defaultEmitters);
+ for (List feedEmitters : emitters.values()) {
+ allEmitters.addAll(feedEmitters);
+ }
+ this.switchingEmitter = new SwitchingEmitter(emitters, defaultEmitters.toArray(new Emitter[0]));
+ }
+
+ @Test
+ public void testStart()
+ {
+ for (Emitter emitter : allEmitters) {
+ emitter.start();
+ EasyMock.replay(emitter);
+ }
+
+ switchingEmitter.start();
+ }
+
+ @Test
+ public void testEmit()
+ {
+ // test emitting events to all 3 feeds and default emitter
+ Event feed1Event = EasyMock.createMock(Event.class);
+ Event feed2Event = EasyMock.createMock(Event.class);
+ Event feed3Event = EasyMock.createMock(Event.class);
+ Event eventWithNoMatchingFeed = EasyMock.createMock(Event.class);
+
+ EasyMock.expect(feed1Event.getFeed()).andReturn(FEED_1).anyTimes();
+ EasyMock.expect(feed2Event.getFeed()).andReturn(FEED_2).anyTimes();
+ EasyMock.expect(feed3Event.getFeed()).andReturn(FEED_3).anyTimes();
+ EasyMock.expect(eventWithNoMatchingFeed.getFeed()).andReturn("no-real-feed").anyTimes();
+ EasyMock.replay(feed1Event, feed2Event, feed3Event, eventWithNoMatchingFeed);
+
+ for (Emitter emitter : defaultEmitters) {
+ emitter.emit(eventWithNoMatchingFeed);
+ }
+ for (Emitter emitter : emitters.get("feed1")) {
+ emitter.emit(feed1Event);
+ }
+ for (Emitter emitter : emitters.get("feed2")) {
+ emitter.emit(feed2Event);
+ }
+ for (Emitter emitter : emitters.get("feed3")) {
+ emitter.emit(feed3Event);
+ }
+ for (Emitter emitter : allEmitters) {
+ EasyMock.replay(emitter);
+ }
+
+ switchingEmitter.emit(feed1Event);
+ switchingEmitter.emit(feed2Event);
+ switchingEmitter.emit(feed3Event);
+ switchingEmitter.emit(eventWithNoMatchingFeed);
+ }
+
+ @Test
+ public void testFlush() throws IOException
+ {
+ for (Emitter emitter : allEmitters) {
+ emitter.flush();
+ EasyMock.replay(emitter);
+ }
+
+ switchingEmitter.flush();
+ }
+
+ @Test
+ public void testClose() throws IOException
+ {
+ for (Emitter emitter : allEmitters) {
+ emitter.close();
+ EasyMock.replay(emitter);
+ }
+
+ switchingEmitter.close();
+ }
+
+ @After
+ public void tearDown()
+ {
+ for (Emitter emitter : allEmitters) {
+ EasyMock.verify(emitter);
+ }
+ }
+}
diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/MetricsVerifier.java b/core/src/test/java/org/apache/druid/java/util/metrics/MetricsVerifier.java
new file mode 100644
index 000000000000..c0f2eaeaf15f
--- /dev/null
+++ b/core/src/test/java/org/apache/druid/java/util/metrics/MetricsVerifier.java
@@ -0,0 +1,98 @@
+/*
+ * 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.java.util.metrics;
+
+import org.junit.Assert;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Test utility to extract and verify metric values.
+ */
+public interface MetricsVerifier
+{
+ /**
+ * Verifies that no event has been emitted for the given metric.
+ */
+ default void verifyNotEmitted(String metricName)
+ {
+ verifyEmitted(metricName, 0);
+ }
+
+ /**
+ * Verifies that the metric was emitted the expected number of times.
+ */
+ default void verifyEmitted(String metricName, int times)
+ {
+ verifyEmitted(metricName, null, times);
+ }
+
+ /**
+ * Verifies that the metric was emitted for the given dimension filters the
+ * expected number of times.
+ */
+ default void verifyEmitted(String metricName, Map dimensionFilters, int times)
+ {
+ Assert.assertEquals(
+ "Metric was emitted unexpected number of times.",
+ times,
+ getMetricValues(metricName, dimensionFilters).size()
+ );
+ }
+
+ /**
+ * Verifies the value of the specified metric emitted in the previous run.
+ */
+ default void verifyValue(String metricName, Number expectedValue)
+ {
+ verifyValue(metricName, null, expectedValue);
+ }
+
+ /**
+ * Verifies the value of the event corresponding to the specified metric and
+ * dimensionFilters emitted in the previous run.
+ */
+ default void verifyValue(String metricName, Map dimensionFilters, Number expectedValue)
+ {
+ Assert.assertEquals(expectedValue, getValue(metricName, dimensionFilters));
+ }
+
+ /**
+ * Gets the value of the event corresponding to the specified metric and
+ * dimensionFilters.
+ */
+ default Number getValue(String metricName, Map dimensionFilters)
+ {
+ List values = getMetricValues(metricName, dimensionFilters);
+ Assert.assertEquals(
+ "Metric must have been emitted exactly once for the given dimensions.",
+ 1,
+ values.size()
+ );
+ return values.get(0);
+ }
+
+ /**
+ * Gets the metric values for the specified dimension filters.
+ */
+ List getMetricValues(String metricName, Map dimensionFilters);
+
+}
diff --git a/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java b/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java
index 653dc8a08aae..395245815792 100644
--- a/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java
+++ b/core/src/test/java/org/apache/druid/java/util/metrics/StubServiceEmitter.java
@@ -24,12 +24,15 @@
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
-public class StubServiceEmitter extends ServiceEmitter
+public class StubServiceEmitter extends ServiceEmitter implements MetricsVerifier
{
private final List events = new ArrayList<>();
- private final List metricEvents = new ArrayList<>();
+ private final Map> metricEvents = new HashMap<>();
public StubServiceEmitter(String service, String host)
{
@@ -40,7 +43,9 @@ public StubServiceEmitter(String service, String host)
public void emit(Event event)
{
if (event instanceof ServiceMetricEvent) {
- metricEvents.add((ServiceMetricEvent) event);
+ ServiceMetricEvent metricEvent = (ServiceMetricEvent) event;
+ metricEvents.computeIfAbsent(metricEvent.getMetric(), name -> new ArrayList<>())
+ .add(metricEvent);
}
events.add(event);
}
@@ -53,12 +58,29 @@ public List getEvents()
return events;
}
- /**
- * Gets all the metric events emitted since the previous {@link #flush()}.
- */
- public List getMetricEvents()
+ @Override
+ public List getMetricValues(
+ String metricName,
+ Map dimensionFilters
+ )
{
- return metricEvents;
+ final List values = new ArrayList<>();
+ final List events =
+ metricEvents.getOrDefault(metricName, Collections.emptyList());
+ final Map filters =
+ dimensionFilters == null ? Collections.emptyMap() : dimensionFilters;
+ for (ServiceMetricEvent event : events) {
+ final Map userDims = event.getUserDims();
+ boolean match = filters.keySet().stream()
+ .map(d -> filters.get(d).equals(userDims.get(d)))
+ .reduce((a, b) -> a && b)
+ .orElse(true);
+ if (match) {
+ values.add(event.getValue());
+ }
+ }
+
+ return values;
}
@Override
diff --git a/core/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java b/core/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java
index fa6d86d21f50..66f4adcdd02d 100644
--- a/core/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java
+++ b/core/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java
@@ -19,6 +19,7 @@
package org.apache.druid.segment.column;
+import com.google.common.collect.Ordering;
import com.google.common.primitives.Longs;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Pair;
@@ -103,7 +104,7 @@ public int write(ByteBuffer buffer, String value, int maxSizeBytes)
}
@Override
- public int compare(String o1, String o2)
+ public int compare(Object o1, Object o2)
{
return 0;
}
@@ -639,10 +640,13 @@ public int compareTo(NullableLongPair o)
public static class NullableLongPairTypeStrategy implements TypeStrategy
{
+
+ private Ordering ordering = Comparators.naturalNullsFirst();
+
@Override
- public int compare(NullableLongPair o1, NullableLongPair o2)
+ public int compare(Object o1, Object o2)
{
- return Comparators.naturalNullsFirst().compare(o1, o2);
+ return ordering.compare((NullableLongPair) o1, (NullableLongPair) o2);
}
@Override
diff --git a/distribution/docker/docker-compose.yml b/distribution/docker/docker-compose.yml
index 58b7a47a4b4a..ea9adcea0d70 100644
--- a/distribution/docker/docker-compose.yml
+++ b/distribution/docker/docker-compose.yml
@@ -49,7 +49,7 @@ services:
- ZOO_MY_ID=1
coordinator:
- image: apache/druid:0.24.0
+ image: apache/druid:24.0.1
container_name: coordinator
volumes:
- druid_shared:/opt/shared
diff --git a/distribution/pom.xml b/distribution/pom.xml
index e2b7773b09d4..7b92e3436b45 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -30,7 +30,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index 6fb3201d98f4..a14b3beb56c7 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -418,6 +418,7 @@ There are several emitters available:
- [`parametrized`](#parametrized-http-emitter-module) operates like the `http` emitter but fine-tunes the recipient URL based on the event feed.
- [`composing`](#composing-emitter-module) initializes multiple emitter modules.
- [`graphite`](#graphite-emitter) emits metrics to a [Graphite](https://graphiteapp.org/) Carbon service.
+- [`switching`](#switching-emitter) initializes and emits to multiple emitter modules based on the event feed.
##### Logging Emitter Module
@@ -483,6 +484,14 @@ Instead use `recipientBaseUrlPattern` described in the table below.
To use graphite as emitter set `druid.emitter=graphite`. For configuration details, see [Graphite emitter](../development/extensions-contrib/graphite.md) for the Graphite emitter Druid extension.
+##### Switching Emitter
+
+To use switching as emitter set `druid.emitter=switching`.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.emitter.switching.emitters`|JSON map of feed to list of emitter modules that will be used for the mapped feed, e.g., {"metrics":["http"], "alerts":["logging"]}|{}|
+|`druid.emitter.switching.defaultEmitters`|JSON list of emitter modules to load that will be used if there is no emitter specifically designated for that event's feed, e.g., ["logging","http"].|[]|
### Metadata storage
@@ -939,7 +948,7 @@ Issuing a GET request at the same URL will return the spec that is currently in
|`mergeBytesLimit`|The maximum total uncompressed size in bytes of segments to merge.|524288000L|
|`mergeSegmentsLimit`|The maximum number of segments that can be in a single [append task](../ingestion/tasks.md).|100|
|`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5|
-|`useBatchedSegmentSampler`|Boolean flag for whether or not we should use the Reservoir Sampling with a reservoir of size k instead of fixed size 1 to pick segments to move. This option can be enabled to speed up segment balancing process, especially if there are huge number of segments in the cluster or if there are too many segments to move.|false|
+|`useBatchedSegmentSampler`|Deprecated. Boolean flag for whether or not we should use the Reservoir Sampling with a reservoir of size k instead of fixed size 1 to pick segments to move. This option can be enabled to speed up the sampling of segments to be balanced, especially if there is a large number of segments in the cluster or if there are too many segments to move.|true|
|`percentOfSegmentsToConsiderPerMove`|Deprecated. This will eventually be phased out by the batched segment sampler. You can enable the batched segment sampler now by setting the dynamic Coordinator config, `useBatchedSegmentSampler`, to `true`. Note that if you choose to enable the batched segment sampler, `percentOfSegmentsToConsiderPerMove` will no longer have any effect on balancing. If `useBatchedSegmentSampler == false`, this config defines the percentage of the total number of segments in the cluster that are considered every time a segment needs to be selected for a move. Druid orders servers by available capacity ascending (the least available capacity first) and then iterates over the servers. For each server, Druid iterates over the segments on the server, considering them for moving. The default config of 100% means that every segment on every server is a candidate to be moved. This should make sense for most small to medium-sized clusters. However, an admin may find it preferable to drop this value lower if they don't think that it is worthwhile to consider every single segment in the cluster each time it is looking for a segment to move.|100|
|`replicantLifetime`|The maximum number of Coordinator runs for a segment to be replicated before we start alerting.|15|
|`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10|
@@ -948,6 +957,7 @@ Issuing a GET request at the same URL will return the spec that is currently in
|`killDataSourceWhitelist`|List of specific data sources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated data source names or a JSON array.|none|
|`killPendingSegmentsSkipList`|List of data sources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated data sources or a JSON array.|none|
|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 100. |100|
+|`useRoundRobinSegmentAssignment`|Boolean flag for whether segments should be assigned to historicals in a round robin fashion. When disabled, segment assignment is done using the chosen balancer strategy. When enabled, this can speed up segment assignments leaving balancing to move the segments to their optimal locations (based on the balancer strategy) lazily. |false|
|`decommissioningNodes`| List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `decommissioningMaxPercentOfMaxSegmentsToMove`.|none|
|`decommissioningMaxPercentOfMaxSegmentsToMove`| Upper limit of segments the Coordinator can move from decommissioning servers to active non-decommissioning servers during a single run. This value is relative to the total maximum number of segments that can be moved at any given time based upon the value of `maxSegmentsToMove`. If `decommissioningMaxPercentOfMaxSegmentsToMove` is 0, the Coordinator does not move segments to decommissioning servers, effectively putting them in a type of "maintenance" mode. In this case, decommissioning servers do not participate in balancing or assignment by load rules. The Coordinator still considers segments on decommissioning servers as candidates to replicate on active servers. Decommissioning can stall if there are no available active servers to move the segments to. You can use the maximum percent of decommissioning segment movements to prioritize balancing or to decrease commissioning time to prevent active servers from being overloaded. The value must be between 0 and 100.|70|
|`pauseCoordination`| Boolean flag for whether or not the coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` Interface. Such duties include: Segment balancing, Segment compaction, Emission of metrics controlled by the dynamic coordinator config `emitBalancingStats`, Submitting kill tasks for unused segments (if enabled), Logging of used segments in the cluster, Marking of newly unused or overshadowed segments, Matching and execution of load/drop rules for used segments, Unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS Name Nodes with downtime and don't want the coordinator to be directing Historical Nodes to hit the Name Node with API requests until maintenance is done and the deep store is declared healthy for use again. |false|
@@ -1102,6 +1112,8 @@ These Overlord static configurations can be defined in the `overlord/runtime.pro
|`druid.indexer.storage.type`|Choices are "local" or "metadata". Indicates whether incoming tasks should be stored locally (in heap) or in metadata storage. "local" is mainly for internal testing while "metadata" is recommended in production because storing incoming tasks in metadata storage allows for tasks to be resumed if the Overlord should fail.|local|
|`druid.indexer.storage.recentlyFinishedThreshold`|Duration of time to store task results. Default is 24 hours. If you have hundreds of tasks running in a day, consider increasing this threshold.|PT24H|
|`druid.indexer.tasklock.forceTimeChunkLock`|_**Setting this to false is still experimental**_ If set, all tasks are enforced to use time chunk lock. If not set, each task automatically chooses a lock type to use. This configuration can be overwritten by setting `forceTimeChunkLock` in the [task context](../ingestion/tasks.md#context). See [Task Locking & Priority](../ingestion/tasks.md#context) for more details about locking in tasks.|true|
+|`druid.indexer.tasklock.batchSegmentAllocation`| If set to true, Druid performs segment allocate actions in batches to improve throughput and reduce the average `task/action/run/time`. See [batching `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions) for details.|false|
+|`druid.indexer.tasklock.batchAllocationWaitTime`|Number of milliseconds after Druid adds the first segment allocate action to a batch, until it executes the batch. Allows the batch to add more requests and improve the average segment allocation run time. This configuration takes effect only if `batchSegmentAllocation` is enabled.|500|
|`druid.indexer.task.default.context`|Default task context that is applied to all tasks submitted to the Overlord. Any default in this config does not override neither the context values the user provides nor `druid.indexer.tasklock.forceTimeChunkLock`.|empty context|
|`druid.indexer.queue.maxSize`|Maximum number of active tasks at one time.|Integer.MAX_VALUE|
|`druid.indexer.queue.startDelay`|Sleep this long before starting Overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M|
@@ -1375,7 +1387,7 @@ For GCE's properties, please refer to the [gce-extensions](../development/extens
This section contains the configuration options for the processes that reside on Data servers (MiddleManagers/Peons and Historicals) in the suggested [three-server configuration](../design/processes.md#server-types).
-Configuration options for the experimental [Indexer process](../design/indexer.md) are also provided here.
+Configuration options for the [Indexer process](../design/indexer.md) are also provided here.
### MiddleManager and Peons
@@ -2188,7 +2200,7 @@ Supported query contexts:
|Key|Description|Default|
|---|-----------|-------|
|`druid.expressions.useStrictBooleans`|Controls the behavior of Druid boolean operators and functions, if set to `true` all boolean values will be either a `1` or `0`. See [expression documentation](../misc/math-expr.md#logical-operator-modes)|false|
-|`druid.expressions.allowNestedArrays`|If enabled, Druid array expressions can create nested arrays. This is experimental and should be used with caution.|false|
+|`druid.expressions.allowNestedArrays`|If enabled, Druid array expressions can create nested arrays.|false|
### Router
#### Router Process Configs
diff --git a/docs/development/experimental-features.md b/docs/development/experimental-features.md
new file mode 100644
index 000000000000..30d8c2f77c2b
--- /dev/null
+++ b/docs/development/experimental-features.md
@@ -0,0 +1,60 @@
+---
+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](../multi-stage-query/api.md)
+
+## Indexer process
+
+- [Indexer process](../design/indexer.md)
+- [Processes and servers](../design/processes.md#indexer-process-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/architecture.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
+ - Expression processing configuration `druid.expressions.allowNestedArrays`
diff --git a/docs/development/extensions-core/datasketches-tuple.md b/docs/development/extensions-core/datasketches-tuple.md
index fc4f74d5c81d..c9a05b5ab197 100644
--- a/docs/development/extensions-core/datasketches-tuple.md
+++ b/docs/development/extensions-core/datasketches-tuple.md
@@ -39,19 +39,52 @@ druid.extensions.loadList=["druid-datasketches"]
"name" : ,
"fieldName" : ,
"nominalEntries": ,
- "numberOfValues" : ,
- "metricColumns" :
+ "metricColumns" : ,
+ "numberOfValues" :
}
```
|property|description|required?|
|--------|-----------|---------|
|type|This String should always be "arrayOfDoublesSketch"|yes|
-|name|A String for the output (result) name of the calculation.|yes|
+|name|String representing the output column to store sketch values.|yes|
|fieldName|A String for the name of the input field.|yes|
|nominalEntries|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2. See the [Theta sketch accuracy](https://datasketches.apache.org/docs/Theta/ThetaErrorTable) for details. |no, defaults to 16384|
-|numberOfValues|Number of values associated with each distinct key. |no, defaults to 1|
-|metricColumns|If building sketches from raw data, an array of names of the input columns containing numeric values to be associated with each distinct key.|no, defaults to empty array|
+|metricColumns|When building sketches from raw data, an array input column that contain numeric values to associate with each distinct key. If not provided, assumes `fieldName` is an `arrayOfDoublesSketch`|no, if not provided `fieldName` is assumed to be an arrayOfDoublesSketch|
+|numberOfValues|Number of values associated with each distinct key. |no, defaults to the length of `metricColumns` if provided and 1 otherwise|
+
+You can use the `arrayOfDoublesSketch` aggregator to:
+
+- Build a sketch from raw data. In this case, set `metricColumns` to an array.
+- Build a sketch from an existing `ArrayOfDoubles` sketch . In this case, leave `metricColumns` unset and set the `fieldName` to an `ArrayOfDoubles` sketch with `numberOfValues` doubles. At ingestion time, you must base64 encode `ArrayOfDoubles` sketches at ingestion time.
+
+#### Example on top of raw data
+
+Compute a theta of unique users. For each user store the `added` and `deleted` scores. The new sketch column will be called `users_theta`.
+
+```json
+{
+ "type": "arrayOfDoublesSketch",
+ "name": "users_theta",
+ "fieldName": "user",
+ "nominalEntries": 16384,
+ "metricColumns": ["added", "deleted"],
+}
+```
+
+#### Example ingesting a precomputed sketch column
+
+Ingest a sketch column called `user_sketches` that has a base64 encoded value of two doubles in its array and store it in a column called `users_theta`.
+
+```json
+{
+ "type": "arrayOfDoublesSketch",
+ "name": "users_theta",
+ "fieldName": "user_sketches",
+ "nominalEntries": 16384,
+ "numberOfValues": 2,
+}
+```
### Post Aggregators
diff --git a/docs/development/extensions-core/druid-basic-security.md b/docs/development/extensions-core/druid-basic-security.md
index b6698b4f153d..4e042b4405f3 100644
--- a/docs/development/extensions-core/druid-basic-security.md
+++ b/docs/development/extensions-core/druid-basic-security.md
@@ -53,12 +53,29 @@ To set the value for the configuration properties, add them to the common runtim
### General properties
-|Property|Description|Default|required|
-|--------|-----------|-------|--------|
-|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often processes should poll the Coordinator for the current Druid metadata store authenticator/authorizer state.|60000|No|
-|`druid.auth.basic.common.maxRandomDelay`|Defines in milliseconds the amount of random delay to add to the pollingPeriod, to spread polling requests across time.|6000|No|
-|`druid.auth.basic.common.maxSyncRetries`|Determines how many times a service will retry if the authentication/authorization Druid metadata store state sync with the Coordinator fails.|10|No|
-|`druid.auth.basic.common.cacheDirectory`|If defined, snapshots of the basic Authenticator and Authorizer Druid metadata store caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the Coordinator.|null|No|
+**`druid.auth.basic.common.pollingPeriod`**
+
+Defines in milliseconds how often processes should poll the Coordinator for the current Druid metadata store authenticator/authorizer state.
+ **Required**: No
+ **Default**: 60000
+
+**`druid.auth.basic.common.maxRandomDelay`**
+
+Defines in milliseconds the amount of random delay to add to the pollingPeriod, to spread polling requests across time.
+ **Required**: No
+ **Default**: 6000
+
+**`druid.auth.basic.common.maxSyncRetries`**
+
+Determines how many times a service will retry if the authentication/authorization Druid metadata store state sync with the Coordinator fails.
+ **Required**: No
+ **Default**: 10
+
+**`druid.auth.basic.common.cacheDirectory`**
+
+If defined, snapshots of the basic Authenticator and Authorizer Druid metadata store caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the Coordinator.
+ **Required**: No
+ **Default**: null
### Authenticator
@@ -96,16 +113,55 @@ The remaining examples of authenticator configuration use either `MyBasicMetadat
#### Properties for Druid metadata store user authentication
-|Property|Description|Default|required|
-|--------|-----------|-------|--------|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialIterations`|Number of iterations to use for password hashing. See [Credential iterations and API performance](#credential-iterations-and-api-performance)|10000|No|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialsValidator.type`|The type of credentials store (metadata) to validate requests credentials.|metadata|No|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.skipOnFailure`|If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.|false|No|
-|`druid.auth.authenticator.MyBasicMetadataAuthenticator.authorizerName`|Authorizer that requests should be directed to|N/A|Yes|
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialAdminPassword`**
+
+Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.
+ **Required**: No
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.initialInternalClientPassword`**
+
+Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.
+ **Required**: No
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.enableCacheNotifications`**
+
+If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.
+ **Required**: No
+ **Default**: True
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.cacheNotificationTimeout`**
+
+The timeout in milliseconds for the cache notifications.
+ **Required**: No
+ **Default**: 5000
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialIterations`**
+
+Number of iterations to use for password hashing. See [Credential iterations and API performance](#credential-iterations-and-api-performance)
+ **Required**: No
+ **Default**: 10000
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.credentialsValidator.type`**
+
+The type of credentials store (metadata) to validate requests credentials.
+ **Required**: No
+ **Default**: metadata
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.skipOnFailure`**
+
+If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.
+ **Required**: No
+ **Default**: false
+
+**`druid.auth.authenticator.MyBasicMetadataAuthenticator.authorizerName`**
+
+Authorizer that requests should be directed to.
+ **Required**: Yes
+ **Default**: N/A
+
##### Credential iterations and API performance
@@ -121,25 +177,107 @@ If Druid uses the default credentials validator (i.e., `credentialsValidator.typ
#### Properties for LDAP user authentication
-|Property|Description|Default|required|
-|--------|-----------|-------|--------|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialIterations`|Number of iterations to use for password hashing.|10000|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.type`|The type of credentials store (ldap) to validate requests credentials.|metadata|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.url`|URL of the LDAP server.|null|Yes|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindUser`|LDAP bind user username.|null|Yes|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindPassword`|[Password Provider](../../operations/password-provider.md) LDAP bind user password.|null|Yes|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.baseDn`|The point from where the LDAP server will search for users.|null|Yes|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userSearch`|The filter/expression to use for the search. For example, (&(sAMAccountName=%s)(objectClass=user))|null|Yes|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userAttribute`|The attribute id identifying the attribute that will be returned as part of the search. For example, sAMAccountName. |null|Yes|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialVerifyDuration`|The duration in seconds for how long valid credentials are verifiable within the cache when not requested.|600|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialMaxDuration`|The max duration in seconds for valid credentials that can reside in cache regardless of how often they are requested.|3600|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialCacheSize`|The valid credentials cache size. The cache uses a LRU policy.|100|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.skipOnFailure`|If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.|false|No|
-|`druid.auth.authenticator.MyBasicLDAPAuthenticator.authorizerName`|Authorizer that requests should be directed to.|N/A|Yes|
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialAdminPassword`**
+
+Initial [Password Provider](../../operations/password-provider.md) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.
+ **Required**: No
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialInternalClientPassword`**
+
+Initial [Password Provider](../../operations/password-provider.md) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.
+ **Required**: No
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.enableCacheNotifications`**
+
+If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.
+ **Required**: No
+ **Default**: true
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.cacheNotificationTimeout`**
+
+The timeout in milliseconds for the cache notifications.
+ **Required**: No
+ **Default**: 5000
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialIterations`**
+
+Number of iterations to use for password hashing.
+ **Required**: No
+ **Default**: 10000
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.type`**
+
+The type of credentials store (ldap) to validate requests credentials.
+ **Required**: No
+ **Default**: metadata
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.url`**
+
+URL of the LDAP server.
+ **Required**: Yes
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindUser`**
+
+LDAP bind user username.
+ **Required**: Yes
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindPassword`**
+
+[Password Provider](../../operations/password-provider.md) LDAP bind user password.
+ **Required**: Yes
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.baseDn`**
+
+The point from where the LDAP server will search for users.
+ **Required**: Yes
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userSearch`**
+
+The filter/expression to use for the search. For example, (&(sAMAccountName=%s)(objectClass=user))
+ **Required**: Yes
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userAttribute`**
+
+The attribute id identifying the attribute that will be returned as part of the search. For example, sAMAccountName.
+ **Required**: Yes
+ **Default**: null
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialVerifyDuration`**
+
+The duration in seconds for how long valid credentials are verifiable within the cache when not requested.
+ **Required**: No
+ **Default**: 600
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialMaxDuration`**
+
+The max duration in seconds for valid credentials that can reside in cache regardless of how often they are requested.
+ **Required**: No
+ **Default**: 3600
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialCacheSize`**
+
+The valid credentials cache size. The cache uses a LRU policy.
+ **Required**: No
+ **Default**: 100
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.skipOnFailure`**
+
+If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.
+ **Required**: No
+ **Default**: false
+
+**`druid.auth.authenticator.MyBasicLDAPAuthenticator.authorizerName`**
+
+Authorizer that requests should be directed to.
+ **Required**: Yes
+ **Default**: N/A
### Escalator
@@ -155,11 +293,24 @@ druid.escalator.authorizerName=MyBasicMetadataAuthorizer
```
#### Properties
-|Property|Description|Default|required|
-|--------|-----------|-------|--------|
-|`druid.escalator.internalClientUsername`|The escalator will use this username for requests made as the internal system user.|n/a|Yes|
-|`druid.escalator.internalClientPassword`|The escalator will use this [Password Provider](../../operations/password-provider.md) for requests made as the internal system user.|n/a|Yes|
-|`druid.escalator.authorizerName`|Authorizer that requests should be directed to.|n/a|Yes|
+
+**`druid.escalator.internalClientUsername`**
+
+The escalator will use this username for requests made as the internal system user.
+ **Required**: Yes
+ **Default**: N/A
+
+**`druid.escalator.internalClientPassword`**
+
+The escalator will use this [Password Provider](../../operations/password-provider.md) for requests made as the internal system user.
+ **Required**: Yes
+ **Default**: N/A
+
+**`druid.escalator.authorizerName`**
+
+Authorizer that requests should be directed to.
+ **Required**: Yes
+ **Default**: N/A
### Authorizer
@@ -182,24 +333,131 @@ druid.auth.authorizer.MyBasicMetadataAuthorizer.type=basic
The examples in the rest of this article use `MyBasicMetadataAuthorizer` or `MyBasicLDAPAuthorizer` as the authorizer name.
#### Properties for Druid metadata store user authorization
-|Property|Description|Default|required|
-|--------|-----------|-------|--------|
-|`druid.auth.authorizer.MyBasicMetadataAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No|
-|`druid.auth.authorizer.MyBasicMetadataAuthorizer.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
-|`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminUser`|The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.|admin|No|
-|`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminRole`|The initial admin role to create if it doesn't already exists.|admin|No|
-|`druid.auth.authorizer.MyBasicMetadataAuthorizer.roleProvider.type`|The type of role provider to authorize requests credentials.|metadata|No
+
+**`druid.auth.authorizer.MyBasicMetadataAuthorizer.enableCacheNotifications`**
+
+If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.
+ **Required**: No
+ **Default**: true
+
+**`druid.auth.authorizer.MyBasicMetadataAuthorizer.cacheNotificationTimeout`**
+
+The timeout in milliseconds for the cache notifications.
+ **Required**: No
+ **Default**: 5000
+
+**`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminUser`**
+
+The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.
+ **Required**: No
+ **Default**: admin
+
+**`druid.auth.authorizer.MyBasicMetadataAuthorizer.initialAdminRole`**
+
+The initial admin role to create if it doesn't already exists.
+ **Required**: No
+ **Default**: admin
+
+**`druid.auth.authorizer.MyBasicMetadataAuthorizer.roleProvider.type`**
+
+The type of role provider to authorize requests credentials.
+ **Required**: No
+ **Default**: metadata
#### Properties for LDAP user authorization
-|Property|Description|Default|required|
-|--------|-----------|-------|--------|
-|`druid.auth.authorizer.MyBasicLDAPAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No|
-|`druid.auth.authorizer.MyBasicLDAPAuthorizer.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
-|`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminUser`|The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.|admin|No|
-|`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminRole`|The initial admin role to create if it doesn't already exists.|admin|No|
-|`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminGroupMapping`|The initial admin group mapping with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned. The name of this initial admin group mapping will be set to adminGroupMapping|null|No|
-|`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.type`|The type of role provider (ldap) to authorize requests credentials.|metadata|No
-|`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.groupFilters`|Array of LDAP group filters used to filter out the allowed set of groups returned from LDAP search. Filters can be begin with *, or end with ,* to provide configurational flexibility to limit or filter allowed set of groups available to LDAP Authorizer.|null|No|
+
+**`druid.auth.authorizer.MyBasicLDAPAuthorizer.enableCacheNotifications`**
+
+If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.
+ **Required**: No
+ **Default**: true
+
+**`druid.auth.authorizer.MyBasicLDAPAuthorizer.cacheNotificationTimeout`**
+
+The timeout in milliseconds for the cache notifications.
+ **Required**: No
+ **Default**: 5000
+
+**`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminUser`**
+
+The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.
+ **Required**: No
+ **Default**: admin
+
+**`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminRole`**
+
+The initial admin role to create if it doesn't already exists.
+ **Required**: No
+ **Default**: admin
+
+**`druid.auth.authorizer.MyBasicLDAPAuthorizer.initialAdminGroupMapping`**
+
+The initial admin group mapping with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned. The name of this initial admin group mapping will be set to adminGroupMapping
+ **Required**: No
+ **Default**: null
+
+**`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.type`**
+
+The type of role provider (ldap) to authorize requests credentials.
+ **Required**: No
+ **Default**: metadata
+
+**`druid.auth.authorizer.MyBasicLDAPAuthorizer.roleProvider.groupFilters`**
+
+Array of LDAP group filters used to filter out the allowed set of groups returned from LDAP search. Filters can be begin with *, or end with ,* to provide configurational flexibility to limit or filter allowed set of groups available to LDAP Authorizer.
+ **Required**: No
+ **Default**: null
+
+#### Properties for LDAPS
+
+Use the following properties to configure Druid authentication with LDAP over TLS (LDAPS). See [Configure LDAP authentication](../../operations/auth-ldap.md) for more information.
+
+**`druid.auth.basic.ssl.protocol`**
+
+SSL protocol to use. The TLS version is 1.2.
+ **Required**: Yes
+ **Default**: tls
+
+**`druid.auth.basic.ssl.trustStorePath`**
+
+Path to the trust store file.
+ **Required**: Yes
+ **Default**: N/A
+
+**`druid.auth.basic.ssl.trustStorePassword`**
+
+Password to access the trust store file.
+ **Required**: Yes
+ **Default**: N/A
+
+**`druid.auth.basic.ssl.trustStoreType`**
+
+Format of the trust store file. For Java the format is jks.
+ **Required**: No
+ **Default**: jks
+
+**`druid.auth.basic.ssl.trustStoreAlgorithm`**
+
+Algorithm used by the trust manager to validate certificate chains.
+ **Required**: No
+ **Default**: N/A
+
+**`druid.auth.basic.ssl.trustStorePassword`**
+
+Password details that enable access to the truststore.
+ **Required**: No
+ **Default**: N/A
+
+Example LDAPS configuration:
+
+```json
+druid.auth.basic.ssl.protocol=tls
+druid.auth.basic.ssl.trustStorePath=/usr/local/druid-path/certs/truststore.jks
+druid.auth.basic.ssl.trustStorePassword=xxxxx
+druid.auth.basic.ssl.trustStoreType=jks
+druid.auth.basic.ssl.trustStoreAlgorithm=PKIX
+```
+You can configure `druid.auth.basic.ssl.trustStorePassword` to be a plain text password or you can set the password as an environment variable. See [Password providers](../../operations/password-provider.md) for more information.
## Usage
@@ -213,19 +471,19 @@ Root path: `/druid-ext/basic-security/authentication`
Each API endpoint includes {authenticatorName}, specifying which Authenticator instance is being configured.
##### User/Credential Management
-`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users)`
+`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users)`
Return a list of all user names.
-`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
+`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
Return the name and credentials information of the user with name {userName}
-`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
+`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
Create a new user with name {userName}
-`DELETE(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
+`DELETE(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName})`
Delete the user with name {userName}
-`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName}/credentials)`
+`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName}/credentials)`
Assign a password used for HTTP basic authentication for {userName}
Content: JSON password request object
@@ -238,20 +496,20 @@ Example request body:
```
##### Cache Load Status
-`GET(/druid-ext/basic-security/authentication/loadStatus)`
+`GET(/druid-ext/basic-security/authentication/loadStatus)`
Return the current load status of the local caches of the authentication Druid metadata store.
#### Authorization API
-Root path: `/druid-ext/basic-security/authorization`
+Root path: `/druid-ext/basic-security/authorization`
Each API endpoint includes {authorizerName}, specifying which Authorizer instance is being configured.
##### User Creation/Deletion
-`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users)`
+`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users)`
Return a list of all user names.
-`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
+`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
Return the name and role information of the user with name {userName}
Example output:
@@ -338,20 +596,20 @@ The `resourceNamePattern` is a compiled version of the resource name regex. It i
}
```
-`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
+`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
Create a new user with name {userName}
-`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
+`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})`
Delete the user with name {userName}
##### Group mapping Creation/Deletion
-`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings)`
+`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings)`
Return a list of all group mappings.
-`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
+`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
Return the group mapping and role information of the group mapping with name {groupMappingName}
-`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
+`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
Create a new group mapping with name {groupMappingName}
Content: JSON group mapping object
Example request body:
@@ -366,14 +624,14 @@ Example request body:
}
```
-`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
+`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName})`
Delete the group mapping with name {groupMappingName}
#### Role Creation/Deletion
-`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles)`
+`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles)`
Return a list of all role names.
-`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
+`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
Return name and permissions for the role named {roleName}.
Example output:
@@ -427,30 +685,30 @@ Example output:
```
-`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
+`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
Create a new role with name {roleName}.
Content: username string
-`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
+`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName})`
Delete the role with name {roleName}.
#### Role Assignment
-`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})`
+`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})`
Assign role {roleName} to user {userName}.
-`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})`
+`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})`
Unassign role {roleName} from user {userName}
-`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})`
+`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})`
Assign role {roleName} to group mapping {groupMappingName}.
-`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})`
+`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName})`
Unassign role {roleName} from group mapping {groupMappingName}
#### Permissions
-`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName}/permissions)`
+`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName}/permissions)`
Set the permissions of {roleName}. This replaces the previous set of permissions on the role.
Content: List of JSON Resource-Action objects, e.g.:
@@ -479,5 +737,5 @@ The "name" field for resources in the permission definitions are regexes used to
Please see [Defining permissions](../../operations/security-user-auth.md#defining-permissions) for more details.
##### Cache Load Status
-`GET(/druid-ext/basic-security/authorization/loadStatus)`
+`GET(/druid-ext/basic-security/authorization/loadStatus)`
Return the current load status of the local caches of the authorization Druid metadata store.
diff --git a/docs/development/extensions-core/druid-lookups.md b/docs/development/extensions-core/druid-lookups.md
index b44f9620bd0a..5b19508c2375 100644
--- a/docs/development/extensions-core/druid-lookups.md
+++ b/docs/development/extensions-core/druid-lookups.md
@@ -22,9 +22,6 @@ title: "Cached Lookup Module"
~ under the License.
-->
-
-> Please note that this is an experimental module and the development/testing still at early stage. Feel free to try it and give us your feedback.
-
## Description
This Apache Druid module provides a per-lookup caching mechanism for JDBC data sources.
The main goal of this cache is to speed up the access to a high latency lookup sources and to provide a caching isolation for every lookup source.
diff --git a/docs/development/extensions-core/kafka-supervisor-reference.md b/docs/development/extensions-core/kafka-supervisor-reference.md
index 210207302f2d..360e75a83de4 100644
--- a/docs/development/extensions-core/kafka-supervisor-reference.md
+++ b/docs/development/extensions-core/kafka-supervisor-reference.md
@@ -56,8 +56,6 @@ This topic contains configuration reference information for the Apache Kafka sup
## Task Autoscaler Properties
-> Note that Task AutoScaler is currently designated as experimental.
-
| Property | Description | Required |
| ------------- | ------------- | ------------- |
| `enableTaskAutoScaler` | Enable or disable autoscaling. `false` or blank disables the `autoScaler` even when `autoScalerConfig` is not null| no (default == false) |
@@ -206,8 +204,8 @@ The `tuningConfig` is optional and default parameters will be used if no `tuning
| `handoffConditionTimeout` | Long | Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. | no (default == 0) |
| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kafka messages that are no longer available (i.e. when `OffsetOutOfRangeException` is encountered). If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../operations/api-reference.md#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion. If true, Druid will automatically reset to the earlier or latest offset available in Kafka, based on the value of the `useEarliestOffset` property (earliest if true, latest if false). Note that this can lead to data being _DROPPED_ (if `useEarliestOffset` is false) or _DUPLICATED_ (if `useEarliestOffset` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data. This feature behaves similarly to the Kafka `auto.offset.reset` consumer property. | no (default == false) |
| `workerThreads` | Integer | The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation. | no (default == min(10, taskCount)) |
-| `chatAsync` | Boolean | If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == false) |
-| `chatThreads` | Integer | The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true`. | no (default == min(10, taskCount * replicas)) |
+| `chatAsync` | Boolean | If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == true) |
+| `chatThreads` | Integer | The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true` (the default). | no (default == min(10, taskCount * replicas)) |
| `chatRetries` | Integer | The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive. | no (default == 8) |
| `httpTimeout` | ISO8601 Period | How long to wait for a HTTP response from an indexing task. | no (default == PT10S) |
| `shutdownTimeout` | ISO8601 Period | How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting. | no (default == PT80S) |
diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md
index 84036665ec9e..9b14ec767c26 100644
--- a/docs/development/extensions-core/kinesis-ingestion.md
+++ b/docs/development/extensions-core/kinesis-ingestion.md
@@ -149,8 +149,6 @@ Where the file `supervisor-spec.json` contains a Kinesis supervisor spec:
#### Task Autoscaler Properties
-> Note that Task AutoScaler is currently designated as experimental.
-
| Property | Description | Required |
| ------------- | ------------- | ------------- |
| `enableTaskAutoScaler` | Enable or disable the auto scaler. When false or absent, Druid disables the `autoScaler` even when `autoScalerConfig` is not null.| no (default == false) |
@@ -293,8 +291,8 @@ The `tuningConfig` is optional. If no `tuningConfig` is specified, default param
|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available. If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../operations/api-reference.md#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion. If true, Druid will automatically reset to the earlier or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if true, latest if false). Please note that this can lead to data being _DROPPED_ (if `useEarliestSequenceNumber` is false) or _DUPLICATED_ (if `useEarliestSequenceNumber` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.|no (default == false)|
|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`.|no (default == false)|
|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|no (default == min(10, taskCount))|
-|`chatAsync`|Boolean| If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == false) |
-|`chatThreads`|Integer| The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true`.| no (default == min(10, taskCount * replicas))|
+|`chatAsync`|Boolean| If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == true) |
+|`chatThreads`|Integer| The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true` (the default).| no (default == min(10, taskCount * replicas))|
|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.| no (default == 8)|
|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)|
|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)|
diff --git a/docs/ingestion/data-formats.md b/docs/ingestion/data-formats.md
index eb08df0cf7a8..557060a5e663 100644
--- a/docs/ingestion/data-formats.md
+++ b/docs/ingestion/data-formats.md
@@ -606,9 +606,9 @@ For example:
### FlattenSpec
-The `flattenSpec` object bridges the gap between potentially nested input data, such as Avro or ORC, and Druid's flat data model. It is an object within the `inputFormat` object.
+You can use the `flattenSpec` object to flatten nested data, as an alternative to the Druid [nested columns](../querying/nested-columns.md) feature, and for nested input formats unsupported by the feature. It is an object within the `inputFormat` object.
-> If you have nested JSON data, you can ingest and store JSON in an Apache Druid column as a `COMPLEX` data type. See [Nested columns](../querying/nested-columns.md) for more information.
+See [Nested columns](../querying/nested-columns.md) for information on ingesting and storing nested data in an Apache Druid column as a `COMPLEX` data type.
Configure your `flattenSpec` as follows:
diff --git a/docs/ingestion/migrate-from-firehose-ingestion.md b/docs/ingestion/migrate-from-firehose-ingestion.md
new file mode 100644
index 000000000000..c5d3f6946f21
--- /dev/null
+++ b/docs/ingestion/migrate-from-firehose-ingestion.md
@@ -0,0 +1,209 @@
+---
+id: migrate-from-firehose
+title: "Migrate from firehose to input source ingestion"
+sidebar_label: "Migrate from firehose"
+---
+
+
+
+Apache deprecated support for Druid firehoses in version 0.17. Support for firehose ingestion will be removed in version 26.0.
+
+If you're using a firehose for batch ingestion, we strongly recommend that you follow the instructions on this page to transition to using native batch ingestion input sources as soon as possible.
+
+Firehose ingestion doesn't work with newer Druid versions, so you must be using an ingestion spec with a defined input source before you upgrade.
+
+## Migrate from firehose ingestion to an input source
+
+To migrate from firehose ingestion, you can use the Druid console to update your ingestion spec, or you can update it manually.
+
+### Use the Druid console
+
+To update your ingestion spec using the Druid console, open the console and copy your spec into the **Edit spec** stage of the data loader.
+
+Druid converts the spec into one with a defined input source. For example, it converts the [example firehose ingestion spec](#example-firehose-ingestion-spec) below into the [example ingestion spec after migration](#example-ingestion-spec-after-migration).
+
+If you're unable to use the console or you have problems with the console method, the alternative is to update your ingestion spec manually.
+
+### Update your ingestion spec manually
+
+To update your ingestion spec manually, copy your existing spec into a new file. Refer to [Native batch ingestion with firehose (Deprecated)](./native-batch-firehose.md) for a description of firehose properties.
+
+Edit the new file as follows:
+
+1. In the `ioConfig` component, replace the `firehose` definition with an `inputSource` definition for your chosen input source. See [Native batch input sources](./native-batch-input-source.md) for details.
+2. Move the `timeStampSpec` definition from `parser.parseSpec` to the `dataSchema` component.
+3. Move the `dimensionsSpec` definition from `parser.parseSpec` to the `dataSchema` component.
+4. Move the `format` definition from `parser.parseSpec` to an `inputFormat` definition in `ioConfig`.
+5. Delete the `parser` definition.
+6. Save the file.
+ You can check the format of your new ingestion file against the [migrated example](#example-ingestion-spec-after-migration) below.
+7. Test the new ingestion spec with a temporary data source.
+8. Once you've successfully ingested sample data with the new spec, stop firehose ingestion and switch to the new spec.
+
+When the transition is complete, you can upgrade Druid to the latest version. See the [Druid release notes](https://druid.apache.org/downloads.html) for upgrade instructions.
+
+### Example firehose ingestion spec
+
+An example firehose ingestion spec is as follows:
+
+```json
+{
+ "type" : "index",
+ "spec" : {
+ "dataSchema" : {
+ "dataSource" : "wikipedia",
+ "metricsSpec" : [
+ {
+ "type" : "count",
+ "name" : "count"
+ },
+ {
+ "type" : "doubleSum",
+ "name" : "added",
+ "fieldName" : "added"
+ },
+ {
+ "type" : "doubleSum",
+ "name" : "deleted",
+ "fieldName" : "deleted"
+ },
+ {
+ "type" : "doubleSum",
+ "name" : "delta",
+ "fieldName" : "delta"
+ }
+ ],
+ "granularitySpec" : {
+ "type" : "uniform",
+ "segmentGranularity" : "DAY",
+ "queryGranularity" : "NONE",
+ "intervals" : [ "2013-08-31/2013-09-01" ]
+ },
+ "parser": {
+ "type": "string",
+ "parseSpec": {
+ "format": "json",
+ "timestampSpec" : {
+ "column" : "timestamp",
+ "format" : "auto"
+ },
+ "dimensionsSpec" : {
+ "dimensions": ["country", "page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","region","city"],
+ "dimensionExclusions" : []
+ }
+ }
+ }
+ },
+ "ioConfig" : {
+ "type" : "index",
+ "firehose" : {
+ "type" : "local",
+ "baseDir" : "examples/indexing/",
+ "filter" : "wikipedia_data.json"
+ }
+ },
+ "tuningConfig" : {
+ "type" : "index",
+ "partitionsSpec": {
+ "type": "single_dim",
+ "partitionDimension": "country",
+ "targetRowsPerSegment": 5000000
+ }
+ }
+ }
+}
+```
+
+### Example ingestion spec after migration
+
+The following example illustrates the result of migrating the [example firehose ingestion spec](#example-firehose-ingestion-spec) to a spec with an input source:
+
+```json
+{
+ "type" : "index",
+ "spec" : {
+ "dataSchema" : {
+ "dataSource" : "wikipedia",
+ "timestampSpec" : {
+ "column" : "timestamp",
+ "format" : "auto"
+ },
+ "dimensionsSpec" : {
+ "dimensions": ["country", "page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","region","city"],
+ "dimensionExclusions" : []
+ },
+ "metricsSpec" : [
+ {
+ "type" : "count",
+ "name" : "count"
+ },
+ {
+ "type" : "doubleSum",
+ "name" : "added",
+ "fieldName" : "added"
+ },
+ {
+ "type" : "doubleSum",
+ "name" : "deleted",
+ "fieldName" : "deleted"
+ },
+ {
+ "type" : "doubleSum",
+ "name" : "delta",
+ "fieldName" : "delta"
+ }
+ ],
+ "granularitySpec" : {
+ "type" : "uniform",
+ "segmentGranularity" : "DAY",
+ "queryGranularity" : "NONE",
+ "intervals" : [ "2013-08-31/2013-09-01" ]
+ }
+ },
+ "ioConfig" : {
+ "type" : "index",
+ "inputSource" : {
+ "type" : "local",
+ "baseDir" : "examples/indexing/",
+ "filter" : "wikipedia_data.json"
+ },
+ "inputFormat": {
+ "type": "json"
+ }
+ },
+ "tuningConfig" : {
+ "type" : "index",
+ "partitionsSpec": {
+ "type": "single_dim",
+ "partitionDimension": "country",
+ "targetRowsPerSegment": 5000000
+ }
+ }
+ }
+}
+```
+
+## Learn more
+
+For more information, see the following pages:
+
+- [Ingestion](./index.md): Overview of the Druid ingestion process.
+- [Native batch ingestion](./native-batch.md): Description of the supported native batch indexing tasks.
+- [Ingestion spec reference](./ingestion-spec.md): Description of the components and properties in the ingestion spec.
diff --git a/docs/ingestion/native-batch-firehose.md b/docs/ingestion/native-batch-firehose.md
index 4e2cad97fc1a..ca848e725c87 100644
--- a/docs/ingestion/native-batch-firehose.md
+++ b/docs/ingestion/native-batch-firehose.md
@@ -1,6 +1,6 @@
---
id: native-batch-firehose
-title: "Native batch ingestion with firehose"
+title: "Native batch ingestion with firehose (Deprecated)"
sidebar_label: "Firehose (deprecated)"
---
@@ -23,14 +23,13 @@ sidebar_label: "Firehose (deprecated)"
~ under the License.
-->
-
-Firehoses are deprecated in 0.17.0. It's highly recommended to use the [Native batch ingestion input sources](./native-batch-input-source.md) instead.
+> Firehose ingestion is deprecated. See [Migrate from firehose to input source ingestion](./migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources.
There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment.
## StaticS3Firehose
-> You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the StaticS3Firehose.
+You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the StaticS3Firehose.
This firehose ingests events from a predefined list of S3 objects.
This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md).
@@ -62,7 +61,7 @@ Note that prefetching or caching isn't that useful in the Parallel task.
## StaticGoogleBlobStoreFirehose
-> You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the StaticGoogleBlobStoreFirehose.
+You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the StaticGoogleBlobStoreFirehose.
This firehose ingests events, similar to the StaticS3Firehose, but from an Google Cloud Store.
@@ -112,7 +111,7 @@ Google Blobs:
## HDFSFirehose
-> You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFSFirehose.
+You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFSFirehose.
This firehose ingests events from a predefined list of files from the HDFS storage.
This firehose is _splittable_ and can be used by the [Parallel task](./native-batch.md).
diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md
index 7106a9a000a3..e8d83f34046a 100644
--- a/docs/ingestion/native-batch.md
+++ b/docs/ingestion/native-batch.md
@@ -679,8 +679,7 @@ Returns the task attempt history of the worker task spec of the given id, or HTT
While ingesting data using the parallel task indexing, Druid creates segments from the input data and pushes them. For segment pushing,
the parallel task index supports the following segment pushing modes based upon your type of [rollup](./rollup.md):
-- Bulk pushing mode: Used for perfect rollup. Druid pushes every segment at the very end of the index task. Until then, Druid stores created segments in memory and local storage of the service running the index task. This mode can cause problems if you have limited storage capacity, and is not recommended to use in production.
-To enable bulk pushing mode, set `forceGuaranteedRollup` in your TuningConfig. You cannot use bulk pushing with `appendToExisting` in your IOConfig.
+- Bulk pushing mode: Used for perfect rollup. Druid pushes every segment at the very end of the index task. Until then, Druid stores created segments in memory and local storage of the service running the index task. To enable bulk pushing mode, set `forceGuaranteedRollup` to `true` in your tuning config. You cannot use bulk pushing with `appendToExisting` in your IOConfig.
- Incremental pushing mode: Used for best-effort rollup. Druid pushes segments are incrementally during the course of the indexing task. The index task collects data and stores created segments in the memory and disks of the services running the task until the total number of collected rows exceeds `maxTotalRows`. At that point the index task immediately pushes all segments created up until that moment, cleans up pushed segments, and continues to ingest the remaining data.
## Capacity planning
diff --git a/docs/ingestion/schema-design.md b/docs/ingestion/schema-design.md
index 10e6ea82cd2b..f006e792bc44 100644
--- a/docs/ingestion/schema-design.md
+++ b/docs/ingestion/schema-design.md
@@ -116,14 +116,13 @@ naturally emitted. It is also useful if you want to combine timeseries and non-t
Similar to log aggregation systems, Druid offers inverted indexes for fast searching and filtering. Druid's search
capabilities are generally less developed than these systems, and its analytical capabilities are generally more
developed. The main data modeling differences between Druid and these systems are that when ingesting data into Druid,
-you must be more explicit. Druid columns have types specific upfront and Druid does not, at this time, natively support
-nested data.
+you must be more explicit. Druid columns have types specific upfront.
Tips for modeling log data in Druid:
* If you don't know ahead of time what columns you'll want to ingest, use an empty dimensions list to trigger
[automatic detection of dimension columns](#schema-less-dimensions).
-* If you have nested data, flatten it using a [`flattenSpec`](./ingestion-spec.md#flattenspec).
+* If you have nested data, you can ingest it using the [nested columns](../querying/nested-columns.md) feature or flatten it using a [`flattenSpec`](./ingestion-spec.md#flattenspec).
* Consider enabling [rollup](./rollup.md) if you have mainly analytical use cases for your log data. This will
mean you lose the ability to retrieve individual events from Druid, but you potentially gain substantial compression and
query performance boosts.
@@ -198,9 +197,9 @@ like `MILLIS_TO_TIMESTAMP`, `TIME_FLOOR`, and others. If you're using native Dru
### Nested dimensions
-You can ingest and store nested JSON in a Druid column as a `COMPLEX` data type. See [Nested columns](../querying/nested-columns.md) for more information.
+You can ingest and store nested data in a Druid column as a `COMPLEX` data type. See [Nested columns](../querying/nested-columns.md) for more information.
-If you want to ingest nested data in a format other than JSON—for example Avro, ORC, and Parquet—you must use the `flattenSpec` object to flatten it. For example, if you have data of the following form:
+If you want to ingest nested data in a format unsupported by the nested columns feature, you must use the `flattenSpec` object to flatten it. For example, if you have data of the following form:
```json
{ "foo": { "bar": 3 } }
diff --git a/docs/ingestion/tasks.md b/docs/ingestion/tasks.md
index c8a2e915d472..5afbadb3d43a 100644
--- a/docs/ingestion/tasks.md
+++ b/docs/ingestion/tasks.md
@@ -343,6 +343,26 @@ You can override the task priority by setting your priority in the task context
"priority" : 100
}
```
+
+
+## Task actions
+
+Task actions are overlord actions performed by tasks during their lifecycle. Some typical task actions are:
+- `lockAcquire`: acquires a time-chunk lock on an interval for the task
+- `lockRelease`: releases a lock acquired by the task on an interval
+- `segmentTransactionalInsert`: publishes new segments created by a task and optionally overwrites and/or drops existing segments in a single transaction
+- `segmentAllocate`: allocates pending segments to a task to write rows
+
+### Batching `segmentAllocate` actions
+
+In a cluster with several concurrent tasks, `segmentAllocate` actions on the overlord can take a long time to finish, causing spikes in the `task/action/run/time`. This can result in ingestion lag building up while a task waits for a segment to be allocated.
+The root cause of such spikes is likely to be one or more of the following:
+- several concurrent tasks trying to allocate segments for the same datasource and interval
+- large number of metadata calls made to the segments and pending segments tables
+- concurrency limitations while acquiring a task lock required for allocating a segment
+
+Since the contention typically arises from tasks allocating segments for the same datasource and interval, you can improve the run times by batching the actions together.
+To enable batched segment allocation on the overlord, set `druid.indexer.tasklock.batchSegmentAllocation` to `true`. See [overlord configuration](../configuration/index.md#overlord-operations) for more details.
diff --git a/docs/multi-stage-query/concepts.md b/docs/multi-stage-query/concepts.md
index 44e5ea43d427..da0e774152d6 100644
--- a/docs/multi-stage-query/concepts.md
+++ b/docs/multi-stage-query/concepts.md
@@ -233,7 +233,8 @@ happens:
The [`maxNumTasks`](./reference.md#context-parameters) query parameter determines the maximum number of tasks your
query will use, including the one `query_controller` task. Generally, queries perform better with more workers. The
lowest possible value of `maxNumTasks` is two (one worker and one controller). Do not set this higher than the number of
-free slots available in your cluster; doing so will result in a [TaskStartTimeout](reference.md#error-codes) error.
+free slots available in your cluster; doing so will result in a [TaskStartTimeout](reference.md#error_TaskStartTimeout)
+error.
When [reading external data](#extern), EXTERN can read multiple files in parallel across
different worker tasks. However, EXTERN does not split individual files across multiple worker tasks. If you have a
diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md
index c76ab57aa7ac..648d3c297b47 100644
--- a/docs/multi-stage-query/known-issues.md
+++ b/docs/multi-stage-query/known-issues.md
@@ -33,16 +33,18 @@ sidebar_label: Known issues
- Worker task stage outputs are stored in the working directory given by `druid.indexer.task.baseDir`. Stages that
generate a large amount of output data may exhaust all available disk space. In this case, the query fails with
-an [UnknownError](./reference.md#error-codes) with a message including "No space left on device".
+an [UnknownError](./reference.md#error_UnknownError) with a message including "No space left on device".
## SELECT
- SELECT from a Druid datasource does not include unpublished real-time data.
- GROUPING SETS and UNION ALL are not implemented. Queries using these features return a
- [QueryNotSupported](reference.md#error-codes) error.
+ [QueryNotSupported](reference.md#error_QueryNotSupported) error.
-- For some COUNT DISTINCT queries, you'll encounter a [QueryNotSupported](reference.md#error-codes) error that includes `Must not have 'subtotalsSpec'` as one of its causes. This is caused by the planner attempting to use GROUPING SETs, which are not implemented.
+- For some COUNT DISTINCT queries, you'll encounter a [QueryNotSupported](reference.md#error_QueryNotSupported) error
+ that includes `Must not have 'subtotalsSpec'` as one of its causes. This is caused by the planner attempting to use
+ GROUPING SETs, which are not implemented.
- The numeric varieties of the EARLIEST and LATEST aggregators do not work properly. Attempting to use the numeric
varieties of these aggregators lead to an error like
diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md
index 3550566874b8..8ea9adf61ae6 100644
--- a/docs/multi-stage-query/reference.md
+++ b/docs/multi-stage-query/reference.md
@@ -32,9 +32,9 @@ sidebar_label: Reference
This topic is a reference guide for the multi-stage query architecture in Apache Druid. For examples of real-world
usage, refer to the [Examples](examples.md) page.
-### EXTERN
+### `EXTERN`
-Use the EXTERN function to read external data.
+Use the `EXTERN` function to read external data.
Function format:
@@ -50,7 +50,7 @@ FROM TABLE(
)
```
-EXTERN consists of the following parts:
+`EXTERN` consists of the following parts:
1. Any [Druid input source](../ingestion/native-batch-input-source.md) as a JSON-encoded string.
2. Any [Druid input format](../ingestion/data-formats.md) as a JSON-encoded string.
@@ -58,12 +58,134 @@ EXTERN consists of the following parts:
For more information, see [Read external data with EXTERN](concepts.md#extern).
-### INSERT
+### `HTTP`, `INLINE` and `LOCALFILES`
-Use the INSERT statement to insert data.
+While `EXTERN` allows you to specify an external table using JSON, other table functions allow you
+describe the external table using SQL syntax. Each function works for one specific kind of input
+source. You provide properties using SQL named arguments. The row signature is given using the
+Druid SQL `EXTEND` keyword using SQL syntax and types. Function format:
-Unlike standard SQL, INSERT loads data into the target table according to column name, not positionally. If necessary,
-use `AS` in your SELECT column list to assign the correct names. Do not rely on their positions within the SELECT
+```sql
+SELECT
+
+FROM TABLE(
+ http(
+ userName => 'bob',
+ password => 'secret',
+ uris => 'http:foo.com/bar.csv',
+ format => 'csv'
+ )
+ ) EXTEND (x VARCHAR, y VARCHAR, z BIGINT)
+```
+
+Note that the `EXTEND` keyword is optional. The following is equally valid (and perhaps
+more convenient):
+
+```sql
+SELECT
+
+FROM TABLE(
+ http(
+ userName => 'bob',
+ password => 'secret',
+ uris => 'http:foo.com/bar.csv',
+ format => 'csv'
+ )
+ ) (x VARCHAR, y VARCHAR, z BIGINT)
+```
+
+
+The set of table functions and formats is preliminary in this release.
+
+#### `HTTP`
+
+The `HTTP` table function represents the `HttpInputSource` class in Druid which allows you to
+read from an HTTP server. The function accepts the following arguments:
+
+| Name | Description | JSON equivalent | Required |
+| ---- | ----------- | --------------- | -------- |
+| `userName` | Basic authentication user name | `httpAuthenticationUsername` | No |
+| `password` | Basic authentication password | `httpAuthenticationPassword` | No |
+| `passwordEnvVar` | Environment variable that contains the basic authentication password| `httpAuthenticationPassword` | No |
+| `uris` | Comma-separated list of URIs to read. | `uris` | Yes |
+
+#### `INLINE`
+
+The `INLINE` table function represents the `InlineInputSource` class in Druid which provides
+data directly in the table function. The function accepts the following arguments:
+
+| Name | Description | JSON equivalent | Required |
+| ---- | ----------- | --------------- | -------- |
+| `data` | Text lines of inline data. Separate lines with a newline. | `data` | Yes |
+
+#### `LOCALFILES`
+
+The `LOCALFILES` table function represents the `LocalInputSource` class in Druid which reads
+files from the file system of the node running Druid. This is most useful for single-node
+installations. The function accepts the following arguments:
+
+| Name | Description | JSON equivalent | Required |
+| ---- | ----------- | --------------- | -------- |
+| `baseDir` | Directory to read from. | `baseDir` | No |
+| `filter` | Filter pattern to read. Example: `*.csv`. | `filter` | No |
+| `files` | Comma-separated list of files to read. | `files` | No |
+
+You must either provide the `baseDir` or the list of `files`. You can provide both, in which case
+the files are assumed relative to the `baseDir`. If you provide a `filter`, you must provide the
+`baseDir`.
+
+Note that, due to [Issue #13359](https://github.com/apache/druid/issues/13359), the functionality
+described above is broken. Until that issue is resolved, you must provide one or more absolute
+file paths in the `files` property and the other two properties are unavailable.
+
+#### Table Function Format
+
+Each of the table functions above requires that you specify a format.
+
+| Name | Description | JSON equivalent | Required |
+| ---- | ----------- | --------------- | -------- |
+| `format` | The input format, using the same names as for `EXTERN`. | `inputFormat.type` | Yes |
+
+#### CSV Format
+
+Use the `csv` format to read from CSV. This choice selects the Druid `CsvInputFormat` class.
+
+| Name | Description | JSON equivalent | Required |
+| ---- | ----------- | --------------- | -------- |
+| `listDelimiter` | The delimiter to use for fields that represent a list of strings. | `listDelimiter` | No |
+| `skipRows` | The number of rows to skip at the start of the file. Default is 0. | `skipHeaderRows` | No |
+
+MSQ does not have the ability to infer schema from a CSV, file, so the `findColumnsFromHeader` property
+is unavailable. Instead, Columns are given using the `EXTEND` syntax described above.
+
+#### Delimited Text Format
+
+Use the `tsv` format to read from an arbitrary delimited (CSV-like) file such as tab-delimited,
+pipe-delimited, etc. This choice selects the Druid `DelimitedInputFormat` class.
+
+| Name | Description | JSON equivalent | Required |
+| ---- | ----------- | --------------- | -------- |
+| `delimiter` | The delimiter which separates fields. | `delimiter` | Yes |
+| `listDelimiter` | The delimiter to use for fields that represent a list of strings. | `listDelimiter` | No |
+| `skipRows` | The number of rows to skip at the start of the file. Default is 0. | `skipHeaderRows` | No |
+
+As noted above, MSQ cannot infer schema using headers. Use `EXTEND` instead.
+
+#### JSON Format
+
+Use the `json` format to read from a JSON input source. This choice selects the Druid `JsonInputFormat` class.
+
+| Name | Description | JSON equivalent | Required |
+| ---- | ----------- | --------------- | -------- |
+| `keepNulls` | Whether to keep null values. Defaults to `false`. | `keepNullColumns` | No |
+
+
+### `INSERT`
+
+Use the `INSERT` statement to insert data.
+
+Unlike standard SQL, `INSERT` loads data into the target table according to column name, not positionally. If necessary,
+use `AS` in your `SELECT` column list to assign the correct names. Do not rely on their positions within the SELECT
clause.
Statement format:
@@ -85,15 +207,15 @@ INSERT consists of the following parts:
For more information, see [Load data with INSERT](concepts.md#insert).
-### REPLACE
+### `REPLACE`
-You can use the REPLACE function to replace all or some of the data.
+You can use the `REPLACE` function to replace all or some of the data.
-Unlike standard SQL, REPLACE loads data into the target table according to column name, not positionally. If necessary,
-use `AS` in your SELECT column list to assign the correct names. Do not rely on their positions within the SELECT
+Unlike standard SQL, `REPLACE` loads data into the target table according to column name, not positionally. If necessary,
+use `AS` in your `SELECT` column list to assign the correct names. Do not rely on their positions within the SELECT
clause.
-#### REPLACE all data
+#### `REPLACE` all data
Function format to replace all data:
@@ -105,7 +227,7 @@ PARTITIONED BY
[ CLUSTERED BY ]
```
-#### REPLACE specific time ranges
+#### `REPLACE` specific time ranges
Function format to replace specific time ranges:
@@ -117,7 +239,7 @@ PARTITIONED BY
[ CLUSTERED BY ]
```
-REPLACE consists of the following parts:
+`REPLACE` consists of the following parts:
1. Optional [context parameters](./reference.md#context-parameters).
2. A `REPLACE INTO ` clause at the start of your query, such as `REPLACE INTO "your-table".`
@@ -132,7 +254,7 @@ REPLACE consists of the following parts:
For more information, see [Overwrite data with REPLACE](concepts.md#replace).
-### PARTITIONED BY
+### `PARTITIONED BY`
The `PARTITIONED BY ` clause is required for [INSERT](#insert) and [REPLACE](#replace). See
[Partitioning](concepts.md#partitioning) for details.
@@ -164,7 +286,7 @@ The following ISO 8601 periods are supported for `TIME_FLOOR`:
For more information about partitioning, see [Partitioning](concepts.md#partitioning).
-### CLUSTERED BY
+### `CLUSTERED BY`
The `CLUSTERED BY ` clause is optional for [INSERT](#insert) and [REPLACE](#replace). It accepts a list of
column names or expressions.
@@ -203,14 +325,37 @@ The following table lists the context parameters for the MSQ task engine:
| `maxParseExceptions`| SELECT, INSERT, REPLACE Maximum number of parse exceptions that are ignored while executing the query before it stops with `TooManyWarningsFault`. To ignore all the parse exceptions, set the value to -1.| 0 |
| `rowsPerSegment` | INSERT or REPLACE The number of rows per segment to target. The actual number of rows per segment may be somewhat higher or lower than this number. In most cases, use the default. For general information about sizing rows per segment, see [Segment Size Optimization](../operations/segment-optimization.md). | 3,000,000 |
| `indexSpec` | INSERT or REPLACE An [`indexSpec`](../ingestion/ingestion-spec.md#indexspec) to use when generating segments. May be a JSON string or object. See [Front coding](../ingestion/ingestion-spec.md#front-coding) for details on configuring an `indexSpec` with front coding. | See [`indexSpec`](../ingestion/ingestion-spec.md#indexspec). |
+| `clusterStatisticsMergeMode` | Whether to use parallel or sequential mode for merging of the worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. See [Sketch Merging Mode](#sketch-merging-mode) for more information. | `AUTO` |
+
+## Sketch Merging Mode
+This section details the advantages and performance of various Cluster By Statistics Merge Modes.
+
+If a query requires key statistics to generate partition boundaries, key statistics are gathered by the workers while
+reading rows from the datasource. These statistics must be transferred to the controller to be merged together.
+`clusterStatisticsMergeMode` configures the way in which this happens.
+
+`PARALLEL` mode fetches the key statistics for all time chunks from all workers together and the controller then downsamples
+the sketch if it does not fit in memory. This is faster than `SEQUENTIAL` mode as there is less over head in fetching sketches
+for all time chunks together. This is good for small sketches which won't be downsampled even if merged together or if
+accuracy in segment sizing for the ingestion is not very important.
+
+`SEQUENTIAL` mode fetches the sketches in ascending order of time and generates the partition boundaries for one time
+chunk at a time. This gives more working memory to the controller for merging sketches, which results in less
+downsampling and thus, more accuracy. There is, however, a time overhead on fetching sketches in sequential order. This is
+good for cases where accuracy is important.
+
+`AUTO` mode tries to find the best approach based on number of workers and size of input rows. If there are more
+than 100 workers or if the combined sketch size among all workers is more than 1GB, `SEQUENTIAL` is chosen, otherwise,
+`PARALLEL` is chosen.
## Durable Storage
+
This section enumerates the advantages and performance implications of enabling durable storage while executing MSQ tasks.
To prevent durable storage from getting filled up with temporary files in case the tasks fail to clean them up, a periodic
cleaner can be scheduled to clean the directories corresponding to which there isn't a controller task running. It utilizes
-the storage connector to work upon the durable storage. The durable storage location should only be utilized to store the output
-for cluster's MSQ tasks. If the location contains other files or directories, then they will get cleaned up as well.
+the storage connector to work upon the durable storage. The durable storage location should only be utilized to store the output
+for cluster's MSQ tasks. If the location contains other files or directories, then they will get cleaned up as well.
Following table lists the properties that can be set to control the behavior of the durable storage of the cluster.
|Parameter |Default | Description |
@@ -227,14 +372,14 @@ The following table lists query limits:
| Limit | Value | Error if exceeded |
|---|---|---|
-| Size of an individual row written to a frame. Row size when written to a frame may differ from the original row size. | 1 MB | `RowTooLarge` |
-| Number of segment-granular time chunks encountered during ingestion. | 5,000 | `TooManyBuckets` |
-| Number of input files/segments per worker. | 10,000 | `TooManyInputFiles` |
-| Number of output partitions for any one stage. Number of segments generated during ingestion. |25,000 | `TooManyPartitions` |
-| Number of output columns for any one stage. | 2,000 | `TooManyColumns` |
-| Number of cluster by columns that can appear in a stage | 1,500 | `TooManyClusteredByColumns` |
-| Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | `TooManyWorkers` |
-| Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](concepts.md#memory-usage). | `BroadcastTablesTooLarge` |
+| Size of an individual row written to a frame. Row size when written to a frame may differ from the original row size. | 1 MB | [`RowTooLarge`](#error_RowTooLarge) |
+| Number of segment-granular time chunks encountered during ingestion. | 5,000 | [`TooManyBuckets`](#error_TooManyBuckets) |
+| Number of input files/segments per worker. | 10,000 | [`TooManyInputFiles`](#error_TooManyInputFiles) |
+| Number of output partitions for any one stage. Number of segments generated during ingestion. |25,000 | [`TooManyPartitions`](#error_TooManyPartitions) |
+| Number of output columns for any one stage. | 2,000 | [`TooManyColumns`](#error_TooManyColumns) |
+| Number of cluster by columns that can appear in a stage | 1,500 | [`TooManyClusteredByColumns`](#error_TooManyClusteredByColumns) |
+| Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | [`TooManyWorkers`](#error_TooManyWorkers) |
+| Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](concepts.md#memory-usage). | [`BroadcastTablesTooLarge`](#error_BroadcastTablesTooLarge) |
@@ -244,30 +389,30 @@ The following table describes error codes you may encounter in the `multiStageQu
| Code | Meaning | Additional fields |
|---|---|---|
-| `BroadcastTablesTooLarge` | The size of the broadcast tables used in the right hand side of the join exceeded the memory reserved for them in a worker task. Try increasing the peon memory or reducing the size of the broadcast tables. | `maxBroadcastTablesSize`: Memory reserved for the broadcast tables, measured in bytes. |
-| `Canceled` | The query was canceled. Common reasons for cancellation:User-initiated shutdown of the controller task via the `/druid/indexer/v1/task/{taskId}/shutdown` API. Restart or failure of the server process that was running the controller task. | |
-| `CannotParseExternalData` | A worker task could not parse data from an external datasource. | `errorMessage`: More details on why parsing failed. |
-| `ColumnNameRestricted` | The query uses a restricted column name. | `columnName`: The restricted column name. |
-| `ColumnTypeNotSupported` | The column type is not supported. This can be because: Support for writing or reading from a particular column type is not supported. The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames. | `columnName`: The column name with an unsupported type. `columnType`: The unknown column type. |
-| `InsertCannotAllocateSegment` | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts: Attempting to mix different granularities in the same intervals of the same datasource. Prior ingestions that used non-extendable shard specs. | `dataSource` `interval`: The interval for the attempted new segment allocation. |
-| `InsertCannotBeEmpty` | An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`. | `dataSource` |
-| `InsertCannotOrderByDescending` | An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. | `columnName` |
-| `InsertCannotReplaceExistingSegment` | A REPLACE query cannot proceed because an existing segment partially overlaps those bounds, and the portion within the bounds is not fully overshadowed by query results. There are two ways to address this without modifying your query:Shrink the OVERLAP filter to match the query results. Expand the OVERLAP filter to fully contain the existing segment. | `segmentId`: The existing segment
-| `InsertLockPreempted` | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task. | |
-| `InsertTimeNull` | An INSERT or REPLACE query encountered a null timestamp in the `__time` field. This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. (TIME_PARSE returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern. If your timestamps may genuinely be null, consider using COALESCE to provide a default value. One option is CURRENT_TIMESTAMP, which represents the start time of the job. |
-| `InsertTimeOutOfBounds` | A REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause. To avoid this error, verify that the you specified is valid. | `interval`: time chunk interval corresponding to the out-of-bounds timestamp |
-| `InvalidNullByte` | A string column included a null byte. Null bytes in strings are not permitted. | `column`: The column that included the null byte |
-| `QueryNotSupported` | QueryKit could not translate the provided native query to a multi-stage query. This can happen if the query uses features that aren't supported, like GROUPING SETS. | |
-| `RowTooLarge` | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for the specific limit on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. |
-| `TaskStartTimeout` | Unable to launch all the worker tasks in time. There might be insufficient available slots to start all the worker tasks simultaneously. Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity. | `numTasks`: The number of tasks attempted to launch. |
-| `TooManyBuckets` | Exceeded the number of partition buckets for a stage. Partition buckets are only used for `segmentGranularity` during INSERT queries. The most common reason for this error is that your `segmentGranularity` is too narrow relative to the data. See the [Limits](#limits) table for the specific limit. | `maxBuckets`: The limit on buckets. |
-| `TooManyInputFiles` | Exceeded the number of input files/segments per worker. See the [Limits](#limits) table for the specific limit. | `numInputFiles`: The total number of input files/segments for the stage. `maxInputFiles`: The maximum number of input files/segments per worker per stage. `minNumWorker`: The minimum number of workers required for a successful run. |
-| `TooManyPartitions` | Exceeded the number of partitions for a stage. The most common reason for this is that the final stage of an INSERT or REPLACE query generated too many segments. See the [Limits](#limits) table for the specific limit. | `maxPartitions`: The limit on partitions which was exceeded |
-| `TooManyClusteredByColumns` | Exceeded the number of cluster by columns for a stage. See the [Limits](#limits) table for the specific limit. | `numColumns`: The number of columns requested. `maxColumns`: The limit on columns which was exceeded.`stage`: The stage number exceeding the limit |
-| `TooManyColumns` | Exceeded the number of columns for a stage. See the [Limits](#limits) table for the specific limit. | `numColumns`: The number of columns requested. `maxColumns`: The limit on columns which was exceeded. |
-| `TooManyWarnings` | Exceeded the allowed number of warnings of a particular type. | `rootErrorCode`: The error code corresponding to the exception that exceeded the required limit. `maxWarnings`: Maximum number of warnings that are allowed for the corresponding `rootErrorCode`. |
-| `TooManyWorkers` | Exceeded the supported number of workers running simultaneously. See the [Limits](#limits) table for the specific limit. | `workers`: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously. `maxWorkers`: The hard or soft limit on workers that was exceeded. |
-| `NotEnoughMemory` | Insufficient memory to launch a stage. | `serverMemory`: The amount of memory available to a single process. `serverWorkers`: The number of workers running in a single process. `serverThreads`: The number of threads in a single process. |
-| `WorkerFailed` | A worker task failed unexpectedly. | `errorMsg` `workerTaskId`: The ID of the worker task. |
-| `WorkerRpcFailed` | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task |
-| `UnknownError` | All other errors. | `message` |
+| `BroadcastTablesTooLarge` | The size of the broadcast tables used in the right hand side of the join exceeded the memory reserved for them in a worker task. Try increasing the peon memory or reducing the size of the broadcast tables. | `maxBroadcastTablesSize`: Memory reserved for the broadcast tables, measured in bytes. |
+| `Canceled` | The query was canceled. Common reasons for cancellation:User-initiated shutdown of the controller task via the `/druid/indexer/v1/task/{taskId}/shutdown` API. Restart or failure of the server process that was running the controller task. | |
+| `CannotParseExternalData` | A worker task could not parse data from an external datasource. | `errorMessage`: More details on why parsing failed. |
+| `ColumnNameRestricted` | The query uses a restricted column name. | `columnName`: The restricted column name. |
+| `ColumnTypeNotSupported` | The column type is not supported. This can be because: Support for writing or reading from a particular column type is not supported. The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames. | `columnName`: The column name with an unsupported type. `columnType`: The unknown column type. |
+| `InsertCannotAllocateSegment` | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts: Attempting to mix different granularities in the same intervals of the same datasource. Prior ingestions that used non-extendable shard specs. | `dataSource` `interval`: The interval for the attempted new segment allocation. |
+| `InsertCannotBeEmpty` | An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`. | `dataSource` |
+| `InsertCannotOrderByDescending` | An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. | `columnName` |
+| `InsertCannotReplaceExistingSegment` | A REPLACE query cannot proceed because an existing segment partially overlaps those bounds, and the portion within the bounds is not fully overshadowed by query results. There are two ways to address this without modifying your query:Shrink the OVERLAP filter to match the query results. Expand the OVERLAP filter to fully contain the existing segment. | `segmentId`: The existing segment
+| `InsertLockPreempted` | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task. | |
+| `InsertTimeNull` | An INSERT or REPLACE query encountered a null timestamp in the `__time` field. This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. (TIME_PARSE returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern. If your timestamps may genuinely be null, consider using COALESCE to provide a default value. One option is CURRENT_TIMESTAMP, which represents the start time of the job. |
+| `InsertTimeOutOfBounds` | A REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause. To avoid this error, verify that the you specified is valid. | `interval`: time chunk interval corresponding to the out-of-bounds timestamp |
+| `InvalidNullByte` | A string column included a null byte. Null bytes in strings are not permitted. | `column`: The column that included the null byte |
+| `QueryNotSupported` | QueryKit could not translate the provided native query to a multi-stage query. This can happen if the query uses features that aren't supported, like GROUPING SETS. | |
+| `RowTooLarge` | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for specific limits on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. |
+| `TaskStartTimeout` | Unable to launch all the worker tasks in time. There might be insufficient available slots to start all the worker tasks simultaneously. Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity. | `numTasks`: The number of tasks attempted to launch. |
+| `TooManyBuckets` | Exceeded the maximum number of partition buckets for a stage (5,000 partition buckets). < br />Partition buckets are created for each [`PARTITIONED BY`](#partitioned-by) time chunk for INSERT and REPLACE queries. The most common reason for this error is that your `PARTITIONED BY` is too narrow relative to your data. | `maxBuckets`: The limit on partition buckets. |
+| `TooManyInputFiles` | Exceeded the maximum number of input files or segments per worker (10,000 files or segments). If you encounter this limit, consider adding more workers, or breaking up your query into smaller queries that process fewer files or segments per query. | `numInputFiles`: The total number of input files/segments for the stage. `maxInputFiles`: The maximum number of input files/segments per worker per stage. `minNumWorker`: The minimum number of workers required for a successful run. |
+| `TooManyPartitions` | Exceeded the maximum number of partitions for a stage (25,000 partitions). This can occur with INSERT or REPLACE statements that generate large numbers of segments, since each segment is associated with a partition. If you encounter this limit, consider breaking up your INSERT or REPLACE statement into smaller statements that process less data per statement. | `maxPartitions`: The limit on partitions which was exceeded |
+| `TooManyClusteredByColumns` | Exceeded the maximum number of clustering columns for a stage (1,500 columns). This can occur with `CLUSTERED BY`, `ORDER BY`, or `GROUP BY` with a large number of columns. | `numColumns`: The number of columns requested. `maxColumns`: The limit on columns which was exceeded.`stage`: The stage number exceeding the limit |
+| `TooManyColumns` | Exceeded the maximum number of columns for a stage (2,000 columns). | `numColumns`: The number of columns requested. `maxColumns`: The limit on columns which was exceeded. |
+| `TooManyWarnings` | Exceeded the maximum allowed number of warnings of a particular type. | `rootErrorCode`: The error code corresponding to the exception that exceeded the required limit. `maxWarnings`: Maximum number of warnings that are allowed for the corresponding `rootErrorCode`. |
+| `TooManyWorkers` | Exceeded the maximum number of simultaneously-running workers. See the [Limits](#limits) table for more details. | `workers`: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously. `maxWorkers`: The hard or soft limit on workers that was exceeded. If this is lower than the hard limit (1,000 workers), then you can increase the limit by adding more memory to each task. |
+| `NotEnoughMemory` | Insufficient memory to launch a stage. | `serverMemory`: The amount of memory available to a single process. `serverWorkers`: The number of workers running in a single process. `serverThreads`: The number of threads in a single process. |
+| `WorkerFailed` | A worker task failed unexpectedly. | `errorMsg` `workerTaskId`: The ID of the worker task. |
+| `WorkerRpcFailed` | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task |
+| `UnknownError` | All other errors. | `message` |
diff --git a/docs/operations/auth-ldap.md b/docs/operations/auth-ldap.md
index b9603a0c38fd..5a2d8b7e292f 100644
--- a/docs/operations/auth-ldap.md
+++ b/docs/operations/auth-ldap.md
@@ -1,6 +1,7 @@
---
id: auth-ldap
-title: "LDAP auth"
+title: "Configure LDAP authentication"
+sidebar_label: "LDAP auth"
---
+You can use [Lightweight Directory Access Protocol (LDAP)](https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol) to secure access to Apache Druid. This topic describes how to set up Druid authentication and authorization with LDAP and LDAP over TLS (LDAPS). The examples on this page show the configuration for an Active Directory LDAP system.
-This page describes how to set up Druid user authentication and authorization through LDAP. The first step is to enable LDAP authentication and authorization for Druid. You then map an LDAP group to roles and assign permissions to roles.
+The first step is to enable LDAP authentication and authorization for Druid. You then map an LDAP group to Druid roles and assign permissions to those roles. After you've completed this configuration you can optionally choose to enable LDAPS to make LDAP traffic confidential and secure.
-## Enable LDAP in Druid
+## Prerequisites
-Before starting, verify that the active directory is reachable from the Druid Master servers. Command line tools such as `ldapsearch` and `ldapwhoami`, which are included with OpenLDAP, are useful for this testing.
+Before you start to configure LDAP for Druid, test your LDAP connection and perform a sample search.
-### Check the connection
+### Check your LDAP connection
-First test that the basic connection and user credential works. For example, given a user `uuser1@example.com`, try:
+Test your LDAP connection to verify it works with user credentials. Later in the process you [configure Druid for LDAP authentication](#configure-druid-for-ldap-authentication) with this user as the `bindUser`.
+
+The following example command tests the connection for the user `myuser@example.com`. Insert your LDAP server IP address. Modify the port number of your LDAP instance if it listens on a port other than `389`.
```bash
-ldapwhoami -vv -H ldap://:389 -D"uuser1@example.com" -W
+ldapwhoami -vv -H ldap://ip_address:389 -D "myuser@example.com" -W
```
-Enter the password associated with the user when prompted and verify that the command succeeded. If it didn't, try the following troubleshooting steps:
+Enter the password for the user when prompted and verify that the command succeeded. If it failed, check the following:
+
+- Make sure you're using the correct port for your LDAP instance.
+- Check if a network firewall is preventing connections to the LDAP port.
+- Review your LDAP implementation details to see whether you need to specifically allow LDAP clients at the LDAP server. If so, add the Druid Coordinator server to the allow list.
-* Verify that you've used the correct port for your LDAP instance. By default, the LDAP port is 389, but double-check with your LDAP admin if unable to connect.
-* Check whether a network firewall is not preventing connections to the LDAP port.
-* Check whether LDAP clients need to be specifically whitelisted at the LDAP server to be able to reach it. If so, add the Druid Coordinator server to the AD whitelist.
+### Test your LDAP search
+Once your LDAP connection is working, search for a user. For example, the following command searches for the user `myuser` in an Active Directory system. The `sAMAccountName` attribute is specific to Active Directory and contains the authenticated user identity:
-### Check the search criteria
+```bash
+ldapsearch -x -W -H ldap://ip_address:389 -D "cn=admin,dc=example,dc=com" -b "dc=example,dc=com" "(sAMAccountName=myuser)" +
+```
-After verifying basic connectivity, check your search criteria. For example, the command for searching for user `uuser1@example.com ` is as follows:
+The `memberOf` attribute in the results shows the groups the user belongs to. For example, the following response shows that the user is a member of the `mygroup` group:
```bash
-ldapsearch -x -W -H ldap:// -D"uuser1@example.com" -b "dc=example,dc=com" "(sAMAccountName=uuser1)"
+memberOf: cn=mygroup,ou=groups,dc=example,dc=com
```
-Note the `memberOf` attribute in the results; it shows the groups that the user belongs to. You will use this value to map the LDAP group to the Druid roles later. This attribute may be implemented differently on different types of LDAP servers. For instance, some LDAP servers may support recursive groupings, and some may not. Some LDAP server implementations may not have any object classes that contain this attribute altogether. If your LDAP server does not use the `memberOf` attribute, then Druid will not be able to determine a user's group membership using LDAP. The sAMAccountName attribute used in this example contains the authenticated user identity. This is an attribute of an object class specific to Microsoft Active Directory. The object classes and attribute used in your LDAP server may be different.
+You use this information to map the LDAP group to Druid roles in a later step.
+
+> Druid uses the `memberOf` attribute to determine a group's membership using LDAP. If your LDAP server implementation doesn't include this attribute, you must complete some additional steps when you [map LDAP groups to Druid roles](#map-ldap-groups-to-druid-roles).
+
+## Configure Druid for LDAP authentication
+
+To configure Druid to use LDAP authentication, follow these steps. See [Configuration reference](../configuration/index.md) for the location of the configuration files.
+
+1. Create a user in your LDAP system that you'll use both for internal communication with Druid and as the LDAP initial admin user. See [Security overview](./security-overview.md) for more information.
+In the example below, the LDAP user is `internal@example.com`.
-## Configure Druid user authentication with LDAP/Active Directory
+2. Enable the `druid-basic-security` extension in the `common.runtime.properties` file.
-1. Enable the `druid-basic-security` extension in the `common.runtime.properties` file. See [Security Overview](security-overview.md) for details.
-2. As a best practice, create a user in LDAP to be used for internal communication with Druid.
-3. In `common.runtime.properties`, update LDAP-related properties, as shown in the following listing:
- ```
- druid.auth.authenticatorChain=["ldap"]
- druid.auth.authenticator.ldap.type=basic
- druid.auth.authenticator.ldap.enableCacheNotifications=true
- druid.auth.authenticator.ldap.credentialsValidator.type=ldap
- druid.auth.authenticator.ldap.credentialsValidator.url=ldap://:
- druid.auth.authenticator.ldap.credentialsValidator.bindUser=
- druid.auth.authenticator.ldap.credentialsValidator.bindPassword=
- druid.auth.authenticator.ldap.credentialsValidator.baseDn=
- druid.auth.authenticator.ldap.credentialsValidator.userSearch=
- druid.auth.authenticator.ldap.credentialsValidator.userAttribute=sAMAccountName
- druid.auth.authenticator.ldap.authorizerName=ldapauth
- druid.escalator.type=basic
- druid.escalator.internalClientUsername=
- druid.escalator.internalClientPassword=Welcome123
- druid.escalator.authorizerName=ldapauth
- druid.auth.authorizers=["ldapauth"]
- druid.auth.authorizer.ldapauth.type=basic
- druid.auth.authorizer.ldapauth.initialAdminUser=AD user who acts as the initial admin user, e.g.: internal@example.com>
- druid.auth.authorizer.ldapauth.initialAdminRole=admin
- druid.auth.authorizer.ldapauth.roleProvider.type=ldap
+3. In the `common.runtime.properties` file, add the following lines for LDAP properties and substitute the values for your own. See [Druid basic security](../development/extensions-core/druid-basic-security.md#properties-for-ldap-user-authentication) for details about these properties.
+
+ ```
+ druid.auth.authenticatorChain=["ldap"]
+ druid.auth.authenticator.ldap.type=basic
+ druid.auth.authenticator.ldap.enableCacheNotifications=true
+ druid.auth.authenticator.ldap.credentialsValidator.type=ldap
+ druid.auth.authenticator.ldap.credentialsValidator.url=ldap://ip_address:port
+ druid.auth.authenticator.ldap.credentialsValidator.bindUser=administrator@example.com
+ druid.auth.authenticator.ldap.credentialsValidator.bindPassword=adminpassword
+ druid.auth.authenticator.ldap.credentialsValidator.baseDn=dc=example,dc=com
+ druid.auth.authenticator.ldap.credentialsValidator.userSearch=(&(sAMAccountName=%s)(objectClass=user))
+ druid.auth.authenticator.ldap.credentialsValidator.userAttribute=sAMAccountName
+ druid.auth.authenticator.ldap.authorizerName=ldapauth
+ druid.escalator.type=basic
+ druid.escalator.internalClientUsername=internal@example.com
+ druid.escalator.internalClientPassword=internaluserpassword
+ druid.escalator.authorizerName=ldapauth
+ druid.auth.authorizers=["ldapauth"]
+ druid.auth.authorizer.ldapauth.type=basic
+ druid.auth.authorizer.ldapauth.initialAdminUser=internal@example.com
+ druid.auth.authorizer.ldapauth.initialAdminRole=admin
+ druid.auth.authorizer.ldapauth.roleProvider.type=ldap
```
+ Note the following:
- Notice that the LDAP user created in the previous step, `internal@example.com`, serves as the internal client user and the initial admin user.
+ - `bindUser`: A user for connecting to LDAP. This should be the same user you used to [test your LDAP search](#test-your-ldap-search).
+ - `userSearch`: Your LDAP search syntax.
+ - `userAttribute`: The user search attribute.
+ - `internal@example.com` is the LDAP user you created in step 1. In the example it serves as both the internal client user and the initial admin user.
-## Use LDAP groups to assign roles
+ > In the above example, the [Druid escalator](../development/extensions-core/druid-basic-security.md#escalator) and LDAP initial admin user are set to the same user - `internal@example.com`. If the escalator is set to a different user, you must follow steps 4 and 5 to create the group mapping and allocate initial roles before the rest of the cluster can function.
-You can map LDAP groups to a role in Druid. Members in the group get access to the permissions of the corresponding role.
+4. Save your group mapping to a JSON file. An example file `groupmap.json` looks like this:
+
+ ```
+ {
+ "name": "mygroupmap",
+ "groupPattern": "CN=mygroup,CN=Users,DC=example,DC=com",
+ "roles": [
+ "readRole"
+ ]
+ }
+ ```
+ In the example, the LDAP group `mygroup` maps to Druid role `readRole` and the name of the mapping is `mygroupmap`.
+5. Use the Druid API to create the group mapping and allocate initial roles according to your JSON file. The following example uses curl to create the mapping defined in `groupmap.json` for the LDAP group `mygroup`:
+
+ ```
+ curl -i -v -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroupmap
+ ```
+6. Check that the group mapping was created successfully. The following example request lists all group mappings:
-### Step 1: Create a role
+ ```
+ curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings
+ ```
-First create the role in Druid using the Druid REST API.
+## Map LDAP groups to Druid roles
-Creating a role involves submitting a POST request to the Coordinator process.
+Once you've completed the initial setup and mapping, you can map more LDAP groups to Druid roles. Members of an LDAP group get access to the permissions of the corresponding Druid role.
-The following REST APIs to create the role to read access for datasource, config, state.
+### Create a Druid role
-> As mentioned, the REST API calls need to address the Coordinator node. The examples used below use localhost as the Coordinator host and 8081 as the port. Adjust these settings according to your deployment.
+To create a Druid role, you can submit a POST request to the Coordinator process using the Druid REST API or you can use the Druid console.
-Call the following API to create role `readRole` .
+The examples below use `localhost` as the Coordinator host and `8081` as the port. Amend these properties according to the details of your deployment.
+
+Example request to create a role named `readRole`:
```
-curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole
+curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole
```
-Check that the role has been created successfully by entering the following:
+Check that Druid created the role successfully. The following example request lists all roles:
```
-curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles
+curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles
```
+### Add permissions to the Druid role
-### Step 2: Add permissions to a role
-
-You can now add one or more permission to the role. The following example adds read-only access to a `wikipedia` data source.
+Once you have a Druid role you can add permissions to it. The following example adds read-only access to a `wikipedia` data source.
Given the following JSON in a file named `perm.json`:
```
-[{ "resource": { "name": "wikipedia", "type": "DATASOURCE" }, "action": "READ" }
-,{ "resource": { "name": ".*", "type": "STATE" }, "action": "READ" },
-{ "resource": {"name": ".*", "type": "CONFIG"}, "action": "READ"}]
-```
-
-The following command associates the permissions in the JSON file with the role
-
-```
-curl -i -v -H "Content-Type: application/json" -u internal -X POST -d@perm.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole/permissions
+[
+ { "resource": { "name": "wikipedia", "type": "DATASOURCE" }, "action": "READ" },
+ { "resource": { "name": ".*", "type": "STATE" }, "action": "READ" },
+ { "resource": {"name": ".*", "type": "CONFIG"}, "action": "READ"}
+]
```
-Note that the STATE and CONFIG permissions in `perm.json` are needed to see the data source in the web console. If only querying permissions are needed, the READ action is sufficient:
+The following request associates the permissions in the JSON file with the `readRole` role:
```
-[{ "resource": { "name": "wikipedia", "type": "DATASOURCE" }, "action": "READ" }]
+curl -i -v -H "Content-Type: application/json" -u internal -X POST -d@perm.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/roles/readRole/permissions
```
-You can also provide the name in the form of regular expression. For example, to give access to all data sources starting with `wiki`, specify the name as `{ "name": "wiki.*", .....`.
+Druid users need the `STATE` and `CONFIG` permissions to view the data source in the Druid console. If you only want to assign querying permissions you can apply just the `READ` permission with the first line in the `perm.json` file.
+You can also provide the data source name in the form of a regular expression. For example, to give access to all data sources starting with `wiki`, you would specify the data source name as `{ "name": "wiki.*" }` .
-### Step 3: Create group Mapping
+### Create the group mapping
-The following shows an example of a group to role mapping. It assumes that a group named `group1` exists in the directory. Also assuming the following role mapping in a file named `groupmap.json`:
+You can now map an LDAP group to the Druid role. The following example request creates a mapping with name `mygroupmap`. It assumes that a group named `mygroup` exists in the directory.
```
{
- "name": "group1map",
- "groupPattern": "CN=group1,CN=Users,DC=example,DC=com",
- "roles": [
- "readRole"
- ]
+ "name": "mygroupmap",
+ "groupPattern": "CN=mygroup,CN=Users,DC=example,DC=com",
+ "roles": [
+ "readRole"
+ ]
}
```
-You can configure the mapping as follows:
+The following example request configures the mapping—the role mapping is in the file `groupmap.json`. See [Configure Druid for LDAP authentication](#configure-druid-for-ldap-authentication) for the contents of an example file.
```
-curl -i -v -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map
+curl -i -v -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroupmap
```
-To check whether the group mapping was created successfully, run the following command:
+To check whether the group mapping was created successfully, the following request lists all group mappings:
```
-curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings
+curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings
```
-To check the details of a specific group mapping, use the following:
+The following example request returns the details of the `mygroupmap` group:
```
-curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map
+curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroupmap
```
-To add additional roles to the group mapping, use the following API:
+The following example request adds the role `queryRole` to the `mygroupmap` mapping:
```
-curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1/roles/
+curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/mygroup/roles/queryrole
```
-In the next two steps you will be creating a user, and assigning previously created roles to it. These steps are only needed in the following cases:
-
- - Your LDAP server does not support the `memberOf` attribute, or
- - You want to configure a user with additional roles that are not mapped to the group(s) that the user is a member of
-
- If this is not the case for your scenario, you can skip these steps.
+### Add an LDAP user to Druid and assign a role
-### Step 4. Create a user
+You only need to complete this step if:
+- Your LDAP user doesn't belong to any of your LDAP groups, or
+- You want to configure a user with additional Druid roles that are not mapped to the LDAP groups that the user belongs to.
-Once LDAP is enabled, only user passwords are verified with LDAP. You add the LDAP user to Druid as follows:
+Example request to add the LDAP user `myuser` to Druid:
```
-curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/
+curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/myuser
```
-### Step 5. Assign the role to the user
-
-The following command shows how to assign a role to a user:
+Example request to assign the `myuser` user to the `queryRole` role:
```
-curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/users//roles/
+curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/myuser/roles/queryRole
```
-For more information about security and the basic security extension, see [Security Overview](security-overview.md).
+## Enable LDAP over TLS (LDAPS)
+
+Once you've configured LDAP authentication in Druid, you can optionally make LDAP traffic confidential and secure by using Transport Layer Security (TLS)—previously Secure Socket Layer(SSL)—technology.
+
+Configuring LDAPS establishes trust between Druid and the LDAP server.
+
+## Prerequisites
+
+Before you start to set up LDAPS in Druid, you must [configure Druid for LDAP authentication](#configure-druid-for-ldap-authentication). You also need:
+
+- A certificate issued by a public certificate authority (CA) or a self-signed certificate by an internal CA.
+- The root certificate for the CA that signed the certificate for the LDAP server. If you're using a common public CA, the certificate may already be in the Java truststore. Otherwise you need to import the certificate for the CA.
+
+## Configure Druid for LDAPS
+
+Complete the following steps to set up LDAPS for Druid. See [Configuration reference](../configuration/index.md) for the location of the configuration files.
+
+1. Import the CA certificate for your LDAP server or a self-signed certificate into the truststore location saved as `druid.client.https.trustStorePath` in your `common.runtime.properties` file.
+
+ ```
+ keytool -import -trustcacerts -keystore path/to/cacerts -storepass truststorepassword -alias aliasName -file path/to/certificate.cer
+ ```
+
+ Replace `path/to/cacerts` with the path to your truststore, `truststorepassword` with your truststore password, `aliasName` with an alias name for the keystore, and `path/to/certificate.cer` with the location and name of your certificate. For example:
+
+ ```
+ keytool -import -trustcacerts -keystore /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/security/cacerts -storepass mypassword -alias myAlias -file /etc/ssl/certs/my-certificate.cer
+ ```
+
+2. If the root certificate for the CA isn't already in the Java truststore, import it:
+
+ ```
+ keytool -importcert -keystore path/to/cacerts -storepass truststorepassword -alias aliasName -file path/to/certificate.cer
+ ```
+
+ Replace `path/to/cacerts` with the path to your truststore, `truststorepassword` with your truststore password, `aliasName` with an alias name for the keystore, and `path/to/certificate.cer` with the location and name of your certificate. For example:
+
+ ```
+ keytool -importcert -keystore /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/security/cacerts -storepass mypassword -alias myAlias -file /etc/ssl/certs/my-certificate.cer
+ ```
+
+3. In your `common.runtime.properties` file, add the following lines to the LDAP configuration section, substituting your own truststore path and password:
+
+ ```
+ druid.auth.basic.ssl.trustStorePath=/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/security/cacerts
+ druid.auth.basic.ssl.protocol=TLS
+ druid.auth.basic.ssl.trustStorePassword=xxxxxx
+ ```
+
+ See [Druid basic security](../development/extensions-core/druid-basic-security.md#properties-for-ldaps) for details about these properties.
+
+4. You can optionally configure additional LDAPS properties in the `common.runtime.properties` file. See [Druid basic security](../development/extensions-core/druid-basic-security.md#properties-for-ldaps) for more information.
+
+5. Restart Druid.
+
+
+## Troubleshooting tips
+
+The following are some ideas to help you troubleshoot issues with LDAP and LDAPS.
+
+### Check the coordinator logs
+
+If your LDAP connection isn't working, check the coordinator logs. See [Logging](../configuration/logging.md) for details.
+
+### Check the Druid escalator configuration
+
+If the coordinator is working but the rest of the cluster isn't, check the escalator configuration. See the [Configuration reference](../configuration/index.md) for details. You can also check other service logs to see why the services are unable to fetch authorization details from the coordinator.
+
+### Check your LDAP server response time
+
+If a user can log in to the Druid console but the landing page shows a 401 error, check your LDAP server response time. In a large organization with a high number of LDAP users, LDAP may be slow to respond, and this can result in a connection timeout.
diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md
index 8822f3fea73d..4e3c961dcb8d 100644
--- a/docs/operations/metrics.md
+++ b/docs/operations/metrics.md
@@ -230,8 +230,14 @@ Note: If the JVM does not support CPU time measurement for the current thread, `
|------|-----------|------------------------------------------------------------|------------|
|`task/run/time`|Milliseconds taken to run a task.| `dataSource`, `taskId`, `taskType`, `taskStatus`|Varies|
|`task/pending/time`|Milliseconds taken for a task to wait for running.| `dataSource`, `taskId`, `taskType`|Varies|
-|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`|< 1000 (subsecond)|
-|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`|Varies from subsecond to a few seconds, based on action type.|
+|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`, `taskActionType`|< 1000 (subsecond)|
+|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`, `taskActionType`|Varies from subsecond to a few seconds, based on action type.|
+|`task/action/success/count`|Number of task actions that were executed successfully during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`|Varies|
+|`task/action/failed/count`|Number of task actions that failed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`|Varies|
+|`task/action/batch/queueTime`|Milliseconds spent by a batch of task actions in queue. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on the `batchAllocationWaitTime` and number of batches in queue.|
+|`task/action/batch/runTime`|Milliseconds taken to execute a batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies from subsecond to a few seconds, based on action type and batch size.|
+|`task/action/batch/size`|Number of task actions in a batch that was executed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on number of concurrent task actions.|
+|`task/action/batch/attempts`|Number of execution attempts for a single batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|1 if there are no failures or retries.|
|`segment/added/bytes`|Size in bytes of new segments created.| `dataSource`, `taskId`, `taskType`, `interval`|Varies|
|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.| `dataSource`, `taskId`, `taskType`, `interval`|Varies|
|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.| `dataSource`, `taskId`, `taskType`, `interval`|Varies|
diff --git a/docs/operations/python.md b/docs/operations/python.md
new file mode 100644
index 000000000000..9f5b9c34909e
--- /dev/null
+++ b/docs/operations/python.md
@@ -0,0 +1,49 @@
+---
+id: python
+title: "Python Installation"
+---
+
+
+
+Apache Druid startup script requires Python2 or Python3 interpreter.
+Since Python2 is deprecated, this document has instructions to install Python3 interpreter.
+
+## Python3 interpreter installation instructions
+
+### Linux
+
+#### Debian or Ubuntu
+ - `sudo apt update`
+ - `sudo apt install -y python3-pip`
+#### RHEL
+ - `sudo yum install -y epel-release`
+ - `sudo yum install -y python3-pip`
+
+### MacOS
+
+#### Install with Homebrew
+Refer [Installing Python 3 on Mac OS X](https://docs.python-guide.org/starting/install3/osx/)
+
+#### Install the official Python release
+* Browse to the [Python Downloads Page](https://www.python.org/downloads/) and download the latest version (3.x.x)
+
+Verify if Python3 is installed by issuing `python3 --version` command.
+
+
diff --git a/docs/operations/security-overview.md b/docs/operations/security-overview.md
index 028cff04f55a..3fa80e24adc2 100644
--- a/docs/operations/security-overview.md
+++ b/docs/operations/security-overview.md
@@ -250,80 +250,7 @@ The following steps walk through a sample setup procedure:
## Configuring an LDAP authenticator
-As an alternative to using the basic metadata authenticator, you can use LDAP to authenticate users. The following steps provide an overview of the setup procedure. For more information on these settings, see [Properties for LDAP user authentication](../development/extensions-core/druid-basic-security.md#properties-for-ldap-user-authentication).
-
-1. In `common.runtime.properties`, add LDAP to the authenticator chain in the order in which you want requests to be evaluated. For example:
- ```
- # Druid basic security
- druid.auth.authenticatorChain=["ldap", "MyBasicMetadataAuthenticator"]
- ```
-
-2. Configure LDAP settings in `common.runtime.properties` as appropriate for your LDAP scheme and system. For example:
- ```
- druid.auth.authenticator.ldap.type=basic
- druid.auth.authenticator.ldap.enableCacheNotifications=true
- druid.auth.authenticator.ldap.credentialsValidator.type=ldap
- druid.auth.authenticator.ldap.credentialsValidator.url=ldap://ad_host:389
- druid.auth.authenticator.ldap.credentialsValidator.bindUser=ad_admin_user
- druid.auth.authenticator.ldap.credentialsValidator.bindPassword=ad_admin_password
- druid.auth.authenticator.ldap.credentialsValidator.baseDn=dc=example,dc=com
- druid.auth.authenticator.ldap.credentialsValidator.userSearch=(&(sAMAccountName=%s)(objectClass=user))
- druid.auth.authenticator.ldap.credentialsValidator.userAttribute=sAMAccountName
- druid.auth.authenticator.ldap.authorizerName=ldapauth
- druid.escalator.type=basic
- druid.escalator.internalClientUsername=ad_interal_user
- druid.escalator.internalClientPassword=Welcome123
- druid.escalator.authorizerName=ldapauth
- druid.auth.authorizers=["ldapauth"]
- druid.auth.authorizer.ldapauth.type=basic
- druid.auth.authorizer.ldapauth.initialAdminUser=
- druid.auth.authorizer.ldapauth.initialAdminRole=admin
- druid.auth.authorizer.ldapauth.roleProvider.type=ldap
- ```
-
-3. Use the Druid API to create the group mapping and allocate initial roles. For example, using curl and given a group named `group1` in the directory, run:
- ```
- curl -i -v -H "Content-Type: application/json" -u internal -X POST -d @groupmap.json http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map
- ```
- The `groupmap.json` file contents would be something like:
- ```
- {
- "name": "group1map",
- "groupPattern": "CN=group1,CN=Users,DC=example,DC=com",
- "roles": [
- "readRole"
- ]
- }
- ```
-4. Check if the group mapping is created successfully by executing the following API. This lists all group mappings.
- ```
- curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings
- ```
-
- Alternatively, to check the details of a specific group mapping, use the following API:
- ```
- curl -i -v -H "Content-Type: application/json" -u internal -X GET http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1map
- ```
-
-5. To add additional roles to the group mapping, use the following API:
- ```
- curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/groupMappings/group1/roles/
- ```
-
-6. Add the LDAP user to Druid. To add a user, use the following authentication API:
- ```
- curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authentication/db/ldap/users/
- ```
-
-7. Use the following command to assign the role to a user:
- ```
- curl -i -v -H "Content-Type: application/json" -u internal -X POST http://localhost:8081/druid-ext/basic-security/authorization/db/ldapauth/users//roles/
- ```
-
-
-
-Congratulations, you have configured permissions for user-assigned roles in Druid!
-
+As an alternative to using the basic metadata authenticator, you can use LDAP to authenticate users. See [Configure LDAP authentication](./auth-ldap.md) for information on configuring Druid for LDAP and LDAPS.
## Druid security trust model
Within Druid's trust model there users can have different authorization levels:
diff --git a/docs/operations/single-server.md b/docs/operations/single-server.md
index 35413952e222..48459a286042 100644
--- a/docs/operations/single-server.md
+++ b/docs/operations/single-server.md
@@ -23,14 +23,17 @@ title: "Single server deployment"
-->
-Druid includes a set of reference configurations and launch scripts for single-machine deployments:
-
-- `nano-quickstart`
-- `micro-quickstart`
-- `small`
-- `medium`
-- `large`
-- `xlarge`
+Druid includes a set of reference configurations and launch scripts for single-machine deployments.
+These configuration bundles are located in `conf/druid/single-server/`.
+
+The `auto` configuration sizes runtime parameters based on available processors and memory. Other configurations include hard-coded runtime parameters for various server sizes. Most users should stick with `auto`. Refer below [Druid auto start](#druid-auto-start)
+- `auto` (run script: `bin/start-druid`)
+- `nano-quickstart` (run script: `bin/start-nano-quickstart`)
+- `micro-quickstart` (run script: `bin/start-micro-quickstart`)
+- `small` (run script: `bin/start-single-server-small`)
+- `medium` (run script: `bin/start-single-server-medium`)
+- `large` (run script: `bin/start-single-server-large`)
+- `xlarge` (run script: `bin/start-single-server-xlarge`)
The `micro-quickstart` is sized for small machines like laptops and is intended for quick evaluation use-cases.
@@ -44,6 +47,18 @@ The example configurations run the Druid Coordinator and Overlord together in a
While example configurations are provided for very large single machines, at higher scales we recommend running Druid in a [clustered deployment](../tutorials/cluster.md), for fault-tolerance and reduced resource contention.
+## Druid auto start
+
+Druid includes a launch script, `bin/start-druid` that automatically sets various memory-related parameters based on available processors and memory. It accepts optional arguments such as list of services, total memory and a config directory to override default JVM arguments and service-specific runtime properties.
+
+`start-druid` is a generic launch script capable of starting any set of Druid services on a server.
+It accepts optional arguments such as list of services, total memory and a config directory to override default JVM arguments and service-specific runtime properties.
+Druid services will use all processors and up to 80% memory on the system.
+For details about possible arguments, run `bin/start-druid --help`.
+
+The corresponding launch scripts (e.g. `start-micro-quickstart`) are now deprecated.
+
+
## Single server reference configurations
### Nano-Quickstart: 1 CPU, 4GiB RAM
@@ -74,5 +89,4 @@ While example configurations are provided for very large single machines, at hig
### X-Large: 64 CPU, 512GiB RAM (~i3.16xlarge)
- Launch command: `bin/start-xlarge`
-- Configuration directory: `conf/druid/single-server/xlarge`
-
+- Configuration directory: `conf/druid/single-server/xlarge`
\ No newline at end of file
diff --git a/docs/querying/lookups.md b/docs/querying/lookups.md
index 57b406afb199..860a3ed2e277 100644
--- a/docs/querying/lookups.md
+++ b/docs/querying/lookups.md
@@ -115,8 +115,8 @@ will not detect this automatically.
Dynamic Configuration
---------------------
-> Dynamic lookup configuration is an [experimental](../development/experimental.md) feature. Static
-> configuration is no longer supported.
+> Static configuration is no longer supported.
+
The following documents the behavior of the cluster-wide config which is accessible through the Coordinator.
The configuration is propagated through the concept of "tier" of servers.
A "tier" is defined as a group of services which should receive a set of lookups.
diff --git a/docs/querying/nested-columns.md b/docs/querying/nested-columns.md
index e8dc628c8fc5..77af91ddff89 100644
--- a/docs/querying/nested-columns.md
+++ b/docs/querying/nested-columns.md
@@ -23,17 +23,17 @@ sidebar_label: Nested columns
~ under the License.
-->
-> Nested columns is an experimental feature available starting in Apache Druid 24.0. Like most experimental features, functionality documented on this page is subject to change in future releases. However, the COMPLEX column type includes versioning to provide backward compatible support in future releases. We strongly encourage you to experiment with nested columns in your development environment to evaluate that they meet your use case. If so, you can use them in production scenarios. Review the release notes and this page to stay up to date with changes.
-
Apache Druid supports directly storing nested data structures in `COMPLEX` columns. `COMPLEX` columns store a copy of the structured data in JSON format and specialized internal columns and indexes for nested literal values—STRING, LONG, and DOUBLE types. An optimized [virtual column](./virtual-columns.md#nested-field-virtual-column) allows Druid to read and filter these values at speeds consistent with standard Druid LONG, DOUBLE, and STRING columns.
Druid [SQL JSON functions](./sql-json-functions.md) allow you to extract, transform, and create `COMPLEX` values in SQL queries, using the specialized virtual columns where appropriate. You can use the [JSON nested columns functions](../misc/math-expr.md#json-functions) in [native queries](./querying.md) using [expression virtual columns](./virtual-columns.md#expression-virtual-column), and in native ingestion with a [`transformSpec`](../ingestion/ingestion-spec.md#transformspec).
You can use the JSON functions in INSERT and REPLACE statements in SQL-based ingestion, or in a `transformSpec` in native ingestion as an alternative to using a [`flattenSpec`](../ingestion/data-formats.md#flattenspec) object to "flatten" nested data for ingestion.
+Druid supports directly ingesting nested data with the following formats: JSON, Parquet, Avro, ORC.
+
## Example nested data
-The examples in this topic use the data in [`nested_example_data.json`](https://static.imply.io/data/nested_example_data.json). The file contains a simple facsimile of an order tracking and shipping table.
+The examples in this topic use the JSON data in [`nested_example_data.json`](https://static.imply.io/data/nested_example_data.json). The file contains a simple facsimile of an order tracking and shipping table.
When pretty-printed, a sample row in `nested_example_data` looks like this:
@@ -63,7 +63,7 @@ When pretty-printed, a sample row in `nested_example_data` looks like this:
## Native batch ingestion
-For native batch ingestion, you can use the [JSON nested columns functions](./sql-json-functions.md) to extract nested data as an alternative to using the [`flattenSpec`](../ingestion/data-formats.md#flattenspec) input format.
+For native batch ingestion, you can use the [SQL JSON functions](./sql-json-functions.md) to extract nested data as an alternative to using the [`flattenSpec`](../ingestion/data-formats.md#flattenspec) input format.
To configure a dimension as a nested data type, specify the `json` type for the dimension in the `dimensions` list in the `dimensionsSpec` property of your ingestion spec.
@@ -124,7 +124,7 @@ For example, the following ingestion spec instructs Druid to ingest `shipTo` and
### Transform data during batch ingestion
-You can use the [JSON nested columns functions](./sql-json-functions.md) to transform JSON data and reference the transformed data in your ingestion spec.
+You can use the [SQL JSON functions](./sql-json-functions.md) to transform nested data and reference the transformed data in your ingestion spec.
To do this, define the output name and expression in the `transforms` list in the `transformSpec` object of your ingestion spec.
@@ -192,7 +192,7 @@ For example, the following ingestion spec extracts `firstName`, `lastName` and `
## SQL-based ingestion
-To ingest nested data using multi-stage query architecture, specify `COMPLEX` as the value for `type` when you define the row signature—`shipTo` and `details` in the following example ingestion spec:
+To ingest nested data using SQL-based ingestion, specify `COMPLEX` as the value for `type` when you define the row signature—`shipTo` and `details` in the following example ingestion spec:

@@ -297,7 +297,7 @@ The [Kafka tutorial](../tutorials/tutorial-kafka.md) guides you through the step
### Transform data during SQL-based ingestion
-You can use the [JSON nested columns functions](./sql-json-functions.md) to transform JSON data in your ingestion query.
+You can use the [SQL JSON functions](./sql-json-functions.md) to transform nested data in your ingestion query.
For example, the following ingestion query is the SQL-based version of the [previous batch example](#transform-data-during-batch-ingestion)—it extracts `firstName`, `lastName`, and `address` from `shipTo` and creates a composite JSON object containing `product`, `details`, and `department`.
@@ -326,7 +326,7 @@ PARTITIONED BY ALL
## Ingest a JSON string as COMPLEX
-If your source data uses a string representation of your JSON column, you can still ingest the data as `COMPLEX` as follows:
+If your source data contains serialized JSON strings, you can ingest the data as `COMPLEX` as follows:
- During native batch ingestion, call the `parse_json` function in a `transform` object in the `transformSpec`.
- During SQL-based ingestion, use the PARSE_JSON keyword within your SELECT statement to transform the string values to JSON.
- If you are concerned that your data may not contain valid JSON, you can use `try_parse_json` for native batch or `TRY_PARSE_JSON` for SQL-based ingestion. For cases where the column does not contain valid JSON, Druid inserts a null value.
@@ -563,7 +563,7 @@ In addition to `JSON_VALUE`, Druid offers a number of operators that focus on tr
- `PARSE_JSON`
- `TO_JSON_STRING`
-These functions are primarily intended for use with the multi-stage query architecture to transform data during insert operations, but they also work in traditional Druid SQL queries. Because most of these functions output JSON objects, they have the same limitations when used in traditional Druid queries as interacting with the JSON objects directly.
+These functions are primarily intended for use with SQL-based ingestion to transform data during insert operations, but they also work in traditional Druid SQL queries. Because most of these functions output JSON objects, they have the same limitations when used in traditional Druid queries as interacting with the JSON objects directly.
#### Example query: Return results in a JSON object
@@ -663,7 +663,7 @@ Before you start using the nested columns feature, consider the following known
- Directly using `COMPLEX` columns and expressions is not well integrated into the Druid query engine. It can result in errors or undefined behavior when grouping and filtering, and when you use `COMPLEX` objects as inputs to aggregators. As a workaround, consider using `TO_JSON_STRING` to coerce the values to strings before you perform these operations.
- Directly using array-typed outputs from `JSON_KEYS` and `JSON_PATHS` is moderately supported by the Druid query engine. You can group on these outputs, and there are a number of array expressions that can operate on these values, such as `ARRAY_CONCAT_AGG`. However, some operations are not well defined for use outside array-specific functions, such as filtering using `=` or `IS NULL`.
- Input validation for JSON SQL operators is currently incomplete, which sometimes results in undefined behavior or unhelpful error messages.
-- Ingesting JSON columns with a very complex nested structure is potentially an expensive operation and may require you to tune ingestion tasks and/or cluster parameters to account for increased memory usage or overall task run time. When you tune your ingestion configuration, treat each nested literal field inside a JSON object as a flattened top-level Druid column.
+- Ingesting data with a very complex nested structure is potentially an expensive operation and may require you to tune ingestion tasks and/or cluster parameters to account for increased memory usage or overall task run time. When you tune your ingestion configuration, treat each nested literal field inside an object as a flattened top-level Druid column.
## Further reading
diff --git a/docs/querying/sql-jdbc.md b/docs/querying/sql-jdbc.md
index 0f041c6fad38..a558637bc110 100644
--- a/docs/querying/sql-jdbc.md
+++ b/docs/querying/sql-jdbc.md
@@ -29,6 +29,12 @@ sidebar_label: "JDBC driver API"
You can make [Druid SQL](./sql.md) queries using the [Avatica JDBC driver](https://calcite.apache.org/avatica/downloads/). We recommend using Avatica JDBC driver version 1.17.0 or later. Note that as of the time of this writing, Avatica 1.17.0, the latest version, does not support passing connection string parameters from the URL to Druid, so you must pass them using a `Properties` object. Once you've downloaded the Avatica client jar, add it to your classpath and use the connect string `jdbc:avatica:remote:url=http://BROKER:8082/druid/v2/sql/avatica/`.
+When using the JDBC connector for the [examples](#examples) or in general, it's helpful to understand the parts of the connect string stored in the `url` variable:
+
+ - `jdbc:avatica:remote:url=` is prepended to the hostname and port.
+ - The hostname and port number for your Druid deployment depends on whether you want to connect to the Router or a specific Broker. For more information, see [Connection stickiness](#connection-stickiness). In the case of the quickstart deployment, the hostname and port are `http://localhost:8888`, which connects to the Router running on your local machine.
+ - The SQL endpoint in Druid for the Avatica driver is `/druid/v2/sql/avatica/`.
+
Example code:
```java
@@ -51,6 +57,8 @@ try (Connection connection = DriverManager.getConnection(url, connectionProperti
}
```
+For a runnable example that includes a query that you might run, see [Examples](#examples).
+
It is also possible to use a protocol buffers JDBC connection with Druid, this offer reduced bloat and potential performance
improvements for larger result sets. To use it apply the following connection url instead, everything else remains the same
```
@@ -60,12 +68,12 @@ String url = "jdbc:avatica:remote:url=http://localhost:8082/druid/v2/sql/avatica
> The protobuf endpoint is also known to work with the official [Golang Avatica driver](https://github.com/apache/calcite-avatica-go)
Table metadata is available over JDBC using `connection.getMetaData()` or by querying the
-["INFORMATION_SCHEMA" tables](sql-metadata-tables.md).
+["INFORMATION_SCHEMA" tables](sql-metadata-tables.md). For an example of this, see [Get the metadata for a datasource](#get-the-metadata-for-a-datasource).
## Connection stickiness
Druid's JDBC server does not share connection state between Brokers. This means that if you're using JDBC and have
-multiple Druid Brokers, you should either connect to a specific Broker, or use a load balancer with sticky sessions
+multiple Druid Brokers, you should either connect to a specific Broker or use a load balancer with sticky sessions
enabled. The Druid Router process provides connection stickiness when balancing JDBC requests, and can be used to achieve
the necessary stickiness even with a normal non-sticky load balancer. Please see the
[Router](../design/router.md) documentation for more details.
@@ -82,3 +90,112 @@ statement.setString(1, "abc");
statement.setString(2, "def");
final ResultSet resultSet = statement.executeQuery();
```
+
+## Examples
+
+
+
+The following section contains two complete samples that use the JDBC connector:
+
+- [Get the metadata for a datasource](#get-the-metadata-for-a-datasource) shows you how to query the `INFORMATION_SCHEMA` to get metadata like column names.
+- [Query data](#query-data) runs a select query against the datasource.
+
+You can try out these examples after verifying that you meet the [prerequisites](#prerequisites).
+
+For more information about the connection options, see [Client Reference](https://calcite.apache.org/avatica/docs/client_reference.html).
+
+### Prerequisites
+
+Make sure you meet the following requirements before trying these examples:
+
+- A supported Java version, such as Java 8
+
+- [Avatica JDBC driver](https://calcite.apache.org/avatica/downloads/). You can add the JAR to your `CLASSPATH` directly or manage it externally, such as through Maven and a `pom.xml` file.
+
+- An available Druid instance. You can use the `micro-quickstart` configuration described in [Quickstart (local)](../tutorials/index.md). The examples assume that you are using the quickstart, so no authentication or authorization is expected unless explicitly mentioned.
+
+- The example `wikipedia` datasource from the quickstart is loaded on your Druid instance. If you have a different datasource loaded, you can still try these examples. You'll have to update the table name and column names to match your datasource.
+
+### Get the metadata for a datasource
+
+Metadata, such as column names, is available either through the [`INFORMATION_SCHEMA`](../querying/sql-metadata-tables.md) table or through `connect.getMetaData()`. The following example uses the `INFORMATION_SCHEMA` table to retrieve and print the list of column names for the `wikipedia` datasource that you loaded during a previous tutorial.
+
+```java
+import java.sql.*;
+import java.util.Properties;
+
+public class JdbcListColumns {
+
+ public static void main(String args[]) throws SQLException
+ {
+ // Connect to /druid/v2/sql/avatica/ on your Router.
+ // You can connect to a Broker but must configure connection stickiness if you do.
+ String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/";
+
+ String query = "SELECT COLUMN_NAME,* FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME = 'wikipedia' and TABLE_SCHEMA='druid'";
+ // Set any connection context parameters you need here
+ // Or leave empty for default behavior.
+ Properties connectionProperties = new Properties();
+
+ try (Connection connection = DriverManager.getConnection(url, connectionProperties)) {
+ try (
+ final Statement statement = connection.createStatement();
+ final ResultSet rs = statement.executeQuery(query)
+ ) {
+ while (rs.next()) {
+ String columnName = rs.getString("COLUMN_NAME");
+ System.out.println(columnName);
+ }
+ }
+ } catch (SQLException e) {
+ throw new RuntimeException(e);
+ }
+
+ }
+}
+```
+
+### Query data
+
+Now that you know what columns are available, you can start querying the data. The following example queries the datasource named `wikipedia` for the timestamps and comments from Japan. It also sets the [query context parameter](../querying/sql-query-context.md) `sqlTimeZone` . Optionally, you can also parameterize queries by using [dynamic parameters](#dynamic-parameters).
+
+```java
+import java.sql.*;
+import java.util.Properties;
+
+public class JdbcCountryAndTime {
+
+ public static void main(String args[]) throws SQLException
+ {
+ // Connect to /druid/v2/sql/avatica/ on your Router.
+ // You can connect to a Broker but must configure connection stickiness if you do.
+ String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/";
+
+ //The query you want to run.
+ String query = "SELECT __time, isRobot, countryName, comment FROM wikipedia WHERE countryName='Japan'";
+ // Set any connection context parameters you need here
+ // Or leave empty for default behavior.
+ Properties connectionProperties = new Properties();
+ connectionProperties.setProperty("sqlTimeZone", "America/Los_Angeles");
+
+ try (Connection connection = DriverManager.getConnection(url, connectionProperties)) {
+ try (
+ final Statement statement = connection.createStatement();
+ final ResultSet rs = statement.executeQuery(query)
+ ) {
+ while (rs.next()) {
+ Timestamp timeStamp = rs.getTimestamp("__time");
+ String comment = rs.getString("comment");
+ System.out.println(timeStamp);
+ System.out.println(comment);
+ }
+ }
+ } catch (SQLException e) {
+ throw new RuntimeException(e);
+ }
+
+ }
+}
+`````
+
+
diff --git a/docs/querying/sql-translation.md b/docs/querying/sql-translation.md
index d26ae6f0d276..370ad002c1cd 100644
--- a/docs/querying/sql-translation.md
+++ b/docs/querying/sql-translation.md
@@ -24,7 +24,7 @@ sidebar_label: "SQL query translation"
-->
> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+> This document describes the Druid SQL language.
Druid uses [Apache Calcite](https://calcite.apache.org/) to parse and plan SQL queries.
Druid translates SQL statements into its [native JSON-based query language](querying.md).
@@ -34,8 +34,8 @@ This topic includes best practices and tools to help you achieve good performanc
## Best practices
-Consider this (non-exhaustive) list of things to look out for when looking into the performance implications of
-how your SQL queries are translated to native queries.
+Consider the following non-exhaustive list of best practices when looking into performance implications of
+translating Druid SQL queries to native queries.
1. If you wrote a filter on the primary time column `__time`, make sure it is being correctly translated to an
`"intervals"` filter, as described in the [Time filters](#time-filters) section below. If not, you may need to change
@@ -241,10 +241,9 @@ enabling logging and running this query, we can see that it actually runs as the
Druid SQL uses four different native query types.
-- [Scan](scan-query.md) is used for queries that do not aggregate (no GROUP BY, no DISTINCT).
+- [Scan](scan-query.md) is used for queries that do not aggregate—no GROUP BY, no DISTINCT.
-- [Timeseries](timeseriesquery.md) is used for queries that GROUP BY `FLOOR(__time TO unit)` or `TIME_FLOOR(__time,
-period)`, have no other grouping expressions, no HAVING or LIMIT clauses, no nesting, and either no ORDER BY, or an
+- [Timeseries](timeseriesquery.md) is used for queries that GROUP BY `FLOOR(__time TO unit)` or `TIME_FLOOR(__time, period)`, have no other grouping expressions, no HAVING clause, no nesting, and either no ORDER BY, or an
ORDER BY that orders by same expression as present in GROUP BY. It also uses Timeseries for "grand total" queries that
have aggregation functions but no GROUP BY. This query type takes advantage of the fact that Druid segments are sorted
by time.
diff --git a/docs/tutorials/cluster.md b/docs/tutorials/cluster.md
index b61953c2f427..5ee68e2cd13e 100644
--- a/docs/tutorials/cluster.md
+++ b/docs/tutorials/cluster.md
@@ -130,7 +130,10 @@ The [basic cluster tuning guide](../operations/basic-cluster-tuning.md) has info
## Select OS
-We recommend running your favorite Linux distribution. You will also need [Java 8 or 11](../operations/java.md).
+We recommend running your favorite Linux distribution. You will also need
+
+* [Java 8 or 11](../operations/java.md).
+* [Python2 or Python3](../operations/python.md)
> If needed, you can specify where to find Java using the environment variables
> `DRUID_JAVA_HOME` or `JAVA_HOME`. For more details run the `bin/verify-java` script.
diff --git a/docs/tutorials/index.md b/docs/tutorials/index.md
index 7d3d2ab68ec1..d854a691e6b9 100644
--- a/docs/tutorials/index.md
+++ b/docs/tutorials/index.md
@@ -22,8 +22,7 @@ title: "Quickstart (local)"
~ under the License.
-->
-
-This quickstart gets you started with Apache Druid using the [`micro-quickstart`](../operations/single-server.md#micro-quickstart-4-cpu-16gib-ram) configuration, and introduces you to Druid ingestion and query features.
+This quickstart gets you started with Apache Druid and introduces you to Druid ingestion and query features. For this tutorial, we recommend a machine with at least 6 GB of RAM.
In this quickstart, you'll do the following:
- install Druid
@@ -37,15 +36,16 @@ Druid supports a variety of ingestion options. Once you're done with this tutori
You can follow these steps on a relatively modest machine, such as a workstation or virtual server with 16 GiB of RAM.
-Druid comes equipped with several [startup configuration profiles](../operations/single-server.md) for a
-range of machine sizes. These range from `nano` (1 CPU, 4GiB RAM) to `x-large` (64 CPU, 512GiB RAM). For more
-information, see [Single server deployment](../operations/single-server.md). For information on deploying Druid services
-across clustered machines, see [Clustered deployment](./cluster.md).
+Druid comes equipped with launch scripts that can be used to start all processes on a single server. Here, we will use [`auto`](../operations/single-server.md#druid-auto-start), which automatically sets various runtime properties based on available processors and memory.
+
+In addition, Druid includes several [bundled non-automatic profiles](../operations/single-server.md) for a range of machine sizes. These range from nano (1 CPU, 4GiB RAM) to x-large (64 CPU, 512GiB RAM).
+We won't use those here, but for more information, see [Single server deployment](../operations/single-server.md). For additional information on deploying Druid services across clustered machines, see [Clustered deployment](./cluster.md).
The software requirements for the installation machine are:
* Linux, Mac OS X, or other Unix-like OS. (Windows is not supported.)
* Java 8u92+ or Java 11.
+* [Python2 or Python3](../operations/python.md)
> Druid relies on the environment variables `JAVA_HOME` or `DRUID_JAVA_HOME` to find Java on the machine. You can set
`DRUID_JAVA_HOME` if there is more than one instance of Java. To verify Java requirements for your environment, run the
@@ -72,30 +72,31 @@ The distribution directory contains `LICENSE` and `NOTICE` files and subdirector
## Start up Druid services
-Start up Druid services using the `micro-quickstart` single-machine configuration.
+Start up Druid services using the `auto` single-machine configuration.
This configuration includes default settings that are appropriate for this tutorial, such as loading the `druid-multi-stage-query` extension by default so that you can use the MSQ task engine.
-You can view that setting and others in the configuration files in the `conf/druid/single-server/micro-quickstart/`.
+You can view that setting and others in the configuration files in the `conf/druid/auto`.
From the apache-druid-{{DRUIDVERSION}} package root, run the following command:
```bash
-./bin/start-micro-quickstart
+./bin/start-druid
```
This brings up instances of ZooKeeper and the Druid services:
```bash
-$ ./bin/start-micro-quickstart
-[Thu Sep 8 18:30:00 2022] Starting Apache Druid.
-[Thu Sep 8 18:30:00 2022] Open http://localhost:8888/ in your browser to access the web console.
-[Thu Sep 8 18:30:00 2022] Or, if you have enabled TLS, use https on port 9088.
-[Thu Sep 8 18:30:00 2022] Running command[zk], logging to[/apache-druid-{{DRUIDVERSION}}/var/sv/zk.log]: bin/run-zk conf
-[Thu Sep 8 18:30:00 2022] Running command[coordinator-overlord], logging to[/apache-druid-{{DRUIDVERSION}}/var/sv/coordinator-overlord.log]: bin/run-druid coordinator-overlord conf/druid/single-server/micro-quickstart
-[Thu Sep 8 18:30:00 2022] Running command[broker], logging to[/apache-druid-{{DRUIDVERSION}}/var/sv/broker.log]: bin/run-druid broker conf/druid/single-server/micro-quickstart
-[Thu Sep 8 18:30:00 2022] Running command[router], logging to[/apache-druid-{{DRUIDVERSION}}/var/sv/router.log]: bin/run-druid router conf/druid/single-server/micro-quickstart
-[Thu Sep 8 18:30:00 2022] Running command[historical], logging to[/apache-druid-{{DRUIDVERSION}}/var/sv/historical.log]: bin/run-druid historical conf/druid/single-server/micro-quickstart
-[Thu Sep 8 18:30:00 2022] Running command[middleManager], logging to[/apache-druid-{{DRUIDVERSION}}/var/sv/middleManager.log]: bin/run-druid middleManager conf/druid/single-server/micro-quickstart
+$ ./bin/start-druid
+[Tue Nov 29 16:31:06 2022] Starting Apache Druid.
+[Tue Nov 29 16:31:06 2022] Open http://localhost:8888/ in your browser to access the web console.
+[Tue Nov 29 16:31:06 2022] Or, if you have enabled TLS, use https on port 9088.
+[Tue Nov 29 16:31:06 2022] Starting services with log directory [/apache-druid-{{DRUIDVERSION}}/log].
+[Tue Nov 29 16:31:06 2022] Running command[zk]: bin/run-zk conf
+[Tue Nov 29 16:31:06 2022] Running command[broker]: bin/run-druid broker /apache-druid-{{DRUIDVERSION}}/conf/druid/single-server/quickstart '-Xms1187m -Xmx1187m -XX:MaxDirectMemorySize=791m'
+[Tue Nov 29 16:31:06 2022] Running command[router]: bin/run-druid router /apache-druid-{{DRUIDVERSION}}/conf/druid/single-server/quickstart '-Xms128m -Xmx128m'
+[Tue Nov 29 16:31:06 2022] Running command[coordinator-overlord]: bin/run-druid coordinator-overlord /apache-druid-{{DRUIDVERSION}}/conf/druid/single-server/quickstart '-Xms1290m -Xmx1290m'
+[Tue Nov 29 16:31:06 2022] Running command[historical]: bin/run-druid historical /apache-druid-{{DRUIDVERSION}}/conf/druid/single-server/quickstart '-Xms1376m -Xmx1376m -XX:MaxDirectMemorySize=2064m'
+[Tue Nov 29 16:31:06 2022] Running command[middleManager]: bin/run-druid middleManager /apache-druid-{{DRUIDVERSION}}/conf/druid/single-server/quickstart '-Xms64m -Xmx64m' '-Ddruid.worker.capacity=2 -Ddruid.indexer.runner.javaOptsArray=["-server","-Duser.timezone=UTC","-Dfile.encoding=UTF-8","-XX:+ExitOnOutOfMemoryError","-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager","-Xms256m","-Xmx256m","-XX:MaxDirectMemorySize=256m"]'
```
All persistent state, such as the cluster metadata store and segments for the services, are kept in the `var` directory under
@@ -103,7 +104,7 @@ the Druid root directory, apache-druid-{{DRUIDVERSION}}. Each service writes to
At any time, you can revert Druid to its original, post-installation state by deleting the entire `var` directory. You may want to do this, for example, between Druid tutorials or after experimentation, to start with a fresh instance.
-To stop Druid at any time, use CTRL+C in the terminal. This exits the `bin/start-micro-quickstart` script and terminates all Druid processes.
+To stop Druid at any time, use CTRL+C in the terminal. This exits the `bin/start-druid` script and terminates all Druid processes.
## Open the web console
@@ -222,4 +223,4 @@ See the following topics for more information:
* [Tutorial: Load stream data from Apache Kafka](./tutorial-kafka.md) to load streaming data from a Kafka topic.
* [Extensions](../development/extensions.md) for details on Druid extensions.
-Remember that after stopping Druid services, you can start clean next time by deleting the `var` directory from the Druid root directory and running the `bin/start-micro-quickstart` script again. You may want to do this before using other data ingestion tutorials, since they use the same Wikipedia datasource.
+Remember that after stopping Druid services, you can start clean next time by deleting the `var` directory from the Druid root directory and running the `bin/start-druid` script again. You may want to do this before using other data ingestion tutorials, since they use the same Wikipedia datasource.
diff --git a/docs/tutorials/tutorial-batch-hadoop.md b/docs/tutorials/tutorial-batch-hadoop.md
index 47cd2d6bcbe5..234e8426b064 100644
--- a/docs/tutorials/tutorial-batch-hadoop.md
+++ b/docs/tutorials/tutorial-batch-hadoop.md
@@ -28,7 +28,7 @@ This tutorial shows you how to load data files into Apache Druid using a remote
For this tutorial, we'll assume that you've already completed the previous
[batch ingestion tutorial](tutorial-batch.md) using Druid's native batch ingestion system and are using the
-`micro-quickstart` single-machine configuration as described in the [quickstart](index.md).
+`auto` single-machine configuration as described in the [quickstart](../operations/single-server.md#druid-auto-start).
## Install Docker
@@ -156,7 +156,7 @@ cp /tmp/shared/hadoop_xml/*.xml {PATH_TO_DRUID}/conf/druid/single-server/micro-q
### Update Druid segment and log storage
-In your favorite text editor, open `conf/druid/single-server/micro-quickstart/_common/common.runtime.properties`, and make the following edits:
+In your favorite text editor, open `conf/druid/auto/_common/common.runtime.properties`, and make the following edits:
#### Disable local deep storage and enable HDFS deep storage
@@ -196,7 +196,7 @@ druid.indexer.logs.directory=/druid/indexing-logs
Once the Hadoop .xml files have been copied to the Druid cluster and the segment/log storage configuration has been updated to use HDFS, the Druid cluster needs to be restarted for the new configurations to take effect.
-If the cluster is still running, CTRL-C to terminate the `bin/start-micro-quickstart` script, and re-run it to bring the Druid services back up.
+If the cluster is still running, CTRL-C to terminate the `bin/start-druid` script, and re-run it to bring the Druid services back up.
## Load batch data
@@ -221,7 +221,7 @@ This tutorial is only meant to be used together with the [query tutorial](../tut
If you wish to go through any of the other tutorials, you will need to:
* Shut down the cluster and reset the cluster state by removing the contents of the `var` directory under the druid package.
-* Revert the deep storage and task storage config back to local types in `conf/druid/single-server/micro-quickstart/_common/common.runtime.properties`
+* Revert the deep storage and task storage config back to local types in `conf/druid/auto/_common/common.runtime.properties`
* Restart the cluster
This is necessary because the other ingestion tutorials will write to the same "wikipedia" datasource, and later tutorials expect the cluster to use local deep storage.
diff --git a/docs/tutorials/tutorial-jdbc.md b/docs/tutorials/tutorial-jdbc.md
new file mode 100644
index 000000000000..28cee144c3b0
--- /dev/null
+++ b/docs/tutorials/tutorial-jdbc.md
@@ -0,0 +1,31 @@
+---
+id: tutorial-jdbc
+title: "Tutorial: Using the JDBC driver to query Druid"
+sidebar_label: JDBC connector
+---
+
+
+
+Redirecting you to the JDBC connector examples...
+
+
+
+Click here if you are not redirected.
+
diff --git a/docs/tutorials/tutorial-kafka.md b/docs/tutorials/tutorial-kafka.md
index eb06f4239f95..0a47d3237fa6 100644
--- a/docs/tutorials/tutorial-kafka.md
+++ b/docs/tutorials/tutorial-kafka.md
@@ -30,7 +30,7 @@ The tutorial guides you through the steps to load sample nested clickstream data
## Prerequisites
-Before you follow the steps in this tutorial, download Druid as described in the [quickstart](index.md) using the [micro-quickstart](../operations/single-server.md#micro-quickstart-4-cpu-16gib-ram) single-machine configuration and have it running on your local machine. You don't need to have loaded any data.
+Before you follow the steps in this tutorial, download Druid as described in the [quickstart](index.md) using the [auto](../operations/single-server.md#druid-auto-start) single-machine configuration and have it running on your local machine. You don't need to have loaded any data.
## Download and start Kafka
diff --git a/examples/bin/run-druid b/examples/bin/run-druid
index c302672209a0..4be0afc5854b 100755
--- a/examples/bin/run-druid
+++ b/examples/bin/run-druid
@@ -17,7 +17,7 @@
# specific language governing permissions and limitations
# under the License.
-if [ "$#" -gt 2 ] || [ "$#" -eq 0 ]
+if [ "$#" -gt 4 ] || [ "$#" -eq 0 ]
then
>&2 echo "usage: $0 [conf-dir]"
exit 1
@@ -47,7 +47,45 @@ if [ ! -d "$LOG_DIR" ]; then mkdir -p $LOG_DIR; fi
echo "Running [$1], logging to [$LOG_DIR/$1.log] if no changes made to log4j2.xml"
+if [ "$WHATAMI" = 'coordinator-overlord' ]
+then
+ SERVER_NAME=coordinator
+else
+ SERVER_NAME="$WHATAMI"
+fi
+
+
+if [ ! -f "$CONFDIR"/$WHATAMI/main.config ];
+ then
+ MAIN_CLASS="org.apache.druid.cli.Main server $SERVER_NAME"
+ else
+ MAIN_CLASS=`cat "$CONFDIR"/$WHATAMI/main.config | xargs`
+fi
+
cd "$WHEREAMI/.."
-exec "$WHEREAMI"/run-java -Ddruid.node.type=$1 "-Ddruid.log.path=$LOG_DIR" `cat "$CONFDIR"/"$WHATAMI"/jvm.config | xargs` \
- -cp "$CONFDIR"/"$WHATAMI":"$CONFDIR"/_common:"$CONFDIR"/_common/hadoop-xml:"$CONFDIR"/../_common:"$CONFDIR"/../_common/hadoop-xml:"$WHEREAMI/../lib/*" \
- `cat "$CONFDIR"/$WHATAMI/main.config | xargs`
+
+CLASS_PATH="$CONFDIR"/"$WHATAMI":"$CONFDIR"/_common:"$CONFDIR"/_common/hadoop-xml:"$CONFDIR"/../_common:"$CONFDIR"/../_common/hadoop-xml:"$WHEREAMI/../lib/*"
+
+if [ "$#" -eq 3 ] || [ "$#" -eq 4 ]
+then
+ # args: or
+ JVMARGS=`cat "$CONFDIR/_common/common.jvm.config" | xargs`
+ JVMARGS+=' '
+ JVMARGS+=$3
+
+ if [ "$#" -eq 3 ]
+ then
+ # args:
+ exec "$WHEREAMI"/run-java -Ddruid.node.type=$1 "-Ddruid.log.path=$LOG_DIR" $JVMARGS \
+ -cp $CLASS_PATH $MAIN_CLASS
+ else
+ # args:
+ exec "$WHEREAMI"/run-java -Ddruid.node.type=$1 $4 "-Ddruid.log.path=$LOG_DIR" $JVMARGS \
+ -cp $CLASS_PATH $MAIN_CLASS
+ fi
+else
+ # args:
+ exec "$WHEREAMI"/run-java -Ddruid.node.type=$1 "-Ddruid.log.path=$LOG_DIR" \
+ `cat "$CONFDIR"/"$WHATAMI"/jvm.config | xargs` \
+ -cp $CLASS_PATH $MAIN_CLASS
+fi
diff --git a/examples/bin/start-druid b/examples/bin/start-druid
new file mode 100755
index 000000000000..81d8938adbb2
--- /dev/null
+++ b/examples/bin/start-druid
@@ -0,0 +1,35 @@
+#!/bin/bash -eu
+
+# 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.
+
+PWD="$(pwd)"
+WHEREAMI="$(dirname "$0")"
+WHEREAMI="$(cd "$WHEREAMI" && pwd)"
+
+if [ -x "$(command -v python3)" ]
+then
+ exec python3 "$WHEREAMI/start-druid-main.py" "$@"
+elif [ -x "$(command -v python2)" ]
+then
+ exec python2 "$WHEREAMI/start-druid-main.py" "$@"
+elif [ -x "$(command -v python)" ]
+then
+ exec python "$WHEREAMI/start-druid-main.py" "$@"
+else
+ echo "python interepreter not found"
+fi
diff --git a/examples/bin/start-druid-main.py b/examples/bin/start-druid-main.py
new file mode 100644
index 000000000000..d1f4e6114918
--- /dev/null
+++ b/examples/bin/start-druid-main.py
@@ -0,0 +1,641 @@
+# 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.
+
+import sys
+import os
+import multiprocessing
+import argparse
+import subprocess
+import platform
+
+BASE_CONFIG_PATH = "conf/druid/auto"
+
+MEM_GB_SUFFIX = "g"
+MEM_MB_SUFFIX = "m"
+XMX_PARAMETER = "-Xmx"
+XMS_PARAMETER = "-Xms"
+DIRECT_MEM_PARAMETER = "-XX:MaxDirectMemorySize"
+SERVICE_SEPARATOR = ","
+
+TASK_JAVA_OPTS_ARRAY = ["-server", "-Duser.timezone=UTC", "-Dfile.encoding=UTF-8", "-XX:+ExitOnOutOfMemoryError",
+ "-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager"]
+TASK_JAVA_OPTS_PROPERTY = "druid.indexer.runner.javaOptsArray"
+TASK_WORKER_CAPACITY_PROPERTY = "druid.worker.capacity"
+TASK_COUNT = "task-count"
+TASK_MEM_TYPE_LOW = "low"
+TASK_MEM_TYPE_HIGH = "high"
+TASK_MEM_MAP = {
+ TASK_MEM_TYPE_LOW: ["-Xms256m", "-Xmx256m", "-XX:MaxDirectMemorySize=256m"],
+ TASK_MEM_TYPE_HIGH: ["-Xms1g", "-Xmx1g", "-XX:MaxDirectMemorySize=1g"]
+}
+
+BROKER = "broker"
+ROUTER = "router"
+COORDINATOR = "coordinator-overlord"
+HISTORICAL = "historical"
+MIDDLE_MANAGER = "middleManager"
+TASKS = "tasks"
+INDEXER = "indexer"
+
+DEFAULT_SERVICES = [
+ BROKER,
+ ROUTER,
+ COORDINATOR,
+ HISTORICAL,
+ MIDDLE_MANAGER
+]
+
+SUPPORTED_SERVICES = [
+ BROKER,
+ ROUTER,
+ COORDINATOR,
+ HISTORICAL,
+ MIDDLE_MANAGER,
+ INDEXER
+]
+
+SERVICE_MEMORY_RATIO = {
+ MIDDLE_MANAGER: 1,
+ ROUTER: 2,
+ COORDINATOR: 30,
+ BROKER: 46,
+ HISTORICAL: 80,
+ TASKS: 30,
+ INDEXER: 32
+}
+
+MINIMUM_MEMORY_MB = {
+ MIDDLE_MANAGER: 64,
+ ROUTER: 128,
+ TASKS: 1024,
+ BROKER: 900,
+ COORDINATOR: 256,
+ HISTORICAL: 900,
+ INDEXER: 1124
+}
+
+HEAP_TO_TOTAL_MEM_RATIO = {
+ MIDDLE_MANAGER: 1,
+ ROUTER: 1,
+ COORDINATOR: 1,
+ BROKER: 0.60,
+ HISTORICAL: 0.40,
+ TASKS: 0.50,
+ INDEXER: 0.50
+}
+
+LOGGING_ENABLED = False
+
+
+def print_if_verbose(message):
+ if LOGGING_ENABLED:
+ print(message)
+
+
+def configure_parser():
+ parser = argparse.ArgumentParser(
+ prog='start-druid',
+ formatter_class=argparse.RawTextHelpFormatter,
+ epilog=
+ """
+sample usage:
+ start-druid
+ Start up all the services (including zk).
+ start-druid -m=100g
+ Start up all the services (including zk)
+ using a total memory of 100GB.
+ start-druid -m=100g --compute
+ Compute memory distribution and validate arguments.
+ start-druid -m=100g -s=broker,router
+ Starts a broker and a router, using a total memory of 100GB.
+ start-druid -m=100g --s=broker,router \\
+ -c=conf/druid/single-server/custom
+ Starts a broker and a router, using a total memory of 100GB.
+ Reads configs for each service (jvm.config, runtime.properties)
+ from respective folders inside the given root config path.
+ start-druid -s=broker,router \\
+ -c=conf/druid/single-server/custom
+ Starts a broker and a router service, reading service configs
+ from the given root directory. Calculates memory requirements for
+ each service, if required, using upto 80% of the total system memory.
+ start-druid -m=100g \\
+ -s=broker,router \\
+ -c=conf/druid/single-server/custom \\
+ --zk
+ Starts broker, router and zookeeper.
+ zookeeper config is read from conf/zk.
+"""
+ )
+ parser.add_argument('--memory', '-m', type=str, required=False,
+ help='Total memory for all processes (services and tasks, if any). \n'
+ 'This parameter is ignored if each service already has a jvm.config \n'
+ 'in the given conf directory. e.g. 500m, 4g, 6g\n')
+ parser.add_argument('--services', '-s', type=str, required=False,
+ help='List of services to be started, subset of \n'
+ '{broker, router, middleManager, historical, coordinator-overlord, indexer}. \n'
+ 'If the argument is not given, broker, router, middleManager, historical, coordinator-overlord \n'
+ 'and zookeeper is started. e.g. -s=broker,historical')
+ parser.add_argument('--config', '-c', type=str, required=False,
+ help='Relative path to the directory containing common and service \n'
+ 'specific properties to be overridden. \n'
+ 'This directory must contain \'_common\' directory with \n'
+ '\'common.jvm.config\' & \'common.runtime.properties\' files. \n'
+ 'If this argument is not given, config from \n'
+ 'conf/druid/auto directory is used.\n')
+ parser.add_argument('--compute', action='store_true',
+ help='Does not start Druid, only displays the memory allocated \n'
+ 'to each service if started with the given total memory.\n')
+ parser.add_argument('--zk', '-zk', action='store_true',
+ help='Specification to run zookeeper, \n'
+ 'zk config is picked up from conf/zk.')
+ parser.add_argument('--verbose', action='store_true', help='Log details')
+
+ parser.set_defaults(zk=False)
+ parser.set_defaults(compute=False)
+ parser.set_defaults(verbose=False)
+
+ return parser
+
+
+def is_file(path):
+ return os.path.isfile(path)
+
+
+def is_dir(path):
+ return os.path.isdir(path)
+
+
+def resolve_path(path):
+ return os.path.abspath(path)
+
+
+def validate_common_jvm_args(config):
+ if is_file('{0}/_common/common.jvm.config'.format(config)) is False:
+ raise ValueError('_common/common.jvm.config file is missing in the root config, '
+ 'check {0}/_common directory'.format(BASE_CONFIG_PATH))
+
+
+def validate_common_directory(config):
+ if is_dir('{0}/_common'.format(config)) is False:
+ raise ValueError(
+ '_common directory is missing in the root config, check {0}/_common directory'.format(BASE_CONFIG_PATH))
+
+ if is_file('{0}/_common/common.runtime.properties'.format(config)) is False:
+ raise ValueError('_common/common.runtime.properties file is missing in the root config, '
+ 'check {0}/_common directory'.format(BASE_CONFIG_PATH))
+
+
+def parse_arguments(args):
+ service_list = []
+ config = ""
+ total_memory = ""
+ compute = False
+ zk = False
+
+ if args.compute:
+ compute = True
+ if args.zk:
+ zk = True
+ if args.config is not None:
+ config = resolve_path(os.path.join(os.getcwd(), args.config))
+ if is_dir(config) is False:
+ raise ValueError('config {0} not found'.format(config))
+ if args.memory is not None:
+ total_memory = args.memory
+ if args.services is not None:
+ services = args.services.split(SERVICE_SEPARATOR)
+
+ for service in services:
+ if service not in SUPPORTED_SERVICES:
+ raise ValueError('Invalid service name {0}, should be one of {1}'.format(service, DEFAULT_SERVICES))
+
+ if service in service_list:
+ raise ValueError('{0} is specified multiple times'.format(service))
+
+ service_list.append(service)
+
+ if INDEXER in services and MIDDLE_MANAGER in services:
+ raise ValueError('one of indexer and middleManager can run')
+
+ if len(service_list) == 0:
+ # start all services
+ service_list = DEFAULT_SERVICES
+ zk = True
+
+ return config, total_memory, service_list, zk, compute
+
+
+def print_startup_config(service_list, config, zk):
+ print_if_verbose('Starting {0}'.format(service_list))
+ print_if_verbose('Reading config from {0}'.format(config))
+ if zk:
+ zk_config = resolve_path('{0}/../conf/zk'.format(os.getcwd()))
+ print_if_verbose('Starting zk, reading default config from {0}'.format(zk_config))
+ print_if_verbose('\n')
+
+
+def task_memory_params_present(config, service):
+ java_opts_property_present = False
+ worker_capacity_property_present = False
+
+ if is_file('{0}/{1}/runtime.properties'.format(config, service)):
+ with open('{0}/{1}/runtime.properties'.format(config, service)) as file:
+ for line in file:
+ if line.startswith(TASK_JAVA_OPTS_PROPERTY):
+ java_opts_property_present = True
+ elif line.startswith(TASK_WORKER_CAPACITY_PROPERTY):
+ worker_capacity_property_present = True
+
+ return java_opts_property_present, worker_capacity_property_present
+
+
+def verify_service_config(service, config):
+ path = '{0}/{1}/jvm.config'.format(config, service)
+
+ required_parameters = [XMX_PARAMETER, XMS_PARAMETER]
+
+ if HEAP_TO_TOTAL_MEM_RATIO.get(service) != 1:
+ required_parameters.append(DIRECT_MEM_PARAMETER)
+
+ with open(path) as file:
+ for line in file:
+ if line.startswith(XMX_PARAMETER) and XMX_PARAMETER in required_parameters:
+ required_parameters.remove(XMX_PARAMETER)
+ if line.startswith(XMS_PARAMETER) and XMS_PARAMETER in required_parameters:
+ required_parameters.remove(XMS_PARAMETER)
+ if line.startswith(DIRECT_MEM_PARAMETER) and DIRECT_MEM_PARAMETER in required_parameters:
+ required_parameters.remove(DIRECT_MEM_PARAMETER)
+
+ if len(required_parameters) > 0:
+ params = ",".join(required_parameters)
+ raise ValueError('{0} missing in {1}/jvm.config'.format(params, service))
+
+ if service == MIDDLE_MANAGER:
+ if is_file('{0}/{1}/runtime.properties'.format(config, service)) is False:
+ raise ValueError('{0}/runtime.properties file is missing in the root config'.format(service))
+
+ mm_task_java_opts_prop, mm_task_worker_capacity_prop = task_memory_params_present(config, MIDDLE_MANAGER)
+
+ if mm_task_java_opts_property is False:
+ raise ValueError('{0} property missing in {1}/runtime.properties'.format(TASK_JAVA_OPTS_PROPERTY, service))
+
+
+def should_compute_memory(config, total_memory, service_list):
+ """
+ if memory argument is given, memory for services and tasks is computed, jvm.config file
+ or runtime.properties with task memory specification shouldn't be present
+ Alternatively, all memory related parameters are specified
+ which implies following should be present:
+ jvm.config file for all services with -Xmx=***, Xms=*** parameters
+ -XX:MaxDirectMemorySize=** in jvm.config for broker and historical
+ druid.indexer.runner.javaOptsArray (optionally druid.worker.capacity) in
+ rootDirectory/middleManager/runtime.properties
+ """
+
+ jvm_config_count = 0
+ for service in service_list:
+ if is_file('{0}/{1}/jvm.config'.format(config, service)):
+ jvm_config_count += 1
+
+ mm_task_property_present = False
+ if MIDDLE_MANAGER in service_list:
+ mm_task_java_opts_prop, mm_task_worker_capacity_prop = task_memory_params_present(config, MIDDLE_MANAGER)
+ mm_task_property_present = mm_task_java_opts_prop or mm_task_worker_capacity_prop
+
+ indexer_task_worker_capacity_prop = False
+ if INDEXER in service_list:
+ indexer_task_java_opts_prop, indexer_task_worker_capacity_prop = task_memory_params_present(config, INDEXER)
+
+ # possible error states
+ # 1. memory argument is specified, also jvm.config or middleManger/runtime.properties having
+ # druid.indexer.runner.javaOptsArray or druid.worker.capacity parameters is present
+ # 2. jvm.config is not present for any service, but middleManger/runtime.properties has
+ # druid.indexer.runner.javaOptsArray or druid.worker.capacity parameters
+ # or indexer/runtime.properties has druid.worker.capacity
+ # 3. jvm.config present for some but not all services
+ # 4. jvm.config file is present for all services, but it doesn't contain required parameters
+ # 5. lastly, if middleManager is to be started, and it is missing task memory properties
+ if jvm_config_count > 0 or mm_task_property_present or indexer_task_worker_capacity_prop:
+ if total_memory != "":
+ raise ValueError(
+ "If jvm.config for services and/or middleManager/indexer configs "
+ "(druid.worker.capacity, druid.indexer.runner.javaOptsArray) is present, "
+ "memory argument shouldn't be specified")
+ if jvm_config_count == 0 and mm_task_property_present:
+ raise ValueError("middleManger configs (druid.indexer.runner.javaOptsArray or druid.worker.capacity) "
+ "is present in middleManager/runtime.properties, "
+ "add jvm.config for all other services")
+ if jvm_config_count == 0 and indexer_task_worker_capacity_prop:
+ raise ValueError("indexer configs (druid.worker.capacity) "
+ "is present in indexer/runtime.properties, "
+ "add jvm.config for all other services")
+ if jvm_config_count != len(service_list):
+ raise ValueError("jvm.config file should be present for all services or none")
+ for service in service_list:
+ verify_service_config(service, config)
+
+ return False
+
+ # compute memory only when none of the specified services contains jvm.config,
+ # if middleManager is to be started it shouldn't contain task memory properties
+ # if indexer is present it shouldn't contain task memory properties
+ return True
+
+
+def get_physical_memory_linux():
+ mem_bytes = os.sysconf('SC_PAGE_SIZE') * os.sysconf('SC_PHYS_PAGES')
+ mem_mbs = int(mem_bytes / (1024 * 1024))
+ return mem_mbs
+
+
+def get_physical_memory_osx():
+ p1 = subprocess.Popen(['sysctl', '-a'], stdout=subprocess.PIPE)
+ p2 = subprocess.check_output(['grep', 'hw.memsize'], stdin=p1.stdout)
+ p2 = p2.decode('utf-8')
+ fields = p2.split(':')
+
+ mem_mbs = int(int(fields[1]) / (1024 * 1024))
+
+ return mem_mbs
+
+
+def get_physical_memory():
+ operating_system = platform.system()
+ print_if_verbose('operating system is {0}'.format(operating_system))
+
+ system_memory = None
+
+ try:
+ if operating_system == 'Darwin':
+ system_memory = get_physical_memory_osx()
+ elif operating_system == 'Linux':
+ system_memory = get_physical_memory_linux()
+ except Exception:
+ pass
+
+ return system_memory
+
+
+def convert_total_memory_string(memory):
+ try:
+ if memory == '':
+ physical_memory = get_physical_memory()
+
+ if physical_memory is None:
+ raise ValueError('Please specify memory argument')
+
+ return physical_memory
+ elif memory.endswith(MEM_MB_SUFFIX):
+ return int(memory[:-1])
+ elif memory.endswith(MEM_GB_SUFFIX):
+ return 1024 * int(memory[:-1])
+ else:
+ raise ValueError('Incorrect format for memory argument, expected format is ')
+ except ValueError as e:
+ raise e
+ except Exception:
+ raise ValueError('Incorrect format for memory argument, expected format is ')
+
+
+def check_memory_constraint(total_memory, services):
+ # 80% of total memory >= sum of lower bound service memory should be
+ lower_bound_memory = 0
+
+ service_list = list(services)
+ if MIDDLE_MANAGER in services:
+ service_list.append(TASKS)
+
+ for service in service_list:
+ lower_bound_memory += MINIMUM_MEMORY_MB.get(service)
+
+ required_memory = int(lower_bound_memory / 0.8)
+
+ if total_memory < required_memory:
+ raise ValueError('Minimum memory required for starting services is {0}m'.format(required_memory))
+
+ if total_memory >= 2 * lower_bound_memory:
+ return int(total_memory / 2)
+ else:
+ return lower_bound_memory
+
+
+def build_mm_task_java_opts_array(memory_type):
+ task_memory = '-D{0}=['.format(TASK_JAVA_OPTS_PROPERTY)
+
+ mem_array = TASK_MEM_MAP.get(memory_type)
+
+ java_opts_list = TASK_JAVA_OPTS_ARRAY + mem_array
+
+ for item in java_opts_list:
+ task_memory += '\"{0}\",'.format(item)
+
+ task_memory = task_memory[:-1]
+ task_memory += ']'
+ return task_memory
+
+
+def compute_tasks_memory(allocated_memory):
+ if allocated_memory >= 4096:
+ task_count = int(allocated_memory / 2048)
+ memory_type = TASK_MEM_TYPE_HIGH
+ task_memory_mb = 2048
+ else:
+ task_count = 2
+ memory_type = TASK_MEM_TYPE_LOW
+ task_memory_mb = 512
+ task_count = min(task_count, multiprocessing.cpu_count())
+
+ return memory_type, task_count, task_memory_mb
+
+
+def build_memory_config(service, allocated_memory):
+ if service == TASKS:
+ memory_type, task_count, task_memory = compute_tasks_memory(allocated_memory)
+ java_opts_array = build_mm_task_java_opts_array(memory_type)
+ return ['-D{0}={1}'.format(TASK_WORKER_CAPACITY_PROPERTY, task_count),
+ java_opts_array], task_memory * task_count
+ elif service == INDEXER:
+ heap_memory = HEAP_TO_TOTAL_MEM_RATIO.get(service) * allocated_memory
+ direct_memory = int(allocated_memory - heap_memory)
+ heap_memory = int(heap_memory)
+ memory_type, task_count, task_memory = compute_tasks_memory(allocated_memory)
+ return ['-D{0}={1}'.format(TASK_WORKER_CAPACITY_PROPERTY, task_count),
+ '-Xms{0}m -Xmx{0}m -XX:MaxDirectMemorySize={1}m'.format(heap_memory, direct_memory)], \
+ task_memory * task_count
+ else:
+ heap_memory = HEAP_TO_TOTAL_MEM_RATIO.get(service) * allocated_memory
+ direct_memory = int(allocated_memory - heap_memory)
+ heap_memory = int(heap_memory)
+
+ if direct_memory == 0:
+ return '-Xms{0}m -Xmx{0}m'.format(heap_memory), allocated_memory
+
+ return '-Xms{0}m -Xmx{0}m -XX:MaxDirectMemorySize={1}m'.format(heap_memory, direct_memory), allocated_memory
+
+
+def distribute_memory(services, total_memory):
+ service_memory_config = {}
+
+ memory_weight_sum = 0
+
+ service_list = list(services)
+ if MIDDLE_MANAGER in services:
+ service_list.append(TASKS)
+
+ for service in service_list:
+ memory_weight_sum += SERVICE_MEMORY_RATIO.get(service)
+
+ multiplier = total_memory / memory_weight_sum
+
+ lower_bound_memory_allocation = 0
+ allocated_services = set()
+
+ for service in service_list:
+ allocated_memory = SERVICE_MEMORY_RATIO.get(service) * multiplier
+ if service in MINIMUM_MEMORY_MB and allocated_memory < MINIMUM_MEMORY_MB.get(service):
+ allocated_memory = MINIMUM_MEMORY_MB.get(service)
+ service_memory_config[service], allocated_memory = build_memory_config(service, allocated_memory)
+ lower_bound_memory_allocation += allocated_memory
+ allocated_services.add(service)
+
+ if lower_bound_memory_allocation > 0:
+ # compute the multiplier again for remaining services
+ memory_weight_sum = 0
+ for service in service_list:
+ if service in allocated_services:
+ continue
+ memory_weight_sum += SERVICE_MEMORY_RATIO.get(service)
+ multiplier = (total_memory - lower_bound_memory_allocation) / memory_weight_sum
+
+ for service in service_list:
+ if service in allocated_services:
+ continue
+ allocated_memory = SERVICE_MEMORY_RATIO.get(service) * multiplier
+ if service in MINIMUM_MEMORY_MB and allocated_memory < MINIMUM_MEMORY_MB.get(service):
+ allocated_memory = MINIMUM_MEMORY_MB.get(service)
+
+ service_memory_config[service], allocated_memory = build_memory_config(service, allocated_memory)
+
+ print_if_verbose('\nMemory distribution for services:')
+ for key, value in service_memory_config.items():
+ print_if_verbose('{0}, {1}'.format(key, value))
+ print_if_verbose('\n')
+
+ return service_memory_config
+
+
+def append_command(commands, command):
+ commands.append('--command')
+ commands.append(command)
+
+
+def build_supervise_script_arguments(service_list, service_memory_config, config, zk):
+ commands = []
+ commands.append('supervise')
+
+ append_command(commands, ":verify bin/verify-java")
+ append_command(commands, ":verify bin/verify-default-ports")
+ append_command(commands, ":notify bin/greet")
+ append_command(commands, ":kill-timeout 10")
+
+ if zk:
+ append_command(commands, "!p10 zk bin/run-zk conf")
+
+ for service in service_list:
+ memory_config = service_memory_config.get(service)
+
+ prefix = ''
+ if service == MIDDLE_MANAGER:
+ prefix = '!p90 '
+
+ if memory_config is None:
+ append_command(commands, '{0}{1} bin/run-druid {1} {2}'.format(prefix, service, config))
+ else:
+ if service == MIDDLE_MANAGER:
+ task_config = service_memory_config.get(TASKS)
+ task_count = task_config[0]
+ task_memory = task_config[1]
+ append_command(
+ commands,
+ '{0}{1} bin/run-druid {1} {2} \'{3}\' \'{4} {5}\''
+ .format(prefix, service, config, memory_config, task_count, task_memory))
+ elif service == INDEXER:
+ task_count = memory_config[0]
+ jvm_args = memory_config[1]
+ append_command(
+ commands,
+ '{0}{1} bin/run-druid {1} {2} \'{3}\' \'{4}\''
+ .format(prefix, service, config, jvm_args, task_count))
+ else:
+ append_command(commands,
+ '{0}{1} bin/run-druid {1} {2} \'{3}\''.format(prefix, service, config, memory_config))
+
+ print_if_verbose('Supervise script args:')
+ for item in commands:
+ print_if_verbose(item)
+
+ print_if_verbose('\n')
+
+ return commands
+
+
+def main():
+ parser = configure_parser()
+ args = parser.parse_args()
+
+ global LOGGING_ENABLED
+ LOGGING_ENABLED = args.verbose or args.compute
+
+ config, total_memory, service_list, zk, compute = parse_arguments(args)
+
+ # change directory to bin
+ os.chdir(os.path.dirname(sys.argv[0]))
+
+ if config == "":
+ config = resolve_path('{0}/../{1}'.format(os.getcwd(), BASE_CONFIG_PATH))
+
+ validate_common_directory(config)
+
+ print_startup_config(service_list, config, zk)
+
+ service_memory_config = {}
+
+ if should_compute_memory(config, total_memory, service_list):
+ # if memory is to be computed, _common directory should contain common.jvm.config
+ validate_common_jvm_args(config)
+ memory_in_mega_bytes = convert_total_memory_string(total_memory)
+ print_if_verbose('Total memory is {0}m\n'.format(memory_in_mega_bytes))
+ memory_to_be_used = check_memory_constraint(memory_in_mega_bytes, service_list)
+ print_if_verbose('Memory used for services & tasks {0}m\n'.format(memory_to_be_used))
+ service_memory_config = distribute_memory(service_list, memory_to_be_used)
+ else:
+ print_if_verbose('Not computing memory distribution, reading memory specification from service jvm.config & '
+ 'middleManager/runtime.properties\n')
+
+ script_arguments = build_supervise_script_arguments(service_list, service_memory_config, config, zk)
+
+ if compute:
+ return
+
+ os.execv('./supervise', script_arguments)
+
+
+try:
+ main()
+except (KeyboardInterrupt, ValueError) as error:
+ print(error)
+ sys.exit(1)
diff --git a/examples/bin/supervise b/examples/bin/supervise
index 81b7e57b049e..ba336c31aa66 100755
--- a/examples/bin/supervise
+++ b/examples/bin/supervise
@@ -46,6 +46,51 @@ sub usage
die "usage: $0 -c [-d ] [-t ] [--svlogd ]\n";
}
+sub process_config
+{
+ my @lines = @_;
+ my @commands;
+ my @verify;
+ my @notify;
+ my $kill_timeout;
+ for my $line (@lines)
+ {
+ if ($line =~ /^(:verify|:notify|:kill-timeout|(?:\!p[0-9]+\s+)?[^:]\S+)\s+(.+)$/) {
+ my $name = $1;
+ my $order = 50;
+ my $command = $2;
+
+ if ($name =~ /^(?:\!p([0-9]+)\s+)(.*)$/) {
+ $order = $1;
+ $name = $2;
+ }
+
+ if ($name eq ':verify') {
+ push @verify, $command;
+ } elsif ($name eq ':notify') {
+ push @notify, $command;
+ } elsif ($name eq ':kill-timeout') {
+ $kill_timeout = int($command);
+ } else {
+ die "Duplicate command: $line\n" if grep { $_->{name} eq $name } @commands;
+ push @commands, {
+ name => $name,
+ command => $command,
+ order => $order, # Stop order for this command
+ pid => 0, # Current pid, or 0 if not running
+ down => 0, # Time the proc should be down until
+ killed => 0, # Signal we sent to this process
+ restarting => 0, # True if this command is currently restarting
+ };
+ }
+ } else {
+ die "Syntax error: $line\n";
+ }
+ }
+
+ return { commands => \@commands, verify => \@verify, notify => \@notify, 'kill-timeout' => $kill_timeout };
+}
+
sub read_config_file
{
my ($config_file) = @_;
@@ -53,49 +98,20 @@ sub read_config_file
open my $config_fh, "<", $config_file
or die "open $config_file: $!";
- my @commands;
- my @verify;
- my @notify;
- my $kill_timeout;
+ my @lines;
while (my $line = <$config_fh>) {
chomp $line;
next if $line =~ /^(\s*\#.*|\s*)$/;
if ($line =~ /^(:verify|:notify|:kill-timeout|(?:\!p[0-9]+\s+)?[^:]\S+)\s+(.+)$/) {
- my $name = $1;
- my $order = 50;
- my $command = $2;
-
- if ($name =~ /^(?:\!p([0-9]+)\s+)(.*)$/) {
- $order = $1;
- $name = $2;
- }
-
- if ($name eq ':verify') {
- push @verify, $command;
- } elsif ($name eq ':notify') {
- push @notify, $command;
- } elsif ($name eq ':kill-timeout') {
- $kill_timeout = int($command);
- } else {
- die "Duplicate command: $line\n" if grep { $_->{name} eq $name } @commands;
- push @commands, {
- name => $name,
- command => $command,
- order => $order, # Stop order for this command
- pid => 0, # Current pid, or 0 if not running
- down => 0, # Time the proc should be down until
- killed => 0, # Signal we sent to this process
- restarting => 0, # True if this command is currently restarting
- };
- }
+ push @lines, $line
} else {
die "Syntax error: $line\n";
}
}
close $config_fh;
- return { commands => \@commands, verify => \@verify, notify => \@notify, 'kill-timeout' => $kill_timeout };
+ return @lines;
}
sub stringify_exit_status
@@ -179,13 +195,23 @@ usage() unless GetOptions(
'vardir|d=s',
'kill-timeout|t=i',
'chdir=s',
- 'svlogd:s'
+ 'svlogd:s',
+ 'command=s@'
);
-usage() unless $opt{'conf'} && $opt{'vardir'};
+usage() unless (($opt{'command'} && @{$opt{'command'}}) || $opt{'conf'}) && $opt{'vardir'};
+
+my @config_lines;
+
+# get commands to execute either from reading the config file or command line
+if (not defined $opt{'conf'}) {
+ @config_lines = @{$opt{'command'}}
+} else {
+ @config_lines = read_config_file($opt{'conf'});
+}
+
+my $config = process_config(@config_lines);
-# Read config file
-my $config = read_config_file($opt{'conf'});
@commands = @{$config->{commands}};
if (!@commands) {
diff --git a/examples/conf/druid/auto/_common/common.jvm.config b/examples/conf/druid/auto/_common/common.jvm.config
new file mode 100644
index 000000000000..fd74cf358979
--- /dev/null
+++ b/examples/conf/druid/auto/_common/common.jvm.config
@@ -0,0 +1,7 @@
+-server
+-XX:+ExitOnOutOfMemoryError
+-XX:+UseG1GC
+-Duser.timezone=UTC
+-Dfile.encoding=UTF-8
+-Djava.io.tmpdir=var/tmp
+-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager
diff --git a/examples/conf/druid/auto/_common/common.runtime.properties b/examples/conf/druid/auto/_common/common.runtime.properties
new file mode 100644
index 000000000000..b0adb0695cd7
--- /dev/null
+++ b/examples/conf/druid/auto/_common/common.runtime.properties
@@ -0,0 +1,158 @@
+#
+# 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.
+#
+
+# Extensions specified in the load list will be loaded by Druid
+# We are using local fs for deep storage - not recommended for production - use S3, HDFS, or NFS instead
+# We are using local derby for the metadata store - not recommended for production - use MySQL or Postgres instead
+
+# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
+# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
+# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
+druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
+
+# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
+# and uncomment the line below to point to your directory.
+#druid.extensions.hadoopDependenciesDir=/my/dir/hadoop-dependencies
+
+
+#
+# Hostname
+#
+druid.host=localhost
+
+#
+# Logging
+#
+
+# Log all runtime properties on startup. Disable to avoid logging properties on startup:
+druid.startup.logging.logProperties=true
+
+#
+# Zookeeper
+#
+
+druid.zk.service.host=localhost
+druid.zk.paths.base=/druid
+
+#
+# Metadata storage
+#
+
+# For Derby server on your Druid Coordinator (only viable in a cluster with a single Coordinator, no fail-over):
+druid.metadata.storage.type=derby
+druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527/var/druid/metadata.db;create=true
+druid.metadata.storage.connector.host=localhost
+druid.metadata.storage.connector.port=1527
+
+# For MySQL (make sure to include the MySQL JDBC driver on the classpath):
+#druid.metadata.storage.type=mysql
+#druid.metadata.storage.connector.connectURI=jdbc:mysql://db.example.com:3306/druid
+#druid.metadata.storage.connector.user=...
+#druid.metadata.storage.connector.password=...
+
+# For PostgreSQL:
+#druid.metadata.storage.type=postgresql
+#druid.metadata.storage.connector.connectURI=jdbc:postgresql://db.example.com:5432/druid
+#druid.metadata.storage.connector.user=...
+#druid.metadata.storage.connector.password=...
+
+#
+# Deep storage
+#
+
+# For local disk (only viable in a cluster if this is a network mount):
+druid.storage.type=local
+druid.storage.storageDirectory=var/druid/segments
+
+# For HDFS:
+#druid.storage.type=hdfs
+#druid.storage.storageDirectory=/druid/segments
+
+# For S3:
+#druid.storage.type=s3
+#druid.storage.bucket=your-bucket
+#druid.storage.baseKey=druid/segments
+#druid.s3.accessKey=...
+#druid.s3.secretKey=...
+
+#
+# Indexing service logs
+#
+
+# For local disk (only viable in a cluster if this is a network mount):
+druid.indexer.logs.type=file
+druid.indexer.logs.directory=var/druid/indexing-logs
+
+# For HDFS:
+#druid.indexer.logs.type=hdfs
+#druid.indexer.logs.directory=/druid/indexing-logs
+
+# For S3:
+#druid.indexer.logs.type=s3
+#druid.indexer.logs.s3Bucket=your-bucket
+#druid.indexer.logs.s3Prefix=druid/indexing-logs
+
+#
+# Service discovery
+#
+
+druid.selectors.indexing.serviceName=druid/overlord
+druid.selectors.coordinator.serviceName=druid/coordinator
+
+#
+# Monitoring
+#
+
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.emitter=noop
+druid.emitter.logging.logLevel=info
+
+# Storage type of double columns
+# ommiting this will lead to index double as float at the storage layer
+
+druid.indexing.doubleStorage=double
+
+#
+# Security
+#
+druid.server.hiddenProperties=["druid.s3.accessKey","druid.s3.secretKey","druid.metadata.storage.connector.password", "password", "key", "token", "pwd"]
+
+
+#
+# SQL
+#
+druid.sql.enable=true
+
+# Planning SQL query when there is aggregate distinct in the statement
+druid.sql.planner.useGroupingSetForExactDistinct=true
+
+#
+# Lookups
+#
+druid.lookup.enableLookupSyncOnStartup=false
+
+#
+# Expression processing config
+#
+druid.expressions.useStrictBooleans=true
+
+#
+# Http client
+#
+druid.global.http.eagerInitialization=false
diff --git a/examples/conf/druid/auto/_common/log4j2.xml b/examples/conf/druid/auto/_common/log4j2.xml
new file mode 100644
index 000000000000..66dc13da4c5e
--- /dev/null
+++ b/examples/conf/druid/auto/_common/log4j2.xml
@@ -0,0 +1,86 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/examples/conf/druid/auto/broker/runtime.properties b/examples/conf/druid/auto/broker/runtime.properties
new file mode 100644
index 000000000000..f4c494019933
--- /dev/null
+++ b/examples/conf/druid/auto/broker/runtime.properties
@@ -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.
+#
+
+druid.service=druid/broker
+druid.plaintextPort=8082
+
+# HTTP server settings
+# HTTP server thread pool size. Higher values increase peak load on the Broker, but
+# may be useful for high-concurrency workloads.
+# Default is max(10, (Number of processors * 17) / 16 + 2) + 30.
+# druid.server.http.numThreads=N
+
+# HTTP client settings
+# Connection pool size from the Broker to each data server. May be useful to
+# raise this for high-concurrency workloads.
+# druid.broker.http.numConnections=20
+
+# Processing threads and buffers
+# Determined automatically based on available memory. For details on how to manually set parameters:
+# https://druid.apache.org/docs/latest/operations/basic-cluster-tuning.html#guidelines-for-processing-threads-and-buffers
+druid.processing.tmpDir=var/druid/processing
+
+# Query cache disabled -- push down caching and merging instead
+druid.broker.cache.useCache=false
+druid.broker.cache.populateCache=false
diff --git a/examples/conf/druid/auto/coordinator-overlord/runtime.properties b/examples/conf/druid/auto/coordinator-overlord/runtime.properties
new file mode 100644
index 000000000000..c053823f8ed8
--- /dev/null
+++ b/examples/conf/druid/auto/coordinator-overlord/runtime.properties
@@ -0,0 +1,33 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+druid.service=druid/coordinator
+druid.plaintextPort=8081
+
+druid.coordinator.startDelay=PT10S
+druid.coordinator.period=PT5S
+druid.manager.segments.pollDuration=PT5S
+
+# Run the overlord service in the coordinator process
+druid.coordinator.asOverlord.enabled=true
+druid.coordinator.asOverlord.overlordService=druid/overlord
+
+druid.indexer.queue.startDelay=PT5S
+
+druid.indexer.storage.type=metadata
diff --git a/examples/conf/druid/auto/historical/runtime.properties b/examples/conf/druid/auto/historical/runtime.properties
new file mode 100644
index 000000000000..6c241aa7cf06
--- /dev/null
+++ b/examples/conf/druid/auto/historical/runtime.properties
@@ -0,0 +1,35 @@
+#
+# 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.
+#
+
+# HTTP server thread pool size. Higher values increase peak load on the Broker, but
+# may be useful for high-concurrency workloads.
+# Default is max(10, (Number of processors * 17) / 16 + 2) + 30.
+# druid.server.http.numThreads=N
+
+# Processing threads and buffers
+# Determined automatically based on available memory. For details on how to manually set parameters:
+# https://druid.apache.org/docs/latest/operations/basic-cluster-tuning.html#guidelines-for-processing-threads-and-buffers
+druid.processing.tmpDir=var/druid/processing
+
+# Segment storage
+druid.segmentCache.locations=[{"path":"var/druid/segment-cache","maxSize":"300g"}]
+# Query cache
+druid.historical.cache.useCache=true
+druid.historical.cache.populateCache=true
+druid.cache.type=caffeine
diff --git a/examples/conf/druid/auto/indexer/runtime.properties b/examples/conf/druid/auto/indexer/runtime.properties
new file mode 100644
index 000000000000..5aef64535b67
--- /dev/null
+++ b/examples/conf/druid/auto/indexer/runtime.properties
@@ -0,0 +1,34 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+druid.service=druid/indexer
+druid.plaintextPort=8091
+
+# Number of tasks (druid.worker.capacity) is automatically
+# determined based on available processor.
+
+# Task launch parameters
+druid.indexer.task.baseTaskDir=var/druid/task
+
+# Processing threads and buffers on Indexer
+# Determined automatically based on available memory. For details on how to manually set parameters:
+# https://druid.apache.org/docs/latest/operations/basic-cluster-tuning.html#guidelines-for-processing-threads-and-buffers
+
+# Hadoop indexing
+druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp
diff --git a/examples/conf/druid/auto/middleManager/runtime.properties b/examples/conf/druid/auto/middleManager/runtime.properties
new file mode 100644
index 000000000000..08c58bae6de0
--- /dev/null
+++ b/examples/conf/druid/auto/middleManager/runtime.properties
@@ -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.
+#
+
+druid.service=druid/middleManager
+druid.plaintextPort=8091
+
+# Number of tasks (druid.worker.capacity) and memory usage per task (druid.indexer.runner.javaOptsArray) is automatically
+# determined based on available memory. For details on how to manually set parameters, see:
+# https://druid.apache.org/docs/latest/operations/basic-cluster-tuning.html#middlemanager
+
+# Task launch parameters
+druid.indexer.runner.javaCommand=bin/run-java
+druid.indexer.task.baseTaskDir=var/druid/task
+
+# Processing threads and buffers on Peons
+# Determined automatically based on available memory. For details on how to manually set parameters:
+# https://druid.apache.org/docs/latest/operations/basic-cluster-tuning.html#guidelines-for-processing-threads-and-buffers
+
+# Hadoop indexing
+druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp
diff --git a/examples/conf/druid/auto/router/runtime.properties b/examples/conf/druid/auto/router/runtime.properties
new file mode 100644
index 000000000000..3858dec044bd
--- /dev/null
+++ b/examples/conf/druid/auto/router/runtime.properties
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+druid.service=druid/router
+druid.plaintextPort=8888
+
+# Service discovery
+druid.router.defaultBrokerServiceName=druid/broker
+druid.router.coordinatorServiceName=druid/coordinator
+
+# Management proxy to coordinator / overlord: required for unified web console.
+druid.router.managementProxy.enabled=true
diff --git a/extendedset/pom.xml b/extendedset/pom.xml
index bdd52ead1656..1927d59cff52 100755
--- a/extendedset/pom.xml
+++ b/extendedset/pom.xml
@@ -31,7 +31,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
diff --git a/extensions-contrib/aliyun-oss-extensions/pom.xml b/extensions-contrib/aliyun-oss-extensions/pom.xml
index 4fdde743a947..9336122ed5ff 100644
--- a/extensions-contrib/aliyun-oss-extensions/pom.xml
+++ b/extensions-contrib/aliyun-oss-extensions/pom.xml
@@ -28,7 +28,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml
index a267edc55485..fc7f73be4a6c 100644
--- a/extensions-contrib/ambari-metrics-emitter/pom.xml
+++ b/extensions-contrib/ambari-metrics-emitter/pom.xml
@@ -24,7 +24,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml
index d9ed4ccff56b..2dd7d409143e 100644
--- a/extensions-contrib/cassandra-storage/pom.xml
+++ b/extensions-contrib/cassandra-storage/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml
index 3bd9c3220371..e3c13d1962cc 100644
--- a/extensions-contrib/cloudfiles-extensions/pom.xml
+++ b/extensions-contrib/cloudfiles-extensions/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/compressed-bigdecimal/pom.xml b/extensions-contrib/compressed-bigdecimal/pom.xml
index a9929c3c949c..f4309fa2d26a 100644
--- a/extensions-contrib/compressed-bigdecimal/pom.xml
+++ b/extensions-contrib/compressed-bigdecimal/pom.xml
@@ -25,7 +25,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregatorTest.java
index fec14699d40c..06709040b645 100644
--- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregatorTest.java
+++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregatorTest.java
@@ -19,23 +19,13 @@
package org.apache.druid.compressedbigdecimal;
-import com.google.common.collect.ImmutableSet;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-
public class CompressedBigDecimalMaxSqlAggregatorTest extends CompressedBigDecimalSqlAggregatorTestBase
{
private static final String FUNCTION_NAME = CompressedBigDecimalMaxSqlAggregator.NAME;
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return new DruidOperatorTable(ImmutableSet.of(new CompressedBigDecimalMaxSqlAggregator()), ImmutableSet.of());
- }
-
@Override
public void testCompressedBigDecimalAggWithNumberParse()
{
-
testCompressedBigDecimalAggWithNumberParseHelper(
FUNCTION_NAME,
new Object[]{"6.000000000", "6.000000000", "10.100000000"},
diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregatorTest.java
index 0031780ee300..3a18dd49652b 100644
--- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregatorTest.java
+++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregatorTest.java
@@ -19,19 +19,10 @@
package org.apache.druid.compressedbigdecimal;
-import com.google.common.collect.ImmutableSet;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-
public class CompressedBigDecimalMinSqlAggregatorTest extends CompressedBigDecimalSqlAggregatorTestBase
{
private static final String FUNCTION_NAME = CompressedBigDecimalMinSqlAggregator.NAME;
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return new DruidOperatorTable(ImmutableSet.of(new CompressedBigDecimalMinSqlAggregator()), ImmutableSet.of());
- }
-
@Override
public void testCompressedBigDecimalAggWithNumberParse()
{
diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java
index 9f661b7d3987..5d1c67c2f38a 100644
--- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java
+++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java
@@ -20,17 +20,17 @@
package org.apache.druid.compressedbigdecimal;
import com.fasterxml.jackson.databind.MapperFeature;
-import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.MapInputRowParser;
import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Druids;
@@ -45,7 +45,6 @@
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
@@ -74,16 +73,17 @@ public abstract class CompressedBigDecimalSqlAggregatorTestBase extends BaseCalc
TestDataBuilder.RAW_ROWS1.stream().map(m -> TestDataBuilder.createRow(m, PARSER)).collect(Collectors.toList());
@Override
- public Iterable extends Module> getJacksonModules()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- CompressedBigDecimalModule bigDecimalModule = new CompressedBigDecimalModule();
- return Iterables.concat(super.getJacksonModules(), bigDecimalModule.getJacksonModules());
+ super.configureGuice(builder);
+ builder.addModule(new CompressedBigDecimalModule());
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
QueryableIndex index =
@@ -121,9 +121,6 @@ public void configureJsonMapper(ObjectMapper objectMapper)
objectMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true);
}
- @Override
- public abstract DruidOperatorTable createOperatorTable();
-
@Test
public abstract void testCompressedBigDecimalAggWithNumberParse();
@@ -279,5 +276,4 @@ public void testCompressedBigDecimalAggDefaultSizeAndScaleHelper(
ImmutableList.of(expectedResults)
);
}
-
}
diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregatorTest.java
index c7bc86d15438..db248ef67f0e 100644
--- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregatorTest.java
+++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregatorTest.java
@@ -19,19 +19,10 @@
package org.apache.druid.compressedbigdecimal;
-import com.google.common.collect.ImmutableSet;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
-
public class CompressedBigDecimalSumSqlAggregatorTest extends CompressedBigDecimalSqlAggregatorTestBase
{
private static final String FUNCTION_NAME = CompressedBigDecimalSumSqlAggregator.NAME;
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return new DruidOperatorTable(ImmutableSet.of(new CompressedBigDecimalSumSqlAggregator()), ImmutableSet.of());
- }
-
@Override
public void testCompressedBigDecimalAggWithNumberParse()
{
@@ -39,7 +30,6 @@ public void testCompressedBigDecimalAggWithNumberParse()
FUNCTION_NAME,
new Object[]{"21.000000000", "21.000000000", "13.100000000"},
CompressedBigDecimalSumAggregatorFactory::new
-
);
}
diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml
index 7dd0b77f69be..35ffdd9510d2 100644
--- a/extensions-contrib/distinctcount/pom.xml
+++ b/extensions-contrib/distinctcount/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/dropwizard-emitter/pom.xml b/extensions-contrib/dropwizard-emitter/pom.xml
index 714ac2090d9a..0ebe49754660 100644
--- a/extensions-contrib/dropwizard-emitter/pom.xml
+++ b/extensions-contrib/dropwizard-emitter/pom.xml
@@ -24,7 +24,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/gce-extensions/pom.xml b/extensions-contrib/gce-extensions/pom.xml
index 3a3fb73b1be0..aebb755c0b1d 100644
--- a/extensions-contrib/gce-extensions/pom.xml
+++ b/extensions-contrib/gce-extensions/pom.xml
@@ -21,7 +21,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml
index e44036db513f..44759f8f4653 100644
--- a/extensions-contrib/graphite-emitter/pom.xml
+++ b/extensions-contrib/graphite-emitter/pom.xml
@@ -24,7 +24,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/influx-extensions/pom.xml b/extensions-contrib/influx-extensions/pom.xml
index 326cea69f81e..aab1f8c05cca 100644
--- a/extensions-contrib/influx-extensions/pom.xml
+++ b/extensions-contrib/influx-extensions/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/influxdb-emitter/pom.xml b/extensions-contrib/influxdb-emitter/pom.xml
index 01381b799c32..12dc1b34bba8 100644
--- a/extensions-contrib/influxdb-emitter/pom.xml
+++ b/extensions-contrib/influxdb-emitter/pom.xml
@@ -28,7 +28,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml
index 38ef7efcf5ba..b1f575d9020d 100644
--- a/extensions-contrib/kafka-emitter/pom.xml
+++ b/extensions-contrib/kafka-emitter/pom.xml
@@ -24,7 +24,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/kubernetes-overlord-extensions/pom.xml b/extensions-contrib/kubernetes-overlord-extensions/pom.xml
index ffd94574111d..9503eeeb6ced 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/pom.xml
+++ b/extensions-contrib/kubernetes-overlord-extensions/pom.xml
@@ -30,7 +30,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/materialized-view-maintenance/pom.xml b/extensions-contrib/materialized-view-maintenance/pom.xml
index 449d2324aa7c..c8057bc78c57 100644
--- a/extensions-contrib/materialized-view-maintenance/pom.xml
+++ b/extensions-contrib/materialized-view-maintenance/pom.xml
@@ -22,7 +22,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/materialized-view-selection/pom.xml b/extensions-contrib/materialized-view-selection/pom.xml
index c65bccd0bc1b..d23344f1fd02 100644
--- a/extensions-contrib/materialized-view-selection/pom.xml
+++ b/extensions-contrib/materialized-view-selection/pom.xml
@@ -22,7 +22,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/momentsketch/pom.xml b/extensions-contrib/momentsketch/pom.xml
index 1d6afba02d1a..08843ed1669f 100644
--- a/extensions-contrib/momentsketch/pom.xml
+++ b/extensions-contrib/momentsketch/pom.xml
@@ -22,7 +22,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml
index d53c79b3f963..f686cdfc6a4e 100644
--- a/extensions-contrib/moving-average-query/pom.xml
+++ b/extensions-contrib/moving-average-query/pom.xml
@@ -24,7 +24,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/opentelemetry-emitter/pom.xml b/extensions-contrib/opentelemetry-emitter/pom.xml
index 5a1c4027e7be..8bdb7b3dd5d8 100644
--- a/extensions-contrib/opentelemetry-emitter/pom.xml
+++ b/extensions-contrib/opentelemetry-emitter/pom.xml
@@ -23,7 +23,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/opentsdb-emitter/pom.xml b/extensions-contrib/opentsdb-emitter/pom.xml
index 18fd9c61442d..2d68ecf1bca9 100644
--- a/extensions-contrib/opentsdb-emitter/pom.xml
+++ b/extensions-contrib/opentsdb-emitter/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml
index 63e7fba6d528..8d5b13aa7281 100644
--- a/extensions-contrib/prometheus-emitter/pom.xml
+++ b/extensions-contrib/prometheus-emitter/pom.xml
@@ -23,7 +23,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml
index f7ec74e8b28b..e6551e6b792b 100644
--- a/extensions-contrib/redis-cache/pom.xml
+++ b/extensions-contrib/redis-cache/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml
index 67c78fbb81c6..388f40cb2955 100644
--- a/extensions-contrib/sqlserver-metadata-storage/pom.xml
+++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml
@@ -28,7 +28,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml
index e4aad827e271..ea0f5ba231e9 100644
--- a/extensions-contrib/statsd-emitter/pom.xml
+++ b/extensions-contrib/statsd-emitter/pom.xml
@@ -21,7 +21,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/tdigestsketch/pom.xml b/extensions-contrib/tdigestsketch/pom.xml
index 7d271fb06708..76cf7b4055d0 100644
--- a/extensions-contrib/tdigestsketch/pom.xml
+++ b/extensions-contrib/tdigestsketch/pom.xml
@@ -22,7 +22,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java
index ae8c5bfe4683..b304dba196a3 100644
--- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java
+++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java
@@ -19,11 +19,10 @@
package org.apache.druid.query.aggregation.tdigestsketch.sql;
-import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Druids;
@@ -50,7 +49,6 @@
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
@@ -64,21 +62,18 @@
public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
{
- private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
- ImmutableSet.of(new TDigestSketchQuantileSqlAggregator(), new TDigestGenerateSketchSqlAggregator()),
- ImmutableSet.of()
- );
-
@Override
- public Iterable extends Module> getJacksonModules()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- return Iterables.concat(super.getJacksonModules(), new TDigestSketchModule().getJacksonModules());
+ super.configureGuice(builder);
+ builder.addModule(new TDigestSketchModule());
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
TDigestSketchModule.registerSerde();
@@ -116,12 +111,6 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
);
}
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return OPERATOR_TABLE;
- }
-
@Test
public void testComputingSketchOnNumericValues()
{
diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml
index 70aef1f61a93..5d61693a2868 100644
--- a/extensions-contrib/thrift-extensions/pom.xml
+++ b/extensions-contrib/thrift-extensions/pom.xml
@@ -28,7 +28,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml
index a11c79367930..d4767a94f30a 100644
--- a/extensions-contrib/time-min-max/pom.xml
+++ b/extensions-contrib/time-min-max/pom.xml
@@ -21,7 +21,7 @@
druid
org.apache.druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
4.0.0
diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml
index abdc8606e45d..7ae88be798cf 100644
--- a/extensions-contrib/virtual-columns/pom.xml
+++ b/extensions-contrib/virtual-columns/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml
index fcabcd7c9daa..53dff82a980c 100644
--- a/extensions-core/avro-extensions/pom.xml
+++ b/extensions-core/avro-extensions/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
@@ -254,6 +254,11 @@
mockito-core
test
+
+ org.hamcrest
+ hamcrest-core
+ test
+
org.apache.druid
druid-core
diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java
index f7006cb02c21..05c5c9fec2fe 100644
--- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java
+++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java
@@ -140,17 +140,17 @@ public GenericRecord parse(ByteBuffer bytes)
schema = parsedSchema instanceof AvroSchema ? ((AvroSchema) parsedSchema).rawSchema() : null;
}
catch (IOException | RestClientException ex) {
- throw new ParseException(null, "Failed to get Avro schema: %s", id);
+ throw new ParseException(null, ex, "Failed to fetch Avro schema from registry: %s", id);
}
if (schema == null) {
- throw new ParseException(null, "Failed to find Avro schema: %s", id);
+ throw new ParseException(null, "No Avro schema in registry: %s", id);
}
DatumReader reader = new GenericDatumReader<>(schema);
try {
return reader.read(null, DecoderFactory.get().binaryDecoder(bytes.array(), offset, length, null));
}
catch (Exception e) {
- throw new ParseException(null, e, "Fail to decode Avro message for schema: %s!", id);
+ throw new ParseException(null, e, "Failed to decode Avro message for schema: %s", id);
}
}
diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java
index 9348e9486aef..7644b61bb106 100644
--- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java
+++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java
@@ -35,6 +35,8 @@
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.utils.DynamicConfigProviderUtils;
+import org.hamcrest.CoreMatchers;
+import org.hamcrest.MatcherAssert;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -158,7 +160,7 @@ public void testParseWrongSchemaType() throws Exception
new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb);
}
- @Test(expected = ParseException.class)
+ @Test
public void testParseWrongId() throws Exception
{
// Given
@@ -166,7 +168,12 @@ public void testParseWrongId() throws Exception
ByteBuffer bb = ByteBuffer.allocate(5).put((byte) 0).putInt(1234);
bb.rewind();
// When
- new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb);
+ final ParseException e = Assert.assertThrows(
+ ParseException.class,
+ () -> new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb)
+ );
+ MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IOException.class));
+ MatcherAssert.assertThat(e.getCause().getMessage(), CoreMatchers.containsString("no pasaran"));
}
private byte[] getAvroDatum(Schema schema, GenericRecord someAvroDatum) throws IOException
diff --git a/extensions-core/azure-extensions/pom.xml b/extensions-core/azure-extensions/pom.xml
index 6a01cea8a316..e34bacbe69ef 100644
--- a/extensions-core/azure-extensions/pom.xml
+++ b/extensions-core/azure-extensions/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml
index a37efd2dc215..83101d2e7c01 100644
--- a/extensions-core/datasketches/pom.xml
+++ b/extensions-core/datasketches/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java
index c8ac48ab186e..1063bbdfec1a 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeComplexMetricSerde.java
@@ -28,6 +28,7 @@
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.data.GenericIndexed;
import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.data.SafeWritableMemory;
import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
import org.apache.druid.segment.serde.ComplexMetricExtractor;
import org.apache.druid.segment.serde.ComplexMetricSerde;
@@ -70,7 +71,7 @@ public HllSketch extractValue(final InputRow inputRow, final String metricName)
if (object == null) {
return null;
}
- return deserializeSketch(object);
+ return deserializeSketchSafe(object);
}
};
}
@@ -98,6 +99,18 @@ static HllSketch deserializeSketch(final Object object)
throw new IAE("Object is not of a type that can be deserialized to an HllSketch:" + object.getClass().getName());
}
+ static HllSketch deserializeSketchSafe(final Object object)
+ {
+ if (object instanceof String) {
+ return HllSketch.wrap(SafeWritableMemory.wrap(StringUtils.decodeBase64(((String) object).getBytes(StandardCharsets.UTF_8))));
+ } else if (object instanceof byte[]) {
+ return HllSketch.wrap(SafeWritableMemory.wrap((byte[]) object));
+ } else if (object instanceof HllSketch) {
+ return (HllSketch) object;
+ }
+ throw new IAE("Object is not of a type that can be deserialized to an HllSketch:" + object.getClass().getName());
+ }
+
// support large columns
@Override
public GenericColumnSerializer getSerializer(final SegmentWriteOutMedium segmentWriteOutMedium, final String column)
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java
index 5044068262d8..2a07c70ec639 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java
@@ -44,7 +44,6 @@
*/
public class HllSketchModule implements DruidModule
{
-
public static final String TYPE_NAME = "HLLSketch"; // common type name to be associated with segment data
public static final String BUILD_TYPE_NAME = "HLLSketchBuild";
public static final String MERGE_TYPE_NAME = "HLLSketchMerge";
@@ -53,7 +52,6 @@ public class HllSketchModule implements DruidModule
public static final String ESTIMATE_WITH_BOUNDS_TYPE_NAME = "HLLSketchEstimateWithBounds";
public static final String ESTIMATE_TYPE_NAME = "HLLSketchEstimate";
-
@Override
public void configure(final Binder binder)
{
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java
index 34145863fdf8..65257b22b796 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java
@@ -22,7 +22,9 @@
import org.apache.datasketches.hll.HllSketch;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.data.SafeWritableMemory;
+import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@@ -55,4 +57,12 @@ public byte[] toBytes(final HllSketch sketch)
return sketch.toCompactByteArray();
}
+ @Nullable
+ @Override
+ public HllSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes)
+ {
+ return HllSketch.wrap(
+ SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java
index 4c18a9785607..e5249853ac34 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerde.java
@@ -91,7 +91,7 @@ public Object extractValue(final InputRow inputRow, final String metricName)
if (object == null || object instanceof KllDoublesSketch || object instanceof Memory) {
return object;
}
- return KllDoublesSketchOperations.deserialize(object);
+ return KllDoublesSketchOperations.deserializeSafe(object);
}
};
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java
index 97e670a625a5..17cb94e2fcf8 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchObjectStrategy.java
@@ -23,7 +23,9 @@
import org.apache.datasketches.kll.KllDoublesSketch;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.data.SafeWritableMemory;
+import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@@ -60,4 +62,15 @@ public byte[] toBytes(final KllDoublesSketch sketch)
return sketch.toByteArray();
}
+ @Nullable
+ @Override
+ public KllDoublesSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes)
+ {
+ if (numBytes == 0) {
+ return KllDoublesSketchOperations.EMPTY_SKETCH;
+ }
+ return KllDoublesSketch.wrap(
+ SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java
index 57cb51747101..6da454d7f868 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperations.java
@@ -23,6 +23,7 @@
import org.apache.datasketches.memory.Memory;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.data.SafeWritableMemory;
import java.nio.charset.StandardCharsets;
@@ -46,6 +47,16 @@ public static KllDoublesSketch deserialize(final Object serializedSketch)
);
}
+ public static KllDoublesSketch deserializeSafe(final Object serializedSketch)
+ {
+ if (serializedSketch instanceof String) {
+ return deserializeFromBase64EncodedStringSafe((String) serializedSketch);
+ } else if (serializedSketch instanceof byte[]) {
+ return deserializeFromByteArraySafe((byte[]) serializedSketch);
+ }
+ return deserialize(serializedSketch);
+ }
+
public static KllDoublesSketch deserializeFromBase64EncodedString(final String str)
{
return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
@@ -56,4 +67,14 @@ public static KllDoublesSketch deserializeFromByteArray(final byte[] data)
return KllDoublesSketch.wrap(Memory.wrap(data));
}
+ public static KllDoublesSketch deserializeFromBase64EncodedStringSafe(final String str)
+ {
+ return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+ }
+
+ public static KllDoublesSketch deserializeFromByteArraySafe(final byte[] data)
+ {
+ return KllDoublesSketch.wrap(SafeWritableMemory.wrap(data));
+ }
+
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java
index 4a71befe0c66..175b307ec33e 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerde.java
@@ -91,7 +91,7 @@ public Object extractValue(final InputRow inputRow, final String metricName)
if (object == null || object instanceof KllFloatsSketch || object instanceof Memory) {
return object;
}
- return KllFloatsSketchOperations.deserialize(object);
+ return KllFloatsSketchOperations.deserializeSafe(object);
}
};
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java
index ff177a2f54f1..93ff0a7dba52 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchObjectStrategy.java
@@ -23,7 +23,9 @@
import org.apache.datasketches.kll.KllFloatsSketch;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.data.SafeWritableMemory;
+import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@@ -60,4 +62,15 @@ public byte[] toBytes(final KllFloatsSketch sketch)
return sketch.toByteArray();
}
+ @Nullable
+ @Override
+ public KllFloatsSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes)
+ {
+ if (numBytes == 0) {
+ return KllFloatsSketchOperations.EMPTY_SKETCH;
+ }
+ return KllFloatsSketch.wrap(
+ SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java
index e32b67b2549a..02fb615da4dc 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperations.java
@@ -23,6 +23,7 @@
import org.apache.datasketches.memory.Memory;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.data.SafeWritableMemory;
import java.nio.charset.StandardCharsets;
@@ -46,6 +47,16 @@ public static KllFloatsSketch deserialize(final Object serializedSketch)
);
}
+ public static KllFloatsSketch deserializeSafe(final Object serializedSketch)
+ {
+ if (serializedSketch instanceof String) {
+ return deserializeFromBase64EncodedStringSafe((String) serializedSketch);
+ } else if (serializedSketch instanceof byte[]) {
+ return deserializeFromByteArraySafe((byte[]) serializedSketch);
+ }
+ return deserialize(serializedSketch);
+ }
+
public static KllFloatsSketch deserializeFromBase64EncodedString(final String str)
{
return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
@@ -56,4 +67,14 @@ public static KllFloatsSketch deserializeFromByteArray(final byte[] data)
return KllFloatsSketch.wrap(Memory.wrap(data));
}
+ public static KllFloatsSketch deserializeFromBase64EncodedStringSafe(final String str)
+ {
+ return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+ }
+
+ public static KllFloatsSketch deserializeFromByteArraySafe(final byte[] data)
+ {
+ return KllFloatsSketch.wrap(SafeWritableMemory.wrap(data));
+ }
+
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java
index d97b5f8c6dd3..3614f214c708 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java
@@ -92,7 +92,7 @@ public Object extractValue(final InputRow inputRow, final String metricName)
if (object == null || object instanceof DoublesSketch || object instanceof Memory) {
return object;
}
- return DoublesSketchOperations.deserialize(object);
+ return DoublesSketchOperations.deserializeSafe(object);
}
};
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java
index cd047695f45d..7896e1c55c10 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java
@@ -43,7 +43,6 @@
public class DoublesSketchModule implements DruidModule
{
-
public static final String DOUBLES_SKETCH = "quantilesDoublesSketch";
public static final String DOUBLES_SKETCH_MERGE = "quantilesDoublesSketchMerge";
public static final ColumnType TYPE = ColumnType.ofComplex(DOUBLES_SKETCH);
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java
index 826de9378f8c..569b60bf03f7 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java
@@ -23,7 +23,9 @@
import org.apache.datasketches.memory.Memory;
import org.apache.datasketches.quantiles.DoublesSketch;
import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.data.SafeWritableMemory;
+import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@@ -60,4 +62,15 @@ public byte[] toBytes(final DoublesSketch sketch)
return sketch.toByteArray(true);
}
+ @Nullable
+ @Override
+ public DoublesSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes)
+ {
+ if (numBytes == 0) {
+ return DoublesSketchOperations.EMPTY_SKETCH;
+ }
+ return DoublesSketch.wrap(
+ SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java
index e30fb9bdae83..a2ca197c11ac 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperations.java
@@ -23,6 +23,7 @@
import org.apache.datasketches.quantiles.DoublesSketch;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.data.SafeWritableMemory;
import java.nio.charset.StandardCharsets;
@@ -46,6 +47,16 @@ public static DoublesSketch deserialize(final Object serializedSketch)
);
}
+ public static DoublesSketch deserializeSafe(final Object serializedSketch)
+ {
+ if (serializedSketch instanceof String) {
+ return deserializeFromBase64EncodedStringSafe((String) serializedSketch);
+ } else if (serializedSketch instanceof byte[]) {
+ return deserializeFromByteArraySafe((byte[]) serializedSketch);
+ }
+ return deserialize(serializedSketch);
+ }
+
public static DoublesSketch deserializeFromBase64EncodedString(final String str)
{
return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
@@ -56,4 +67,13 @@ public static DoublesSketch deserializeFromByteArray(final byte[] data)
return DoublesSketch.wrap(Memory.wrap(data));
}
+ public static DoublesSketch deserializeFromBase64EncodedStringSafe(final String str)
+ {
+ return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+ }
+
+ public static DoublesSketch deserializeFromByteArraySafe(final byte[] data)
+ {
+ return DoublesSketch.wrap(SafeWritableMemory.wrap(data));
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java
index b3541bd50649..64c182a0d62a 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java
@@ -51,7 +51,7 @@ public SketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProp
Preconditions.checkArgument(value != null && !value.isEmpty(),
"Constant value cannot be null or empty, expecting base64 encoded sketch string");
this.value = value;
- this.sketchValue = SketchHolder.deserialize(value);
+ this.sketchValue = SketchHolder.deserializeSafe(value);
}
@Override
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java
index 59ca453bb25a..838b4ae91f0f 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolder.java
@@ -34,6 +34,7 @@
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.data.SafeWritableMemory;
import javax.annotation.Nullable;
@@ -224,6 +225,17 @@ public static SketchHolder deserialize(Object serializedSketch)
);
}
+ public static SketchHolder deserializeSafe(Object serializedSketch)
+ {
+ if (serializedSketch instanceof String) {
+ return SketchHolder.of(deserializeFromBase64EncodedStringSafe((String) serializedSketch));
+ } else if (serializedSketch instanceof byte[]) {
+ return SketchHolder.of(deserializeFromByteArraySafe((byte[]) serializedSketch));
+ }
+
+ return deserialize(serializedSketch);
+ }
+
private static Sketch deserializeFromBase64EncodedString(String str)
{
return deserializeFromByteArray(StringUtils.decodeBase64(StringUtils.toUtf8(str)));
@@ -234,6 +246,16 @@ private static Sketch deserializeFromByteArray(byte[] data)
return deserializeFromMemory(Memory.wrap(data));
}
+ private static Sketch deserializeFromBase64EncodedStringSafe(String str)
+ {
+ return deserializeFromByteArraySafe(StringUtils.decodeBase64(StringUtils.toUtf8(str)));
+ }
+
+ private static Sketch deserializeFromByteArraySafe(byte[] data)
+ {
+ return deserializeFromMemory(SafeWritableMemory.wrap(data));
+ }
+
private static Sketch deserializeFromMemory(Memory mem)
{
if (Sketch.getSerializationVersion(mem) < 3) {
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java
index e98bc3d95a04..96fafe826211 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategy.java
@@ -23,6 +23,7 @@
import org.apache.datasketches.memory.Memory;
import org.apache.datasketches.theta.Sketch;
import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.data.SafeWritableMemory;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@@ -66,4 +67,17 @@ public byte[] toBytes(@Nullable SketchHolder obj)
return ByteArrays.EMPTY_ARRAY;
}
}
+
+ @Nullable
+ @Override
+ public SketchHolder fromByteBufferSafe(ByteBuffer buffer, int numBytes)
+ {
+ if (numBytes == 0) {
+ return SketchHolder.EMPTY;
+ }
+
+ return SketchHolder.of(
+ SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java
index a824312c0ef7..4f3ecfae2915 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java
@@ -59,7 +59,7 @@ public Class extractedClass()
public SketchHolder extractValue(InputRow inputRow, String metricName)
{
final Object object = inputRow.getRaw(metricName);
- return object == null ? null : SketchHolder.deserialize(object);
+ return object == null ? null : SketchHolder.deserializeSafe(object);
}
};
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSetBaseOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSetBaseOperatorConversion.java
index 0afad6cbe6b1..62c686cd2b2a 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSetBaseOperatorConversion.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSetBaseOperatorConversion.java
@@ -63,6 +63,8 @@ public DruidExpression toDruidExpression(
RexNode rexNode
)
{
+ plannerContext.setPlanningError("%s can only be used on aggregates. " +
+ "It cannot be used directly on a column or on a scalar expression.", getFunctionName());
return null;
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java
index 19c8da292b4e..028bcdc35491 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchMergeComplexMetricSerde.java
@@ -60,7 +60,7 @@ public Object extractValue(final InputRow inputRow, final String metricName)
if (object == null || object instanceof ArrayOfDoublesSketch) {
return object;
}
- return ArrayOfDoublesSketchOperations.deserialize(object);
+ return ArrayOfDoublesSketchOperations.deserializeSafe(object);
}
};
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java
index 1ae950e068f8..f893c83b5707 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategy.java
@@ -23,6 +23,7 @@
import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.data.SafeWritableMemory;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@@ -48,7 +49,9 @@ public Class getClazz()
@Override
public ArrayOfDoublesSketch fromByteBuffer(final ByteBuffer buffer, final int numBytes)
{
- return ArrayOfDoublesSketches.wrapSketch(Memory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes));
+ return ArrayOfDoublesSketches.wrapSketch(
+ Memory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)
+ );
}
@Override
@@ -61,4 +64,12 @@ public byte[] toBytes(@Nullable final ArrayOfDoublesSketch sketch)
return sketch.toByteArray();
}
+ @Nullable
+ @Override
+ public ArrayOfDoublesSketch fromByteBufferSafe(ByteBuffer buffer, int numBytes)
+ {
+ return ArrayOfDoublesSketches.wrapSketch(
+ SafeWritableMemory.wrap(buffer, ByteOrder.LITTLE_ENDIAN).region(buffer.position(), numBytes)
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java
index b1658a995714..2768858ffe32 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperations.java
@@ -30,6 +30,7 @@
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.data.SafeWritableMemory;
import java.nio.charset.StandardCharsets;
@@ -115,6 +116,17 @@ public static ArrayOfDoublesSketch deserialize(final Object serializedSketch)
throw new ISE("Object is not of a type that can deserialize to sketch: %s", serializedSketch.getClass());
}
+ public static ArrayOfDoublesSketch deserializeSafe(final Object serializedSketch)
+ {
+ if (serializedSketch instanceof String) {
+ return deserializeFromBase64EncodedStringSafe((String) serializedSketch);
+ } else if (serializedSketch instanceof byte[]) {
+ return deserializeFromByteArraySafe((byte[]) serializedSketch);
+ }
+
+ return deserialize(serializedSketch);
+ }
+
public static ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str)
{
return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
@@ -122,8 +134,16 @@ public static ArrayOfDoublesSketch deserializeFromBase64EncodedString(final Stri
public static ArrayOfDoublesSketch deserializeFromByteArray(final byte[] data)
{
- final Memory mem = Memory.wrap(data);
- return ArrayOfDoublesSketches.wrapSketch(mem);
+ return ArrayOfDoublesSketches.wrapSketch(Memory.wrap(data));
+ }
+
+ public static ArrayOfDoublesSketch deserializeFromBase64EncodedStringSafe(final String str)
+ {
+ return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
}
+ public static ArrayOfDoublesSketch deserializeFromByteArraySafe(final byte[] data)
+ {
+ return ArrayOfDoublesSketches.wrapSketch(SafeWritableMemory.wrap(data));
+ }
}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategyTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategyTest.java
new file mode 100644
index 000000000000..ff1eb947408e
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategyTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.query.aggregation.datasketches.hll;
+
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.hll.HllSketch;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class HllSketchObjectStrategyTest
+{
+ @Test
+ public void testSafeRead()
+ {
+ HllSketch sketch = new HllSketch();
+ sketch.update(new int[]{1, 2, 3});
+
+ final byte[] bytes = sketch.toCompactByteArray();
+
+ ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
+ HllSketchObjectStrategy objectStrategy = new HllSketchObjectStrategy();
+
+ // valid sketch should not explode when copied, which reads the memory
+ objectStrategy.fromByteBufferSafe(buf, bytes.length).copy();
+
+ // corrupted sketch should fail with a regular java buffer exception
+ for (int subset = 3; subset < bytes.length - 1; subset++) {
+ final byte[] garbage2 = new byte[subset];
+ for (int i = 0; i < garbage2.length; i++) {
+ garbage2[i] = buf.get(i);
+ }
+
+ final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).copy()
+ );
+ }
+
+ // non sketch that is too short to contain header should fail with regular java buffer exception
+ final byte[] garbage = new byte[]{0x01, 0x02};
+ final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).copy()
+ );
+
+ // non sketch that is long enough to check (this one doesn't actually need 'safe' read)
+ final byte[] garbageLonger = StringUtils.toUtf8("notasketch");
+ final ByteBuffer buf4 = ByteBuffer.wrap(garbageLonger).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ SketchesArgumentException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf4, garbageLonger.length).copy()
+ );
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
index 86e76ca8c219..aa0e9afd13a1 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
@@ -19,12 +19,11 @@
package org.apache.druid.query.aggregation.datasketches.hll.sql;
-import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
@@ -61,13 +60,11 @@
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
-import org.apache.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
-import org.apache.druid.sql.calcite.aggregation.builtin.CountSqlAggregator;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
+import org.apache.druid.sql.guice.SqlModule;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.joda.time.DateTimeZone;
@@ -78,22 +75,34 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
+import java.util.Properties;
public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
{
private static final boolean ROUND = true;
@Override
- public Iterable extends Module> getJacksonModules()
+ public void gatherProperties(Properties properties)
{
- return Iterables.concat(super.getJacksonModules(), new HllSketchModule().getJacksonModules());
+ super.gatherProperties(properties);
+
+ // Use APPROX_COUNT_DISTINCT_DS_HLL as APPROX_COUNT_DISTINCT impl for these tests.
+ properties.put(SqlModule.PROPERTY_SQL_APPROX_COUNT_DISTINCT_CHOICE, HllSketchApproxCountDistinctSqlAggregator.NAME);
+ }
+
+ @Override
+ public void configureGuice(DruidInjectorBuilder builder)
+ {
+ super.configureGuice(builder);
+ builder.addModule(new HllSketchModule());
}
@SuppressWarnings("resource")
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
HllSketchModule.registerSerde();
@@ -132,30 +141,6 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
);
}
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- final HllSketchApproxCountDistinctSqlAggregator approxCountDistinctSqlAggregator =
- new HllSketchApproxCountDistinctSqlAggregator();
-
- return new DruidOperatorTable(
- ImmutableSet.of(
- approxCountDistinctSqlAggregator,
- new HllSketchObjectSqlAggregator(),
-
- // Use APPROX_COUNT_DISTINCT_DS_HLL as APPROX_COUNT_DISTINCT impl for these tests.
- new CountSqlAggregator(new ApproxCountDistinctSqlAggregator(approxCountDistinctSqlAggregator)),
- new ApproxCountDistinctSqlAggregator(approxCountDistinctSqlAggregator)
- ),
- ImmutableSet.of(
- new HllSketchSetUnionOperatorConversion(),
- new HllSketchEstimateOperatorConversion(),
- new HllSketchToStringOperatorConversion(),
- new HllSketchEstimateWithErrorBoundsOperatorConversion()
- )
- );
- }
-
@Test
public void testApproxCountDistinctHllSketch()
{
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java
index 3628c5e6212f..0ae46bef496b 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchComplexMetricSerdeTest.java
@@ -23,10 +23,14 @@
import com.google.common.collect.ImmutableMap;
import org.apache.datasketches.kll.KllDoublesSketch;
import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.segment.data.ObjectStrategy;
import org.apache.druid.segment.serde.ComplexMetricExtractor;
import org.junit.Assert;
import org.junit.Test;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
public class KllDoublesSketchComplexMetricSerdeTest
{
@Test
@@ -92,4 +96,44 @@ public void testExtractorOnLeadingDecimalNumber()
Assert.assertEquals(1, sketch.getNumRetained());
Assert.assertEquals(0.1d, sketch.getMaxValue(), 0.01d);
}
+
+ @Test
+ public void testSafeRead()
+ {
+ final KllDoublesSketchComplexMetricSerde serde = new KllDoublesSketchComplexMetricSerde();
+ final ObjectStrategy objectStrategy = serde.getObjectStrategy();
+
+ KllDoublesSketch sketch = KllDoublesSketch.newHeapInstance();
+ sketch.update(1.1);
+ sketch.update(1.2);
+ final byte[] bytes = sketch.toByteArray();
+
+ ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
+
+ // valid sketch should not explode when converted to byte array, which reads the memory
+ objectStrategy.fromByteBufferSafe(buf, bytes.length).toByteArray();
+
+ // corrupted sketch should fail with a regular java buffer exception, not all subsets actually fail with the same
+ // index out of bounds exceptions, but at least this many do
+ for (int subset = 3; subset < 24; subset++) {
+ final byte[] garbage2 = new byte[subset];
+ for (int i = 0; i < garbage2.length; i++) {
+ garbage2[i] = buf.get(i);
+ }
+
+ final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray()
+ );
+ }
+
+ // non sketch that is too short to contain header should fail with regular java buffer exception
+ final byte[] garbage = new byte[]{0x01, 0x02};
+ final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray()
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperationsTest.java
new file mode 100644
index 000000000000..d2b0e3839848
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchOperationsTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.query.aggregation.datasketches.kll;
+
+import org.apache.datasketches.kll.KllDoublesSketch;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class KllDoublesSketchOperationsTest
+{
+ @Test
+ public void testDeserializeSafe()
+ {
+ KllDoublesSketch sketch = KllDoublesSketch.newHeapInstance();
+ sketch.update(1.1);
+ sketch.update(1.2);
+ final byte[] bytes = sketch.toByteArray();
+ final String base64 = StringUtils.encodeBase64String(bytes);
+
+ Assert.assertArrayEquals(bytes, KllDoublesSketchOperations.deserializeSafe(sketch).toByteArray());
+ Assert.assertArrayEquals(bytes, KllDoublesSketchOperations.deserializeSafe(bytes).toByteArray());
+ Assert.assertArrayEquals(bytes, KllDoublesSketchOperations.deserializeSafe(base64).toByteArray());
+
+ final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 20);
+ Assert.assertThrows(IndexOutOfBoundsException.class, () -> KllDoublesSketchOperations.deserializeSafe(trunacted));
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> KllDoublesSketchOperations.deserializeSafe(StringUtils.encodeBase64String(trunacted))
+ );
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java
index 5ff441df1c14..c6b8c310221c 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchComplexMetricSerdeTest.java
@@ -23,10 +23,14 @@
import com.google.common.collect.ImmutableMap;
import org.apache.datasketches.kll.KllFloatsSketch;
import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.segment.data.ObjectStrategy;
import org.apache.druid.segment.serde.ComplexMetricExtractor;
import org.junit.Assert;
import org.junit.Test;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
public class KllFloatsSketchComplexMetricSerdeTest
{
@Test
@@ -92,4 +96,44 @@ public void testExtractorOnLeadingDecimalNumber()
Assert.assertEquals(1, sketch.getNumRetained());
Assert.assertEquals(0.1d, sketch.getMaxValue(), 0.01d);
}
+
+ @Test
+ public void testSafeRead()
+ {
+ final KllFloatsSketchComplexMetricSerde serde = new KllFloatsSketchComplexMetricSerde();
+ final ObjectStrategy objectStrategy = serde.getObjectStrategy();
+
+ KllFloatsSketch sketch = KllFloatsSketch.newHeapInstance();
+ sketch.update(1.1f);
+ sketch.update(1.2f);
+ final byte[] bytes = sketch.toByteArray();
+
+ ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
+
+ // valid sketch should not explode when converted to byte array, which reads the memory
+ objectStrategy.fromByteBufferSafe(buf, bytes.length).toByteArray();
+
+ // corrupted sketch should fail with a regular java buffer exception, not all subsets actually fail with the same
+ // index out of bounds exceptions, but at least this many do
+ for (int subset = 3; subset < 24; subset++) {
+ final byte[] garbage2 = new byte[subset];
+ for (int i = 0; i < garbage2.length; i++) {
+ garbage2[i] = buf.get(i);
+ }
+
+ final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray()
+ );
+ }
+
+ // non sketch that is too short to contain header should fail with regular java buffer exception
+ final byte[] garbage = new byte[]{0x01, 0x02};
+ final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray()
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperationsTest.java
new file mode 100644
index 000000000000..613b38c6601a
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchOperationsTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.query.aggregation.datasketches.kll;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class KllFloatsSketchOperationsTest
+{
+ @Test
+ public void testDeserializeSafe()
+ {
+ KllFloatsSketch sketch = KllFloatsSketch.newHeapInstance();
+ sketch.update(1.1f);
+ sketch.update(1.2f);
+ final byte[] bytes = sketch.toByteArray();
+ final String base64 = StringUtils.encodeBase64String(bytes);
+
+ Assert.assertArrayEquals(bytes, KllFloatsSketchOperations.deserializeSafe(sketch).toByteArray());
+ Assert.assertArrayEquals(bytes, KllFloatsSketchOperations.deserializeSafe(bytes).toByteArray());
+ Assert.assertArrayEquals(bytes, KllFloatsSketchOperations.deserializeSafe(base64).toByteArray());
+
+ final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 20);
+ Assert.assertThrows(IndexOutOfBoundsException.class, () -> KllFloatsSketchOperations.deserializeSafe(trunacted));
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> KllFloatsSketchOperations.deserializeSafe(StringUtils.encodeBase64String(trunacted))
+ );
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java
index e198c7704257..7dc82baee925 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerdeTest.java
@@ -22,11 +22,16 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.datasketches.quantiles.DoublesSketch;
+import org.apache.datasketches.quantiles.DoublesUnion;
import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.segment.data.ObjectStrategy;
import org.apache.druid.segment.serde.ComplexMetricExtractor;
import org.junit.Assert;
import org.junit.Test;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
public class DoublesSketchComplexMetricSerdeTest
{
@Test
@@ -92,4 +97,42 @@ public void testExtractorOnLeadingDecimalNumber()
Assert.assertEquals(1, sketch.getRetainedItems());
Assert.assertEquals(0.1d, sketch.getMaxValue(), 0.01d);
}
+
+ @Test
+ public void testSafeRead()
+ {
+ final DoublesSketchComplexMetricSerde serde = new DoublesSketchComplexMetricSerde();
+ DoublesUnion union = DoublesUnion.builder().setMaxK(1024).build();
+ union.update(1.1);
+ final byte[] bytes = union.toByteArray();
+
+ ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
+ ObjectStrategy objectStrategy = serde.getObjectStrategy();
+
+ // valid sketch should not explode when copied, which reads the memory
+ objectStrategy.fromByteBufferSafe(buf, bytes.length).toByteArray(true);
+
+ // corrupted sketch should fail with a regular java buffer exception
+ for (int subset = 3; subset < 15; subset++) {
+ final byte[] garbage2 = new byte[subset];
+ for (int i = 0; i < garbage2.length; i++) {
+ garbage2[i] = buf.get(i);
+ }
+
+ final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ "i " + subset,
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).toByteArray(true)
+ );
+ }
+
+ // non sketch that is too short to contain header should fail with regular java buffer exception
+ final byte[] garbage = new byte[]{0x01, 0x02};
+ final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).toByteArray(true)
+ );
+ }
}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperationsTest.java
new file mode 100644
index 000000000000..38e5d39a91b8
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchOperationsTest.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.query.aggregation.datasketches.quantiles;
+
+import org.apache.datasketches.quantiles.DoublesUnion;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class DoublesSketchOperationsTest
+{
+ @Test
+ public void testDeserializeSafe()
+ {
+ DoublesUnion union = DoublesUnion.builder().setMaxK(1024).build();
+ union.update(1.1);
+ final byte[] bytes = union.getResult().toByteArray();
+ final String base64 = StringUtils.encodeBase64String(bytes);
+
+ Assert.assertArrayEquals(bytes, DoublesSketchOperations.deserializeSafe(union.getResult()).toByteArray());
+ Assert.assertArrayEquals(bytes, DoublesSketchOperations.deserializeSafe(bytes).toByteArray());
+ Assert.assertArrayEquals(bytes, DoublesSketchOperations.deserializeSafe(base64).toByteArray());
+
+ final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 4);
+ Assert.assertThrows(IndexOutOfBoundsException.class, () -> DoublesSketchOperations.deserializeSafe(trunacted));
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> DoublesSketchOperations.deserializeSafe(StringUtils.encodeBase64(trunacted))
+ );
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java
index e4a66a9a56c8..f2253462bd01 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java
@@ -19,12 +19,11 @@
package org.apache.druid.query.aggregation.datasketches.quantiles.sql;
-import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Druids;
@@ -63,7 +62,6 @@
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
@@ -79,31 +77,18 @@
public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
{
- private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
- ImmutableSet.of(
- new DoublesSketchApproxQuantileSqlAggregator(),
- new DoublesSketchObjectSqlAggregator()
- ),
- ImmutableSet.of(
- new DoublesSketchQuantileOperatorConversion(),
- new DoublesSketchQuantilesOperatorConversion(),
- new DoublesSketchToHistogramOperatorConversion(),
- new DoublesSketchRankOperatorConversion(),
- new DoublesSketchCDFOperatorConversion(),
- new DoublesSketchSummaryOperatorConversion()
- )
- );
-
@Override
- public Iterable extends Module> getJacksonModules()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- return Iterables.concat(super.getJacksonModules(), new DoublesSketchModule().getJacksonModules());
+ super.configureGuice(builder);
+ builder.addModule(new DoublesSketchModule());
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
DoublesSketchModule.registerSerde();
@@ -141,12 +126,6 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
);
}
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return OPERATOR_TABLE;
- }
-
@Test
public void testQuantileOnFloatAndLongs()
{
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategyTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategyTest.java
new file mode 100644
index 000000000000..5619facd5f62
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderObjectStrategyTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.query.aggregation.datasketches.theta;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.theta.SetOperation;
+import org.apache.datasketches.theta.Union;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class SketchHolderObjectStrategyTest
+{
+ @Test
+ public void testSafeRead()
+ {
+ SketchHolderObjectStrategy objectStrategy = new SketchHolderObjectStrategy();
+ Union union = (Union) SetOperation.builder().setNominalEntries(1024).build(Family.UNION);
+ union.update(1234L);
+
+ final byte[] bytes = union.getResult().toByteArray();
+
+ ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
+
+ // valid sketch should not explode when copied, which reads the memory
+ objectStrategy.fromByteBufferSafe(buf, bytes.length).getSketch().compact().getCompactBytes();
+
+ // corrupted sketch should fail with a regular java buffer exception
+ for (int subset = 3; subset < bytes.length - 1; subset++) {
+ final byte[] garbage2 = new byte[subset];
+ for (int i = 0; i < garbage2.length; i++) {
+ garbage2[i] = buf.get(i);
+ }
+
+ final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).getSketch().compact().getCompactBytes()
+ );
+ }
+
+ // non sketch that is too short to contain header should fail with regular java buffer exception
+ final byte[] garbage = new byte[]{0x01, 0x02};
+ final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).getSketch().compact().getCompactBytes()
+ );
+
+ // non sketch that is long enough to check (this one doesn't actually need 'safe' read)
+ final byte[] garbageLonger = StringUtils.toUtf8("notasketch");
+ final ByteBuffer buf4 = ByteBuffer.wrap(garbageLonger).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ SketchesArgumentException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf4, garbageLonger.length).getSketch().compact().getCompactBytes()
+ );
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderTest.java
new file mode 100644
index 000000000000..ef68fdeb8c58
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchHolderTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.query.aggregation.datasketches.theta;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.theta.SetOperation;
+import org.apache.datasketches.theta.Union;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class SketchHolderTest
+{
+ @Test
+ public void testDeserializeSafe()
+ {
+ Union union = (Union) SetOperation.builder().setNominalEntries(1024).build(Family.UNION);
+ union.update(1234L);
+ final byte[] bytes = union.getResult().toByteArray();
+ final String base64 = StringUtils.encodeBase64String(bytes);
+
+ Assert.assertArrayEquals(bytes, SketchHolder.deserializeSafe(union.getResult()).getSketch().toByteArray());
+ Assert.assertArrayEquals(bytes, SketchHolder.deserializeSafe(bytes).getSketch().toByteArray());
+ Assert.assertArrayEquals(bytes, SketchHolder.deserializeSafe(base64).getSketch().toByteArray());
+
+ final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 10);
+ Assert.assertThrows(IndexOutOfBoundsException.class, () -> SketchHolder.deserializeSafe(trunacted));
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> SketchHolder.deserializeSafe(StringUtils.encodeBase64String(trunacted))
+ );
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
index 128ffd9419e3..f5a23178dd02 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
@@ -19,17 +19,14 @@
package org.apache.druid.query.aggregation.datasketches.theta.sql;
-import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.guice.ExpressionModule;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
-import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
@@ -57,58 +54,48 @@
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
-import org.apache.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
-import org.apache.druid.sql.calcite.aggregation.builtin.CountSqlAggregator;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
+import org.apache.druid.sql.guice.SqlModule;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.joda.time.DateTimeZone;
import org.joda.time.Period;
-import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
+import java.util.Properties;
public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
{
private static final String DATA_SOURCE = "foo";
- private ExprMacroTable macroTable;
-
- @Before
- public void setUp()
- {
- macroTable = createMacroTable();
- }
-
@Override
- public Iterable extends Module> getJacksonModules()
+ public void gatherProperties(Properties properties)
{
- return Iterables.concat(super.getJacksonModules(), new SketchModule().getJacksonModules());
+ super.gatherProperties(properties);
+
+ // Use APPROX_COUNT_DISTINCT_DS_THETA as APPROX_COUNT_DISTINCT impl for these tests.
+ properties.put(SqlModule.PROPERTY_SQL_APPROX_COUNT_DISTINCT_CHOICE, ThetaSketchApproxCountDistinctSqlAggregator.NAME);
}
@Override
- public ExprMacroTable createMacroTable()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- final List exprMacros = new ArrayList<>();
- for (Class extends ExprMacroTable.ExprMacro> clazz : ExpressionModule.EXPR_MACROS) {
- exprMacros.add(CalciteTests.INJECTOR.getInstance(clazz));
- }
- return new ExprMacroTable(exprMacros);
+ super.configureGuice(builder);
+ builder.addModule(new SketchModule());
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
SketchModule.registerSerde();
@@ -148,31 +135,6 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
);
}
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- final ThetaSketchApproxCountDistinctSqlAggregator approxCountDistinctSqlAggregator =
- new ThetaSketchApproxCountDistinctSqlAggregator();
-
- return new DruidOperatorTable(
- ImmutableSet.of(
- new ThetaSketchApproxCountDistinctSqlAggregator(),
- new ThetaSketchObjectSqlAggregator(),
-
- // Use APPROX_COUNT_DISTINCT_DS_THETA as APPROX_COUNT_DISTINCT impl for these tests.
- new CountSqlAggregator(new ApproxCountDistinctSqlAggregator(approxCountDistinctSqlAggregator)),
- new ApproxCountDistinctSqlAggregator(approxCountDistinctSqlAggregator)
- ),
- ImmutableSet.of(
- new ThetaSketchEstimateOperatorConversion(),
- new ThetaSketchEstimateWithErrorBoundsOperatorConversion(),
- new ThetaSketchSetIntersectOperatorConversion(),
- new ThetaSketchSetUnionOperatorConversion(),
- new ThetaSketchSetNotOperatorConversion()
- )
- );
- }
-
@Test
public void testApproxCountDistinctThetaSketch()
{
@@ -235,13 +197,13 @@ public void testApproxCountDistinctThetaSketch()
"v0",
"substring(\"dim2\", 0, 1)",
ColumnType.STRING,
- macroTable
+ queryFramework().macroTable()
),
new ExpressionVirtualColumn(
"v1",
"concat(substring(\"dim2\", 0, 1),'x')",
ColumnType.STRING,
- macroTable
+ queryFramework().macroTable()
)
)
.aggregators(
@@ -444,7 +406,7 @@ public void testThetaSketchPostAggs()
"v0",
"concat(\"dim2\",'hello')",
ColumnType.STRING,
- macroTable
+ queryFramework().macroTable()
)
)
.aggregators(
@@ -621,7 +583,7 @@ public void testThetaSketchPostAggsFinalizeOuterSketches()
"v0",
"concat(\"dim2\",'hello')",
ColumnType.STRING,
- macroTable
+ queryFramework().macroTable()
)
)
.aggregators(
@@ -1052,4 +1014,12 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches()
ImmutableList.of(new Object[]{"a", 0L, 0L, "0.0", "0.0"})
);
}
+
+ @Test
+ public void testThetaSketchIntersectOnScalarExpression()
+ {
+ assertQueryIsUnplannable("SELECT THETA_SKETCH_INTERSECT(NULL, NULL) FROM foo",
+ "Possible error: THETA_SKETCH_INTERSECT can only be used on aggregates. " +
+ "It cannot be used directly on a column or on a scalar expression.");
+ }
}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategyTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategyTest.java
new file mode 100644
index 000000000000..ee59ddf57645
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchObjectStrategyTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.query.aggregation.datasketches.tuple;
+
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketchBuilder;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class ArrayOfDoublesSketchObjectStrategyTest
+{
+ @Test
+ public void testSafeRead()
+ {
+ ArrayOfDoublesSketchObjectStrategy objectStrategy = new ArrayOfDoublesSketchObjectStrategy();
+ ArrayOfDoublesUpdatableSketch sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(1024)
+ .setNumberOfValues(4)
+ .build();
+ sketch.update(1L, new double[]{1.0, 2.0, 3.0, 4.0});
+
+ final byte[] bytes = sketch.compact().toByteArray();
+
+ ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
+
+ // valid sketch should not explode when copied, which reads the memory
+ objectStrategy.fromByteBufferSafe(buf, bytes.length).compact().toByteArray();
+
+ // corrupted sketch should fail with a regular java buffer exception
+ for (int subset = 3; subset < bytes.length - 1; subset++) {
+ final byte[] garbage2 = new byte[subset];
+ for (int i = 0; i < garbage2.length; i++) {
+ garbage2[i] = buf.get(i);
+ }
+
+ final ByteBuffer buf2 = ByteBuffer.wrap(garbage2).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf2, garbage2.length).compact().toByteArray()
+ );
+ }
+
+ // non sketch that is too short to contain header should fail with regular java buffer exception
+ final byte[] garbage = new byte[]{0x01, 0x02};
+ final ByteBuffer buf3 = ByteBuffer.wrap(garbage).order(ByteOrder.LITTLE_ENDIAN);
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> objectStrategy.fromByteBufferSafe(buf3, garbage.length).compact().toByteArray()
+ );
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperationsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperationsTest.java
new file mode 100644
index 000000000000..415f3acab970
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchOperationsTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.query.aggregation.datasketches.tuple;
+
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesUpdatableSketchBuilder;
+import org.apache.druid.java.util.common.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class ArrayOfDoublesSketchOperationsTest
+{
+ @Test
+ public void testDeserializeSafe()
+ {
+ ArrayOfDoublesSketchObjectStrategy objectStrategy = new ArrayOfDoublesSketchObjectStrategy();
+ ArrayOfDoublesUpdatableSketch sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(1024)
+ .setNumberOfValues(4)
+ .build();
+ sketch.update(1L, new double[]{1.0, 2.0, 3.0, 4.0});
+
+ final byte[] bytes = sketch.toByteArray();
+ final String base64 = StringUtils.encodeBase64String(bytes);
+
+ Assert.assertArrayEquals(bytes, ArrayOfDoublesSketchOperations.deserializeSafe(sketch).toByteArray());
+ Assert.assertArrayEquals(bytes, ArrayOfDoublesSketchOperations.deserializeSafe(bytes).toByteArray());
+ Assert.assertArrayEquals(bytes, ArrayOfDoublesSketchOperations.deserializeSafe(base64).toByteArray());
+
+ final byte[] trunacted = Arrays.copyOfRange(bytes, 0, 10);
+ Assert.assertThrows(IndexOutOfBoundsException.class, () -> ArrayOfDoublesSketchOperations.deserializeSafe(trunacted));
+ Assert.assertThrows(
+ IndexOutOfBoundsException.class,
+ () -> ArrayOfDoublesSketchOperations.deserializeSafe(StringUtils.encodeBase64String(trunacted))
+ );
+ }
+}
diff --git a/extensions-core/druid-aws-rds-extensions/pom.xml b/extensions-core/druid-aws-rds-extensions/pom.xml
index b18e1aea7214..f242e1e485c5 100644
--- a/extensions-core/druid-aws-rds-extensions/pom.xml
+++ b/extensions-core/druid-aws-rds-extensions/pom.xml
@@ -30,7 +30,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/druid-basic-security/pom.xml b/extensions-core/druid-basic-security/pom.xml
index c8a3ab20e8ee..4bce2588c574 100644
--- a/extensions-core/druid-basic-security/pom.xml
+++ b/extensions-core/druid-basic-security/pom.xml
@@ -30,7 +30,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/druid-bloom-filter/pom.xml b/extensions-core/druid-bloom-filter/pom.xml
index 0bc86e8e271e..4a457f92a129 100644
--- a/extensions-core/druid-bloom-filter/pom.xml
+++ b/extensions-core/druid-bloom-filter/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java
index 26f1ce108fee..c1426048faf3 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java
@@ -19,10 +19,9 @@
package org.apache.druid.query.aggregation.bloom.sql;
-import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.DimensionSchema;
@@ -35,6 +34,7 @@
import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.guice.BloomFilterExtensionModule;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
@@ -59,7 +59,6 @@
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
@@ -76,21 +75,18 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
private static final String DATA_SOURCE = "numfoo";
- private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
- ImmutableSet.of(new BloomFilterSqlAggregator()),
- ImmutableSet.of()
- );
-
@Override
- public Iterable extends Module> getJacksonModules()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- return Iterables.concat(super.getJacksonModules(), new BloomFilterExtensionModule().getJacksonModules());
+ super.configureGuice(builder);
+ builder.addModule(new BloomFilterExtensionModule());
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
InputRowParser parser = new MapInputRowParser(
@@ -135,12 +131,6 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
);
}
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return OPERATOR_TABLE;
- }
-
@Test
public void testBloomFilterAgg() throws Exception
{
@@ -178,7 +168,7 @@ public void testBloomFilterAgg() throws Exception
.build()
),
ImmutableList.of(
- new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+ new Object[]{queryFramework().queryJsonMapper().writeValueAsString(expected1)}
)
);
}
@@ -211,6 +201,7 @@ public void testBloomFilterTwoAggs() throws Exception
}
}
+ ObjectMapper jsonMapper = queryFramework().queryJsonMapper();
testQuery(
"SELECT\n"
+ "BLOOM_FILTER(dim1, 1000),\n"
@@ -239,8 +230,8 @@ public void testBloomFilterTwoAggs() throws Exception
),
ImmutableList.of(
new Object[] {
- CalciteTests.getJsonMapper().writeValueAsString(expected1),
- CalciteTests.getJsonMapper().writeValueAsString(expected2)
+ jsonMapper.writeValueAsString(expected1),
+ jsonMapper.writeValueAsString(expected2)
}
)
);
@@ -288,10 +279,9 @@ public void testBloomFilterAggExtractionFn() throws Exception
.build()
),
ImmutableList.of(
- new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+ new Object[]{queryFramework().queryJsonMapper().writeValueAsString(expected1)}
)
);
-
}
@Test
@@ -335,7 +325,7 @@ public void testBloomFilterAggLong() throws Exception
.build()
),
ImmutableList.of(
- new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected3)}
+ new Object[]{queryFramework().queryJsonMapper().writeValueAsString(expected3)}
)
);
}
@@ -388,7 +378,7 @@ public void testBloomFilterAggLongVirtualColumn() throws Exception
.build()
),
ImmutableList.of(
- new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+ new Object[]{queryFramework().queryJsonMapper().writeValueAsString(expected1)}
)
);
}
@@ -442,7 +432,7 @@ public void testBloomFilterAggFloatVirtualColumn() throws Exception
.build()
),
ImmutableList.of(
- new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+ new Object[]{queryFramework().queryJsonMapper().writeValueAsString(expected1)}
)
);
}
@@ -496,7 +486,7 @@ public void testBloomFilterAggDoubleVirtualColumn() throws Exception
.build()
),
ImmutableList.of(
- new Object[]{CalciteTests.getJsonMapper().writeValueAsString(expected1)}
+ new Object[]{queryFramework().queryJsonMapper().writeValueAsString(expected1)}
)
);
}
@@ -510,6 +500,7 @@ public void testEmptyTimeseriesResults() throws Exception
BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
BloomKFilter expected2 = new BloomKFilter(TEST_NUM_ENTRIES);
+ ObjectMapper jsonMapper = queryFramework().queryJsonMapper();
testQuery(
"SELECT\n"
+ "BLOOM_FILTER(dim1, 1000),\n"
@@ -540,8 +531,8 @@ public void testEmptyTimeseriesResults() throws Exception
),
ImmutableList.of(
new Object[] {
- CalciteTests.getJsonMapper().writeValueAsString(expected1),
- CalciteTests.getJsonMapper().writeValueAsString(expected2)
+ jsonMapper.writeValueAsString(expected1),
+ jsonMapper.writeValueAsString(expected2)
}
)
);
@@ -556,6 +547,7 @@ public void testGroupByAggregatorDefaultValues() throws Exception
BloomKFilter expected1 = new BloomKFilter(TEST_NUM_ENTRIES);
BloomKFilter expected2 = new BloomKFilter(TEST_NUM_ENTRIES);
+ ObjectMapper jsonMapper = queryFramework().queryJsonMapper();
testQuery(
"SELECT\n"
+ "dim2,\n"
@@ -596,8 +588,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception
ImmutableList.of(
new Object[] {
"a",
- CalciteTests.getJsonMapper().writeValueAsString(expected1),
- CalciteTests.getJsonMapper().writeValueAsString(expected2)
+ jsonMapper.writeValueAsString(expected1),
+ jsonMapper.writeValueAsString(expected2)
}
)
);
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
index 4e1659c228fb..c5d6e631c410 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
@@ -19,22 +19,16 @@
package org.apache.druid.query.filter.sql;
-import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
import org.apache.calcite.avatica.SqlType;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.BloomFilterExtensionModule;
import org.apache.druid.guice.BloomFilterSerializersModule;
-import org.apache.druid.guice.ExpressionModule;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Druids;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
-import org.apache.druid.query.expression.LookupExprMacro;
-import org.apache.druid.query.expressions.BloomFilterExpressions;
import org.apache.druid.query.filter.BloomDimFilter;
import org.apache.druid.query.filter.BloomKFilter;
import org.apache.druid.query.filter.BloomKFilterHolder;
@@ -42,50 +36,21 @@
import org.apache.druid.query.filter.OrDimFilter;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
-import org.apache.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator;
-import org.apache.druid.sql.calcite.aggregation.builtin.BuiltinApproxCountDistinctSqlAggregator;
-import org.apache.druid.sql.calcite.aggregation.builtin.CountSqlAggregator;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.http.SqlParameter;
import org.junit.Ignore;
import org.junit.Test;
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
{
@Override
- public DruidOperatorTable createOperatorTable()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- CalciteTests.getJsonMapper().registerModule(new BloomFilterSerializersModule());
- return new DruidOperatorTable(
- ImmutableSet.of(
- new CountSqlAggregator(new ApproxCountDistinctSqlAggregator(new BuiltinApproxCountDistinctSqlAggregator()))
- ),
- ImmutableSet.of(new BloomFilterOperatorConversion())
- );
- }
-
- @Override
- public ExprMacroTable createMacroTable()
- {
- final List exprMacros = new ArrayList<>();
- for (Class extends ExprMacroTable.ExprMacro> clazz : ExpressionModule.EXPR_MACROS) {
- exprMacros.add(CalciteTests.INJECTOR.getInstance(clazz));
- }
- exprMacros.add(CalciteTests.INJECTOR.getInstance(LookupExprMacro.class));
- exprMacros.add(new BloomFilterExpressions.TestExprMacro());
- return new ExprMacroTable(exprMacros);
- }
-
- @Override
- public Iterable extends Module> getJacksonModules()
- {
- return Iterables.concat(super.getJacksonModules(), new BloomFilterExtensionModule().getJacksonModules());
+ super.configureGuice(builder);
+ builder.addModule(new BloomFilterExtensionModule());
}
@Test
@@ -144,7 +109,7 @@ public void testBloomFilterExprFilter() throws IOException
base64
),
null,
- createMacroTable()
+ queryFramework().macroTable()
)
)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml
index 0678f95ff047..6a40adb17bc8 100644
--- a/extensions-core/druid-catalog/pom.xml
+++ b/extensions-core/druid-catalog/pom.xml
@@ -31,7 +31,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml
index a2d248348222..9a36a98ba6eb 100644
--- a/extensions-core/druid-kerberos/pom.xml
+++ b/extensions-core/druid-kerberos/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/druid-pac4j/pom.xml b/extensions-core/druid-pac4j/pom.xml
index e3e675800438..5e4807def663 100644
--- a/extensions-core/druid-pac4j/pom.xml
+++ b/extensions-core/druid-pac4j/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/druid-ranger-security/pom.xml b/extensions-core/druid-ranger-security/pom.xml
index 39d42ccb2685..edb31fe29336 100644
--- a/extensions-core/druid-ranger-security/pom.xml
+++ b/extensions-core/druid-ranger-security/pom.xml
@@ -30,7 +30,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/ec2-extensions/pom.xml b/extensions-core/ec2-extensions/pom.xml
index 57f41e3ce8f6..85e95f53a624 100644
--- a/extensions-core/ec2-extensions/pom.xml
+++ b/extensions-core/ec2-extensions/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/google-extensions/pom.xml b/extensions-core/google-extensions/pom.xml
index c29738a0cba3..ea6ab401f34c 100644
--- a/extensions-core/google-extensions/pom.xml
+++ b/extensions-core/google-extensions/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml
index 83a6fd8b1849..587474b5475f 100644
--- a/extensions-core/hdfs-storage/pom.xml
+++ b/extensions-core/hdfs-storage/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml
index 9dd821478071..67fdf2003d45 100644
--- a/extensions-core/histogram/pom.xml
+++ b/extensions-core/histogram/pom.xml
@@ -28,7 +28,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java
index dd6399038c3e..a4eae57756ce 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java
@@ -19,12 +19,11 @@
package org.apache.druid.query.aggregation.histogram.sql;
-import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryContexts;
@@ -55,7 +54,6 @@
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
@@ -68,21 +66,18 @@
public class FixedBucketsHistogramQuantileSqlAggregatorTest extends BaseCalciteQueryTest
{
- private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
- ImmutableSet.of(new QuantileSqlAggregator(), new FixedBucketsHistogramQuantileSqlAggregator()),
- ImmutableSet.of()
- );
-
@Override
- public Iterable extends Module> getJacksonModules()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- return Iterables.concat(super.getJacksonModules(), new ApproximateHistogramDruidModule().getJacksonModules());
+ super.configureGuice(builder);
+ builder.addModule(new ApproximateHistogramDruidModule());
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
ApproximateHistogramDruidModule.registerSerde();
@@ -123,13 +118,6 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
);
}
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return OPERATOR_TABLE;
- }
-
-
@Test
public void testQuantileOnFloatAndLongs()
{
@@ -568,7 +556,6 @@ public void testEmptyTimeseriesResults()
);
}
-
@Test
public void testGroupByAggregatorDefaultValues()
{
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
index 6ed4c7e45106..4a67833b5249 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java
@@ -19,11 +19,10 @@
package org.apache.druid.query.aggregation.histogram.sql;
-import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Druids;
@@ -54,7 +53,6 @@
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.filtration.Filtration;
-import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestDataBuilder;
@@ -67,21 +65,18 @@
public class QuantileSqlAggregatorTest extends BaseCalciteQueryTest
{
- private static final DruidOperatorTable OPERATOR_TABLE = new DruidOperatorTable(
- ImmutableSet.of(new QuantileSqlAggregator()),
- ImmutableSet.of()
- );
-
@Override
- public Iterable extends Module> getJacksonModules()
+ public void configureGuice(DruidInjectorBuilder builder)
{
- return Iterables.concat(super.getJacksonModules(), new ApproximateHistogramDruidModule().getJacksonModules());
+ super.configureGuice(builder);
+ builder.addModule(new ApproximateHistogramDruidModule());
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
- final JoinableFactoryWrapper joinableFactory
+ final JoinableFactoryWrapper joinableFactory,
+ final Injector injector
) throws IOException
{
ApproximateHistogramDruidModule.registerSerde();
@@ -122,12 +117,6 @@ public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
);
}
- @Override
- public DruidOperatorTable createOperatorTable()
- {
- return OPERATOR_TABLE;
- }
-
@Test
public void testQuantileOnFloatAndLongs()
{
diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml
index 29f39f4dc2e8..1fa023dd3e64 100644
--- a/extensions-core/kafka-extraction-namespace/pom.xml
+++ b/extensions-core/kafka-extraction-namespace/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml
index 61102acc2bee..05a5bd840595 100644
--- a/extensions-core/kafka-indexing-service/pom.xml
+++ b/extensions-core/kafka-indexing-service/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
index 458955e58070..56e94215940b 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
@@ -30,24 +30,24 @@
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
import org.apache.druid.java.util.common.CloseableIterators;
import org.apache.druid.java.util.common.Pair;
-import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.java.util.common.parsers.ParseException;
import javax.annotation.Nullable;
-
import java.io.IOException;
+import java.util.AbstractMap;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.function.Function;
+import java.util.stream.Collectors;
public class KafkaInputReader implements InputEntityReader
{
- private static final Logger log = new Logger(KafkaInputReader.class);
-
private final InputRowSchema inputRowSchema;
private final SettableByteEntity source;
private final Function headerParserSupplier;
@@ -85,7 +85,60 @@ public KafkaInputReader(
this.timestampColumnName = timestampColumnName;
}
- private List getFinalDimensionList(HashSet newDimensions)
+ @Override
+ public CloseableIterator read() throws IOException
+ {
+ final KafkaRecordEntity record = source.getEntity();
+ final Map mergedHeaderMap = new HashMap<>();
+ if (headerParserSupplier != null) {
+ KafkaHeaderReader headerParser = headerParserSupplier.apply(record);
+ List> headerList = headerParser.read();
+ for (Pair ele : headerList) {
+ mergedHeaderMap.put(ele.lhs, ele.rhs);
+ }
+ }
+
+ // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in
+ // the header list
+ mergedHeaderMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+
+ InputEntityReader keyParser = (keyParserSupplier == null) ? null : keyParserSupplier.apply(record);
+ if (keyParser != null) {
+ try (CloseableIterator keyIterator = keyParser.read()) {
+ // Key currently only takes the first row and ignores the rest.
+ if (keyIterator.hasNext()) {
+ // Return type for the key parser should be of type MapBasedInputRow
+ // Parsers returning other types are not compatible currently.
+ MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
+ // Add the key to the mergeList only if the key string is not already present
+ mergedHeaderMap.putIfAbsent(
+ keyColumnName,
+ keyRow.getEvent().entrySet().stream().findFirst().get().getValue()
+ );
+ }
+ }
+ catch (ClassCastException e) {
+ throw new IOException(
+ "Unsupported keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows"
+ );
+ }
+ }
+
+ // Ignore tombstone records that have null values.
+ if (record.getRecord().value() != null) {
+ return buildBlendedRows(valueParser, mergedHeaderMap);
+ } else {
+ return buildRowsWithoutValuePayload(mergedHeaderMap);
+ }
+ }
+
+ @Override
+ public CloseableIterator sample() throws IOException
+ {
+ return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent()));
+ }
+
+ private List getFinalDimensionList(Set newDimensions)
{
final List schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames();
if (!schemaDimensions.isEmpty()) {
@@ -97,11 +150,14 @@ private List getFinalDimensionList(HashSet newDimensions)
}
}
- private CloseableIterator buildBlendedRows(InputEntityReader valueParser, Map headerKeyList) throws IOException
+ private CloseableIterator buildBlendedRows(
+ InputEntityReader valueParser,
+ Map headerKeyList
+ ) throws IOException
{
return valueParser.read().map(
r -> {
- MapBasedInputRow valueRow;
+ final MapBasedInputRow valueRow;
try {
// Return type for the value parser should be of type MapBasedInputRow
// Parsers returning other types are not compatible currently.
@@ -113,14 +169,9 @@ private CloseableIterator buildBlendedRows(InputEntityReader valuePars
"Unsupported input format in valueFormat. KafkaInputFormat only supports input format that return MapBasedInputRow rows"
);
}
- Map event = new HashMap<>(headerKeyList);
- /* Currently we prefer payload attributes if there is a collision in names.
- We can change this beahvior in later changes with a config knob. This default
- behavior lets easy porting of existing inputFormats to the new one without any changes.
- */
- event.putAll(valueRow.getEvent());
-
- HashSet newDimensions = new HashSet(valueRow.getDimensions());
+
+ final Map event = buildBlendedEventMap(valueRow.getEvent(), headerKeyList);
+ final HashSet newDimensions = new HashSet<>(valueRow.getDimensions());
newDimensions.addAll(headerKeyList.keySet());
// Remove the dummy timestamp added in KafkaInputFormat
newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
@@ -136,60 +187,70 @@ private CloseableIterator buildBlendedRows(InputEntityReader valuePars
private CloseableIterator buildRowsWithoutValuePayload(Map headerKeyList)
{
- HashSet newDimensions = new HashSet(headerKeyList.keySet());
- InputRow row = new MapBasedInputRow(
+ final InputRow row = new MapBasedInputRow(
inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
- getFinalDimensionList(newDimensions),
+ getFinalDimensionList(headerKeyList.keySet()),
headerKeyList
);
- List rows = Collections.singletonList(row);
+ final List rows = Collections.singletonList(row);
return CloseableIterators.withEmptyBaggage(rows.iterator());
}
- @Override
- public CloseableIterator read() throws IOException
+ /**
+ * Builds a map that blends two {@link Map}, presenting the combined keyset of both maps, and preferring to read
+ * from the first map and falling back to the second map if the value is not present.
+ *
+ * This strategy is used rather than just copying the values of the keyset into a new map so that any 'flattening'
+ * machinery (such as {@link Map} created by {@link org.apache.druid.java.util.common.parsers.ObjectFlatteners}) is
+ * still in place to be lazily evaluated instead of eagerly copying.
+ */
+ private static Map buildBlendedEventMap(Map map, Map fallback)
{
- KafkaRecordEntity record = source.getEntity();
- Map mergeMap = new HashMap<>();
- if (headerParserSupplier != null) {
- KafkaHeaderReader headerParser = headerParserSupplier.apply(record);
- List> headerList = headerParser.read();
- for (Pair ele : headerList) {
- mergeMap.put(ele.lhs, ele.rhs);
- }
- }
-
- // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list
- mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+ final Set keySet = new HashSet<>(fallback.keySet());
+ keySet.addAll(map.keySet());
- InputEntityReader keyParser = (keyParserSupplier == null) ? null : keyParserSupplier.apply(record);
- if (keyParser != null) {
- try (CloseableIterator keyIterator = keyParser.read()) {
- // Key currently only takes the first row and ignores the rest.
- if (keyIterator.hasNext()) {
- // Return type for the key parser should be of type MapBasedInputRow
- // Parsers returning other types are not compatible currently.
- MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
- // Add the key to the mergeList only if the key string is not already present
- mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue());
- }
+ return new AbstractMap()
+ {
+ @Override
+ public Object get(Object key)
+ {
+ return map.getOrDefault((String) key, fallback.get(key));
}
- catch (ClassCastException e) {
- throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+
+ @Override
+ public Set keySet()
+ {
+ return keySet;
}
- }
- // Ignore tombstone records that have null values.
- if (record.getRecord().value() != null) {
- return buildBlendedRows(valueParser, mergeMap);
- } else {
- return buildRowsWithoutValuePayload(mergeMap);
- }
- }
+ @Override
+ public Set> entrySet()
+ {
+ return keySet().stream()
+ .map(
+ field -> new Entry()
+ {
+ @Override
+ public String getKey()
+ {
+ return field;
+ }
- @Override
- public CloseableIterator sample() throws IOException
- {
- return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent()));
+ @Override
+ public Object getValue()
+ {
+ return get(field);
+ }
+
+ @Override
+ public Object setValue(final Object value)
+ {
+ throw new UnsupportedOperationException();
+ }
+ }
+ )
+ .collect(Collectors.toCollection(LinkedHashSet::new));
+ }
+ };
}
}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java
index fe0b89e996f8..a45730005a9c 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java
@@ -22,12 +22,12 @@
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import org.apache.druid.data.input.ColumnsFilter;
import org.apache.druid.data.input.InputEntityReader;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.TimestampSpec;
@@ -205,6 +205,7 @@ public void testWithHeaderKeyAndValue() throws IOException
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
+ Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
// Header verification
Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding")));
@@ -342,7 +343,6 @@ public byte[] value()
while (iterator.hasNext()) {
final InputRow row = iterator.next();
- final MapBasedInputRow mrow = (MapBasedInputRow) row;
// Payload verifications
Assert.assertEquals(DateTimes.of("2021-06-24"), row.getTimestamp());
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
@@ -350,6 +350,7 @@ public byte[] value()
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
+ Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
// Header verification
Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding")));
@@ -444,6 +445,7 @@ public void testWithOutKeyAndHeaderSpecs() throws IOException
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
+ Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
numActualIterations++;
}
@@ -521,6 +523,7 @@ public void testWithMultipleMixedRecords() throws IOException
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
+ Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index")));
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
index ab14f4dac7d6..28b915d22cd4 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
@@ -4150,7 +4150,7 @@ public void testIsTaskCurrent()
@Test
public void testResumeAllActivelyReadingTasks() throws Exception
{
- supervisor = getTestableSupervisor(2, 2, true, "PT1H", null, null);
+ supervisor = getTestableSupervisor(2, 3, true, "PT1H", null, null);
// Mock with task based setup for resumeAsync
EasyMock.reset(taskClient);
addSomeEvents(100);
@@ -4195,7 +4195,27 @@ public void testResumeAllActivelyReadingTasks() throws Exception
supervisor.getTuningConfig()
);
- List tasks = ImmutableList.of(readingTask, publishingTask, pausedTask, failsToResumePausedTask);
+ KafkaIndexTask waitingTask = createKafkaIndexTask("waitingTask",
+ DATASOURCE,
+ 2,
+ new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(2, 0L), Collections.emptySet()),
+ new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(2, Long.MAX_VALUE)),
+ null,
+ null,
+ supervisor.getTuningConfig()
+ );
+
+ KafkaIndexTask pendingTask = createKafkaIndexTask("pendingTask",
+ DATASOURCE,
+ 2,
+ new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(2, 0L), Collections.emptySet()),
+ new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(2, Long.MAX_VALUE)),
+ null,
+ null,
+ supervisor.getTuningConfig()
+ );
+
+ List tasks = ImmutableList.of(readingTask, publishingTask, pausedTask, failsToResumePausedTask, waitingTask, pendingTask);
Collection taskRunnerWorkItems = ImmutableList.of(
new TestTaskRunnerWorkItem(readingTask, null, new TaskLocation("testHost", 1001, -1)),
new TestTaskRunnerWorkItem(publishingTask, null, new TaskLocation("testHost", 1002, -1)),
@@ -4220,6 +4240,10 @@ public void testResumeAllActivelyReadingTasks() throws Exception
.andReturn(Optional.of(TaskStatus.running(pausedTask.getId()))).anyTimes();
EasyMock.expect(taskStorage.getStatus(failsToResumePausedTask.getId()))
.andReturn(Optional.of(TaskStatus.running(failsToResumePausedTask.getId()))).anyTimes();
+ EasyMock.expect(taskStorage.getStatus(waitingTask.getId()))
+ .andReturn(Optional.of(TaskStatus.running(waitingTask.getId()))).anyTimes();
+ EasyMock.expect(taskStorage.getStatus(pendingTask.getId()))
+ .andReturn(Optional.of(TaskStatus.running(pendingTask.getId()))).anyTimes();
EasyMock.expect(taskStorage.getTask(readingTask.getId()))
.andReturn(Optional.of(readingTask)).anyTimes();
@@ -4229,6 +4253,10 @@ public void testResumeAllActivelyReadingTasks() throws Exception
.andReturn(Optional.of(pausedTask)).anyTimes();
EasyMock.expect(taskStorage.getTask(failsToResumePausedTask.getId()))
.andReturn(Optional.of(failsToResumePausedTask)).anyTimes();
+ EasyMock.expect(taskStorage.getTask(waitingTask.getId()))
+ .andReturn(Optional.of(waitingTask)).anyTimes();
+ EasyMock.expect(taskStorage.getTask(pendingTask.getId()))
+ .andReturn(Optional.of(pendingTask)).anyTimes();
EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
@@ -4237,6 +4265,12 @@ public void testResumeAllActivelyReadingTasks() throws Exception
).anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true);
+
+ EasyMock.expect(taskClient.getStartTimeAsync(waitingTask.getId()))
+ .andReturn(Futures.immediateFuture(null));
+ EasyMock.expect(taskClient.getStartTimeAsync(pendingTask.getId()))
+ .andReturn(Futures.immediateFuture(null));
+
EasyMock.expect(taskClient.getStatusAsync(readingTask.getId()))
.andReturn(Futures.immediateFuture(Status.READING));
EasyMock.expect(taskClient.getStatusAsync(publishingTask.getId()))
@@ -4245,6 +4279,10 @@ public void testResumeAllActivelyReadingTasks() throws Exception
.andReturn(Futures.immediateFuture(Status.PAUSED));
EasyMock.expect(taskClient.getStatusAsync(failsToResumePausedTask.getId()))
.andReturn(Futures.immediateFuture(Status.PAUSED));
+ EasyMock.expect(taskClient.getStatusAsync(waitingTask.getId()))
+ .andReturn(Futures.immediateFuture(Status.NOT_STARTED));
+ EasyMock.expect(taskClient.getStatusAsync(pendingTask.getId()))
+ .andReturn(Futures.immediateFuture(Status.NOT_STARTED));
EasyMock.expect(taskClient.getEndOffsetsAsync(publishingTask.getId()))
.andReturn(Futures.immediateFuture(ImmutableMap.of(0, 0L)));
@@ -4258,6 +4296,12 @@ public void testResumeAllActivelyReadingTasks() throws Exception
EasyMock.expect(taskClient.getCheckpointsAsync(failsToResumePausedTask.getId(), true))
.andReturn(Futures.immediateFuture(new TreeMap<>()));
+ EasyMock.expect(taskClient.getCheckpointsAsync(waitingTask.getId(), true))
+ .andReturn(Futures.immediateFuture(null));
+
+ EasyMock.expect(taskClient.getCheckpointsAsync(pendingTask.getId(), true))
+ .andReturn(Futures.immediateFuture(null));
+
taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class));
// Only the active i.e non-publishing tasks are resumed
diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml
index 166b0291e00c..28533af40191 100644
--- a/extensions-core/kinesis-indexing-service/pom.xml
+++ b/extensions-core/kinesis-indexing-service/pom.xml
@@ -29,7 +29,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
index 50fbe2f721db..265a9fc144f8 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
@@ -3102,6 +3102,11 @@ public void testNoDataIngestionTasks() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
+ final Collection workItems = new ArrayList();
+ workItems.add(new TestTaskRunnerWorkItem(id1, null, new TaskLocation(id1.getId(), 8100, 8100)));
+ workItems.add(new TestTaskRunnerWorkItem(id2, null, new TaskLocation(id2.getId(), 8100, 8100)));
+ workItems.add(new TestTaskRunnerWorkItem(id3, null, new TaskLocation(id3.getId(), 8100, 8100)));
+ EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems);
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();
diff --git a/extensions-core/kubernetes-extensions/pom.xml b/extensions-core/kubernetes-extensions/pom.xml
index 80e110f231a9..aec1de9b8ea1 100644
--- a/extensions-core/kubernetes-extensions/pom.xml
+++ b/extensions-core/kubernetes-extensions/pom.xml
@@ -30,7 +30,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
@@ -90,7 +90,7 @@
org.bouncycastle
bcprov-jdk15on
- 1.68
+ 1.69
runtime
diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml
index b9538f7cacbb..79bff849cbf5 100644
--- a/extensions-core/lookups-cached-global/pom.xml
+++ b/extensions-core/lookups-cached-global/pom.xml
@@ -28,7 +28,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml
index de98aaa38bd1..37fe9bf0e10d 100644
--- a/extensions-core/lookups-cached-single/pom.xml
+++ b/extensions-core/lookups-cached-single/pom.xml
@@ -28,7 +28,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml
index f8382a967b09..443b792cca65 100644
--- a/extensions-core/multi-stage-query/pom.xml
+++ b/extensions-core/multi-stage-query/pom.xml
@@ -31,7 +31,7 @@
org.apache.druid
druid
- 25.0.0-SNAPSHOT
+ 26.0.0-SNAPSHOT
../../pom.xml
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java
new file mode 100644
index 000000000000..92ed82ff5e11
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ClusterStatisticsMergeMode.java
@@ -0,0 +1,44 @@
+/*
+ * 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.msq.exec;
+
+/**
+ * Mode which dictates how {@link WorkerSketchFetcher} gets sketches for the partition boundaries from workers.
+ */
+public enum ClusterStatisticsMergeMode
+{
+ /**
+ * Fetches sketch in sequential order based on time. Slower due to overhead, but more accurate.
+ */
+ SEQUENTIAL,
+
+ /**
+ * Fetch all sketches from the worker at once. Faster to generate partitions, but less accurate.
+ */
+ PARALLEL,
+
+ /**
+ * Tries to decide between sequential and parallel modes based on the number of workers and size of the input
+ *
+ * If there are more than 100 workers or if the combined sketch size among all workers is more than
+ * 1,000,000,000 bytes, SEQUENTIAL mode is chosen, otherwise, PARALLEL mode is chosen.
+ */
+ AUTO
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java
index 07730de45e7a..fe010b219e49 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java
@@ -27,7 +27,7 @@
import org.apache.druid.msq.counters.CounterSnapshotsTree;
import org.apache.druid.msq.indexing.MSQControllerTask;
import org.apache.druid.msq.indexing.error.MSQErrorReport;
-import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
import javax.annotation.Nullable;
import java.util.List;
@@ -81,9 +81,11 @@ public String getId()
// Worker-to-controller messages
/**
- * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+ * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key
+ * statistics have been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate partiton boundaries.
+ * This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}.
*/
- void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject);
+ void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject);
/**
* System error reported by a subtask. Note that the errors are organized by
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java
index faf1c3ff5e98..9d197b76dd17 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java
@@ -22,7 +22,7 @@
import org.apache.druid.msq.counters.CounterSnapshotsTree;
import org.apache.druid.msq.indexing.error.MSQErrorReport;
import org.apache.druid.msq.kernel.StageId;
-import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
import javax.annotation.Nullable;
import java.io.IOException;
@@ -34,13 +34,13 @@
public interface ControllerClient extends AutoCloseable
{
/**
- * Client side method to update the controller with key statistics for a particular stage and worker.
- * Controller's implementation collates all the key statistics for a stage to generate the partition boundaries.
+ * Client side method to update the controller with partial key statistics information for a particular stage and worker.
+ * Controller's implementation collates all the information for a stage to fetch key statistics from workers.
*/
- void postKeyStatistics(
+ void postPartialKeyStatistics(
StageId stageId,
int workerNumber,
- ClusterByStatisticsSnapshot keyStatistics
+ PartialKeyStatisticsInformation partialKeyStatisticsInformation
) throws IOException;
/**
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
index a4812292fb15..528baa4c27d1 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java
@@ -64,6 +64,7 @@
import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Either;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
@@ -107,6 +108,7 @@
import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher;
import org.apache.druid.msq.indexing.error.MSQWarnings;
import org.apache.druid.msq.indexing.error.QueryNotSupportedFault;
+import org.apache.druid.msq.indexing.error.TooManyPartitionsFault;
import org.apache.druid.msq.indexing.error.TooManyWarningsFault;
import org.apache.druid.msq.indexing.error.UnknownFault;
import org.apache.druid.msq.indexing.report.MSQResultsReport;
@@ -149,7 +151,8 @@
import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory;
import org.apache.druid.msq.shuffle.DurableStorageUtils;
import org.apache.druid.msq.shuffle.WorkerInputChannelFactory;
-import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation;
+import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
import org.apache.druid.msq.util.DimensionSchemaUtils;
import org.apache.druid.msq.util.IntervalUtils;
import org.apache.druid.msq.util.MSQFutureUtils;
@@ -201,6 +204,7 @@
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ThreadLocalRandom;
@@ -259,6 +263,7 @@ public class ControllerImpl implements Controller
// For live reports. Written by the main controller thread, read by HTTP threads.
private final ConcurrentHashMap stagePartitionCountsForLiveReports = new ConcurrentHashMap<>();
+ private WorkerSketchFetcher workerSketchFetcher;
// Time at which the query started.
// For live reports. Written by the main controller thread, read by HTTP threads.
private volatile DateTime queryStartTime = null;
@@ -521,6 +526,16 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer)
this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this));
closer.register(netClient::close);
+ ClusterStatisticsMergeMode clusterStatisticsMergeMode =
+ MultiStageQueryContext.getClusterStatisticsMergeMode(task.getQuerySpec().getQuery().context());
+
+ log.debug("Query [%s] cluster statistics merge mode is set to %s.", id(), clusterStatisticsMergeMode);
+
+ int statisticsMaxRetainedBytes = WorkerMemoryParameters.createProductionInstanceForController(context.injector())
+ .getPartitionStatisticsMaxRetainedBytes();
+ this.workerSketchFetcher = new WorkerSketchFetcher(netClient, clusterStatisticsMergeMode, statisticsMaxRetainedBytes);
+ closer.register(workerSketchFetcher::close);
+
final boolean isDurableStorageEnabled =
MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context());
@@ -565,10 +580,12 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer)
}
/**
- * Provide a {@link ClusterByStatisticsSnapshot} for shuffling stages.
+ * Accepts a {@link PartialKeyStatisticsInformation} and updates the controller key statistics information. If all key
+ * statistics information has been gathered, enqueues the task with the {@link WorkerSketchFetcher} to generate
+ * partiton boundaries. This is intended to be called by the {@link org.apache.druid.msq.indexing.ControllerChatHandler}.
*/
@Override
- public void updateStatus(int stageNumber, int workerNumber, Object keyStatisticsObject)
+ public void updatePartialKeyStatisticsInformation(int stageNumber, int workerNumber, Object partialKeyStatisticsInformationObject)
{
addToKernelManipulationQueue(
queryKernel -> {
@@ -582,9 +599,9 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
stageDef.getShuffleSpec().get().doesAggregateByClusterKey()
);
- final ClusterByStatisticsSnapshot keyStatistics;
+ final PartialKeyStatisticsInformation partialKeyStatisticsInformation;
try {
- keyStatistics = mapper.convertValue(keyStatisticsObject, ClusterByStatisticsSnapshot.class);
+ partialKeyStatisticsInformation = mapper.convertValue(partialKeyStatisticsInformationObject, PartialKeyStatisticsInformation.class);
}
catch (IllegalArgumentException e) {
throw new IAE(
@@ -595,7 +612,36 @@ public void updateStatus(int stageNumber, int workerNumber, Object keyStatistics
);
}
- queryKernel.addResultKeyStatisticsForStageAndWorker(stageId, workerNumber, keyStatistics);
+ queryKernel.addPartialKeyStatisticsForStageAndWorker(stageId, workerNumber, partialKeyStatisticsInformation);
+
+ if (queryKernel.getStagePhase(stageId).equals(ControllerStagePhase.MERGING_STATISTICS)) {
+ List workerTaskIds = workerTaskLauncher.getTaskList();
+ CompleteKeyStatisticsInformation completeKeyStatisticsInformation =
+ queryKernel.getCompleteKeyStatisticsInformation(stageId);
+
+ // Queue the sketch fetching task into the worker sketch fetcher.
+ CompletableFuture> clusterByPartitionsCompletableFuture =
+ workerSketchFetcher.submitFetcherTask(
+ completeKeyStatisticsInformation,
+ workerTaskIds,
+ stageDef
+ );
+
+ // Add the listener to handle completion.
+ clusterByPartitionsCompletableFuture.whenComplete((clusterByPartitionsEither, throwable) -> {
+ addToKernelManipulationQueue(holder -> {
+ if (throwable != null) {
+ holder.failStageForReason(stageId, UnknownFault.forException(throwable));
+ } else if (clusterByPartitionsEither.isError()) {
+ holder.failStageForReason(stageId, new TooManyPartitionsFault(stageDef.getMaxPartitionCount()));
+ } else {
+ log.debug("Query [%s] Partition boundaries generated for stage %s", id(), stageId);
+ holder.setClusterByPartitionBoundaries(stageId, clusterByPartitionsEither.valueOrThrow());
+ }
+ holder.transitionStageKernel(stageId, queryKernel.getStagePhase(stageId));
+ });
+ });
+ }
}
);
}
@@ -943,7 +989,7 @@ private QueryKit makeQueryControllerToolKit()
final Map, QueryKit> kitMap =
ImmutableMap., QueryKit>builder()
.put(ScanQuery.class, new ScanQueryKit(context.jsonMapper()))
- .put(GroupByQuery.class, new GroupByQueryKit())
+ .put(GroupByQuery.class, new GroupByQueryKit(context.jsonMapper()))
.build();
return new MultiQueryKit(kitMap);
@@ -1959,11 +2005,7 @@ public RunQueryUntilDone(
this.queryDef = queryDef;
this.inputSpecSlicerFactory = inputSpecSlicerFactory;
this.closer = closer;
- this.queryKernel = new ControllerQueryKernel(
- queryDef,
- WorkerMemoryParameters.createProductionInstanceForController(context.injector())
- .getPartitionStatisticsMaxRetainedBytes()
- );
+ this.queryKernel = new ControllerQueryKernel(queryDef);
}
/**
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java
index 1c3cc39987c7..3d78b7c9ced4 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ExceptionWrappingWorkerClient.java
@@ -31,6 +31,7 @@
import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.WorkOrder;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import javax.annotation.Nullable;
import java.io.IOException;
@@ -55,6 +56,23 @@ public ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workO
return wrap(workerTaskId, client, c -> c.postWorkOrder(workerTaskId, workOrder));
}
+ @Override
+ public ListenableFuture fetchClusterByStatisticsSnapshot(String workerTaskId, String queryId, int stageNumber)
+ {
+ return client.fetchClusterByStatisticsSnapshot(workerTaskId, queryId, stageNumber);
+ }
+
+ @Override
+ public ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk(
+ String workerTaskId,
+ String queryId,
+ int stageNumber,
+ long timeChunk
+ )
+ {
+ return client.fetchClusterByStatisticsSnapshotForTimeChunk(workerTaskId, queryId, stageNumber, timeChunk);
+ }
+
@Override
public ListenableFuture postResultPartitionBoundaries(
final String workerTaskId,
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java
index f069c91e145a..cc5f0fae1732 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Worker.java
@@ -25,6 +25,7 @@
import org.apache.druid.msq.indexing.MSQWorkerTask;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.WorkOrder;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import javax.annotation.Nullable;
import java.io.IOException;
@@ -67,6 +68,18 @@ public interface Worker
*/
void postWorkOrder(WorkOrder workOrder);
+ /**
+ * Returns the statistics snapshot for the given stageId. This is called from {@link WorkerSketchFetcher} under
+ * PARALLEL OR AUTO modes.
+ */
+ ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId);
+
+ /**
+ * Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk.
+ * This is called from {@link WorkerSketchFetcher} under SEQUENTIAL OR AUTO modes.
+ */
+ ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk);
+
/**
* Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber
* and queryId
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java
index 8f4c7bac239a..5c02a79f89a3 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java
@@ -25,6 +25,7 @@
import org.apache.druid.msq.counters.CounterSnapshotsTree;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.WorkOrder;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import java.io.IOException;
@@ -38,6 +39,27 @@ public interface WorkerClient extends AutoCloseable
*/
ListenableFuture postWorkOrder(String workerTaskId, WorkOrder workOrder);
+ /**
+ * Fetches the {@link ClusterByStatisticsSnapshot} from a worker. This is intended to be used by the
+ * {@link WorkerSketchFetcher} under PARALLEL or AUTO modes.
+ */
+ ListenableFuture fetchClusterByStatisticsSnapshot(
+ String workerTaskId,
+ String queryId,
+ int stageNumber
+ );
+
+ /**
+ * Fetches a {@link ClusterByStatisticsSnapshot} which contains only the sketch of the specified timeChunk.
+ * This is intended to be used by the {@link WorkerSketchFetcher} under SEQUENTIAL or AUTO modes.
+ */
+ ListenableFuture fetchClusterByStatisticsSnapshotForTimeChunk(
+ String workerTaskId,
+ String queryId,
+ int stageNumber,
+ long timeChunk
+ );
+
/**
* Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the
* controller after collating the result statistics from all the workers processing the query
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
index 5b68041d0e97..49d6f9080d7c 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java
@@ -106,6 +106,7 @@
import org.apache.druid.msq.shuffle.WorkerInputChannelFactory;
import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
import org.apache.druid.msq.util.DecoratedExecutorService;
import org.apache.druid.msq.util.MultiStageQueryContext;
import org.apache.druid.query.PrioritizedCallable;
@@ -159,6 +160,7 @@ public class WorkerImpl implements Worker
private final BlockingQueue> kernelManipulationQueue = new LinkedBlockingDeque<>();
private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>();
private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap stageKernelMap = new ConcurrentHashMap<>();
private final boolean durableStageStorageEnabled;
/**
@@ -365,10 +367,14 @@ public Optional runTask(final Closer closer) throws Exception
if (kernel.getPhase() == WorkerStagePhase.READING_INPUT && kernel.hasResultKeyStatisticsSnapshot()) {
if (controllerAlive) {
- controllerClient.postKeyStatistics(
+ PartialKeyStatisticsInformation partialKeyStatisticsInformation =
+ kernel.getResultKeyStatisticsSnapshot()
+ .partialKeyStatistics();
+
+ controllerClient.postPartialKeyStatistics(
stageDefinition.getId(),
kernel.getWorkOrder().getWorkerNumber(),
- kernel.getResultKeyStatisticsSnapshot()
+ partialKeyStatisticsInformation
);
}
kernel.startPreshuffleWaitingForResultPartitionBoundaries();
@@ -562,6 +568,19 @@ public void postFinish()
kernelManipulationQueue.add(KernelHolder::setDone);
}
+ @Override
+ public ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId)
+ {
+ return stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();
+ }
+
+ @Override
+ public ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk)
+ {
+ ClusterByStatisticsSnapshot snapshot = stageKernelMap.get(stageId).getResultKeyStatisticsSnapshot();
+ return snapshot.getSnapshotForTimeChunk(timeChunk);
+ }
+
@Override
public CounterSnapshotsTree getCounters()
{
@@ -1273,9 +1292,8 @@ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int p
}
}
- private static class KernelHolder
+ private class KernelHolder
{
- private final Map stageKernelMap = new HashMap<>();
private boolean done = false;
public Map getStageKernelMap()
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java
new file mode 100644
index 000000000000..dc6f21990587
--- /dev/null
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerSketchFetcher.java
@@ -0,0 +1,344 @@
+/*
+ * 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.msq.exec;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.frame.key.ClusterBy;
+import org.apache.druid.frame.key.ClusterByPartition;
+import org.apache.druid.frame.key.ClusterByPartitions;
+import org.apache.druid.java.util.common.Either;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.msq.kernel.StageDefinition;
+import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.IntStream;
+
+/**
+ * Queues up fetching sketches from workers and progressively generates partitions boundaries.
+ */
+public class WorkerSketchFetcher implements AutoCloseable
+{
+ private static final Logger log = new Logger(WorkerSketchFetcher.class);
+ private static final int DEFAULT_THREAD_COUNT = 4;
+ // If the combined size of worker sketches is more than this threshold, SEQUENTIAL merging mode is used.
+ static final long BYTES_THRESHOLD = 1_000_000_000L;
+ // If there are more workers than this threshold, SEQUENTIAL merging mode is used.
+ static final long WORKER_THRESHOLD = 100;
+
+ private final ClusterStatisticsMergeMode clusterStatisticsMergeMode;
+ private final int statisticsMaxRetainedBytes;
+ private final WorkerClient workerClient;
+ private final ExecutorService executorService;
+
+ public WorkerSketchFetcher(
+ WorkerClient workerClient,
+ ClusterStatisticsMergeMode clusterStatisticsMergeMode,
+ int statisticsMaxRetainedBytes
+ )
+ {
+ this.workerClient = workerClient;
+ this.clusterStatisticsMergeMode = clusterStatisticsMergeMode;
+ this.executorService = Execs.multiThreaded(DEFAULT_THREAD_COUNT, "SketchFetcherThreadPool-%d");
+ this.statisticsMaxRetainedBytes = statisticsMaxRetainedBytes;
+ }
+
+ /**
+ * Submits a request to fetch and generate partitions for the given worker statistics and returns a future for it. It
+ * decides based on the statistics if it should fetch sketches one by one or together.
+ */
+ public CompletableFuture> submitFetcherTask(
+ CompleteKeyStatisticsInformation completeKeyStatisticsInformation,
+ List workerTaskIds,
+ StageDefinition stageDefinition
+ )
+ {
+ ClusterBy clusterBy = stageDefinition.getClusterBy();
+
+ switch (clusterStatisticsMergeMode) {
+ case SEQUENTIAL:
+ return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds);
+ case PARALLEL:
+ return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+ case AUTO:
+ if (clusterBy.getBucketByCount() == 0) {
+ log.info("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId());
+ // If there is no time clustering, there is no scope for sequential merge
+ return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+ } else if (stageDefinition.getMaxWorkerCount() > WORKER_THRESHOLD || completeKeyStatisticsInformation.getBytesRetained() > BYTES_THRESHOLD) {
+ log.info("Query [%s] AUTO mode: chose SEQUENTIAL mode to merge key statistics", stageDefinition.getId().getQueryId());
+ return sequentialTimeChunkMerging(completeKeyStatisticsInformation, stageDefinition, workerTaskIds);
+ }
+ log.info("Query [%s] AUTO mode: chose PARALLEL mode to merge key statistics", stageDefinition.getId().getQueryId());
+ return inMemoryFullSketchMerging(stageDefinition, workerTaskIds);
+ default:
+ throw new IllegalStateException("No fetching strategy found for mode: " + clusterStatisticsMergeMode);
+ }
+ }
+
+ /**
+ * Fetches the full {@link ClusterByStatisticsCollector} from all workers and generates partition boundaries from them.
+ * This is faster than fetching them timechunk by timechunk but the collector will be downsampled till it can fit
+ * on the controller, resulting in less accurate partition boundries.
+ */
+ CompletableFuture> inMemoryFullSketchMerging(
+ StageDefinition stageDefinition,
+ List workerTaskIds
+ )
+ {
+ CompletableFuture> partitionFuture = new CompletableFuture<>();
+
+ // Create a new key statistics collector to merge worker sketches into
+ final ClusterByStatisticsCollector mergedStatisticsCollector =
+ stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes);
+ final int workerCount = workerTaskIds.size();
+ // Guarded by synchronized mergedStatisticsCollector
+ final Set finishedWorkers = new HashSet<>();
+
+ // Submit a task for each worker to fetch statistics
+ IntStream.range(0, workerCount).forEach(workerNo -> {
+ executorService.submit(() -> {
+ ListenableFuture snapshotFuture =
+ workerClient.fetchClusterByStatisticsSnapshot(
+ workerTaskIds.get(workerNo),
+ stageDefinition.getId().getQueryId(),
+ stageDefinition.getStageNumber()
+ );
+ try {
+ ClusterByStatisticsSnapshot clusterByStatisticsSnapshot = snapshotFuture.get();
+ if (clusterByStatisticsSnapshot == null) {
+ throw new ISE("Worker %s returned null sketch, this should never happen", workerNo);
+ }
+ synchronized (mergedStatisticsCollector) {
+ mergedStatisticsCollector.addAll(clusterByStatisticsSnapshot);
+ finishedWorkers.add(workerNo);
+
+ if (finishedWorkers.size() == workerCount) {
+ log.debug("Query [%s] Received all statistics, generating partitions", stageDefinition.getId().getQueryId());
+ partitionFuture.complete(stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector));
+ }
+ }
+ }
+ catch (Exception e) {
+ synchronized (mergedStatisticsCollector) {
+ if (!partitionFuture.isDone()) {
+ partitionFuture.completeExceptionally(e);
+ mergedStatisticsCollector.clear();
+ }
+ }
+ }
+ });
+ });
+
+ return partitionFuture;
+ }
+
+ /**
+ * Fetches cluster statistics from all workers and generates partition boundaries from them one time chunk at a time.
+ * This takes longer due to the overhead of fetching sketches, however, this prevents any loss in accuracy from
+ * downsampling on the controller.
+ */
+ CompletableFuture> sequentialTimeChunkMerging(
+ CompleteKeyStatisticsInformation completeKeyStatisticsInformation,
+ StageDefinition stageDefinition,
+ List workerTaskIds
+ )
+ {
+ SequentialFetchStage sequentialFetchStage = new SequentialFetchStage(
+ stageDefinition,
+ workerTaskIds,
+ completeKeyStatisticsInformation.getTimeSegmentVsWorkerMap().entrySet().iterator()
+ );
+ sequentialFetchStage.submitFetchingTasksForNextTimeChunk();
+ return sequentialFetchStage.getPartitionFuture();
+ }
+
+ private class SequentialFetchStage
+ {
+ private final StageDefinition stageDefinition;
+ private final List workerTaskIds;
+ private final Iterator>> timeSegmentVsWorkerIdIterator;
+ private final CompletableFuture> partitionFuture;
+ // Final sorted list of partition boundaries. This is appended to after statistics for each time chunk are gathered.
+ private final List finalPartitionBoundries;
+
+ public SequentialFetchStage(
+ StageDefinition stageDefinition,
+ List workerTaskIds,
+ Iterator>> timeSegmentVsWorkerIdIterator
+ )
+ {
+ this.finalPartitionBoundries = new ArrayList<>();
+ this.stageDefinition = stageDefinition;
+ this.workerTaskIds = workerTaskIds;
+ this.timeSegmentVsWorkerIdIterator = timeSegmentVsWorkerIdIterator;
+ this.partitionFuture = new CompletableFuture<>();
+ }
+
+ /**
+ * Submits the tasks to fetch key statistics for the time chunk pointed to by {@link #timeSegmentVsWorkerIdIterator}.
+ * Once the statistics have been gathered from all workers which have them, generates partitions and adds it to
+ * {@link #finalPartitionBoundries}, stiching the partitions between time chunks using
+ * {@link #abutAndAppendPartitionBoundries(List, List)} to make them continuous.
+ *
+ * The time chunks returned by {@link #timeSegmentVsWorkerIdIterator} should be in ascending order for the partitions
+ * to be generated correctly.
+ *
+ * If {@link #timeSegmentVsWorkerIdIterator} doesn't have any more values, assumes that partition boundaries have
+ * been successfully generated and completes {@link #partitionFuture} with the result.
+ *
+ * Completes the future with an error as soon as the number of partitions exceed max partition count for the stage
+ * definition.
+ */
+ public void submitFetchingTasksForNextTimeChunk()
+ {
+ if (!timeSegmentVsWorkerIdIterator.hasNext()) {
+ partitionFuture.complete(Either.value(new ClusterByPartitions(finalPartitionBoundries)));
+ } else {
+ Map.Entry> entry = timeSegmentVsWorkerIdIterator.next();
+ // Time chunk for which partition boundries are going to be generated for
+ Long timeChunk = entry.getKey();
+ Set workerIdsWithTimeChunk = entry.getValue();
+ // Create a new key statistics collector to merge worker sketches into
+ ClusterByStatisticsCollector mergedStatisticsCollector =
+ stageDefinition.createResultKeyStatisticsCollector(statisticsMaxRetainedBytes);
+ // Guarded by synchronized mergedStatisticsCollector
+ Set finishedWorkers = new HashSet<>();
+
+ log.debug("Query [%s]. Submitting request for statistics for time chunk %s to %s workers",
+ stageDefinition.getId().getQueryId(),
+ timeChunk,
+ workerIdsWithTimeChunk.size());
+
+ // Submits a task for every worker which has a certain time chunk
+ for (int workerNo : workerIdsWithTimeChunk) {
+ executorService.submit(() -> {
+ ListenableFuture snapshotFuture =
+ workerClient.fetchClusterByStatisticsSnapshotForTimeChunk(
+ workerTaskIds.get(workerNo),
+ stageDefinition.getId().getQueryId(),
+ stageDefinition.getStageNumber(),
+ timeChunk
+ );
+
+ try {
+ ClusterByStatisticsSnapshot snapshotForTimeChunk = snapshotFuture.get();
+ if (snapshotForTimeChunk == null) {
+ throw new ISE("Worker %s returned null sketch for %s, this should never happen", workerNo, timeChunk);
+ }
+ synchronized (mergedStatisticsCollector) {
+ mergedStatisticsCollector.addAll(snapshotForTimeChunk);
+ finishedWorkers.add(workerNo);
+
+ if (finishedWorkers.size() == workerIdsWithTimeChunk.size()) {
+ Either longClusterByPartitionsEither =
+ stageDefinition.generatePartitionsForShuffle(mergedStatisticsCollector);
+
+ log.debug("Query [%s]. Received all statistics for time chunk %s, generating partitions",
+ stageDefinition.getId().getQueryId(),
+ timeChunk);
+
+ long totalPartitionCount = finalPartitionBoundries.size() + getPartitionCountFromEither(longClusterByPartitionsEither);
+ if (totalPartitionCount > stageDefinition.getMaxPartitionCount()) {
+ // Fail fast if more partitions than the maximum have been reached.
+ partitionFuture.complete(Either.error(totalPartitionCount));
+ mergedStatisticsCollector.clear();
+ } else {
+ List timeSketchPartitions = longClusterByPartitionsEither.valueOrThrow().ranges();
+ abutAndAppendPartitionBoundries(finalPartitionBoundries, timeSketchPartitions);
+ log.debug("Query [%s]. Finished generating partitions for time chunk %s, total count so far %s",
+ stageDefinition.getId().getQueryId(),
+ timeChunk,
+ finalPartitionBoundries.size());
+ submitFetchingTasksForNextTimeChunk();
+ }
+ }
+ }
+ }
+ catch (Exception e) {
+ synchronized (mergedStatisticsCollector) {
+ if (!partitionFuture.isDone()) {
+ partitionFuture.completeExceptionally(e);
+ mergedStatisticsCollector.clear();
+ }
+ }
+ }
+ });
+ }
+ }
+ }
+
+ /**
+ * Takes a list of sorted {@link ClusterByPartitions} {@param timeSketchPartitions} and adds it to a sorted list
+ * {@param finalPartitionBoundries}. If {@param finalPartitionBoundries} is not empty, the end time of the last
+ * partition of {@param finalPartitionBoundries} is changed to abut with the starting time of the first partition
+ * of {@param timeSketchPartitions}.
+ *
+ * This is used to make the partitions generated continuous.
+ */
+ private void abutAndAppendPartitionBoundries(
+ List finalPartitionBoundries,
+ List timeSketchPartitions
+ )
+ {
+ if (!finalPartitionBoundries.isEmpty()) {
+ // Stitch up the end time of the last partition with the start time of the first partition.
+ ClusterByPartition clusterByPartition = finalPartitionBoundries.remove(finalPartitionBoundries.size() - 1);
+ finalPartitionBoundries.add(new ClusterByPartition(clusterByPartition.getStart(), timeSketchPartitions.get(0).getStart()));
+ }
+ finalPartitionBoundries.addAll(timeSketchPartitions);
+ }
+
+ public CompletableFuture> getPartitionFuture()
+ {
+ return partitionFuture;
+ }
+ }
+
+ /**
+ * Gets the partition size from an {@link Either}. If it is an error, the long denotes the number of partitions
+ * (in the case of creating too many partitions), otherwise checks the size of the list.
+ */
+ private static long getPartitionCountFromEither(Either either)
+ {
+ if (either.isError()) {
+ return either.error();
+ } else {
+ return either.valueOrThrow().size();
+ }
+ }
+
+ @Override
+ public void close()
+ {
+ executorService.shutdownNow();
+ }
+}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java
index e2c47c2a2dda..5335e40b0e9c 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java
@@ -21,7 +21,6 @@
import com.fasterxml.jackson.databind.Module;
import com.google.inject.Binder;
-import com.google.inject.Inject;
import com.google.inject.Provides;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.guice.LazySingleton;
@@ -33,10 +32,12 @@
import org.apache.druid.sql.SqlStatementFactory;
import org.apache.druid.sql.SqlToolbox;
import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
+import org.apache.druid.sql.calcite.external.HttpOperatorConversion;
+import org.apache.druid.sql.calcite.external.InlineOperatorConversion;
+import org.apache.druid.sql.calcite.external.LocalOperatorConversion;
import org.apache.druid.sql.guice.SqlBindings;
import java.util.List;
-import java.util.Properties;
/**
* Module for providing the {@code EXTERN} operator.
@@ -44,9 +45,6 @@
@LoadScope(roles = NodeRole.BROKER_JSON_NAME)
public class MSQSqlModule implements DruidModule
{
- @Inject
- Properties properties = null;
-
@Override
public List extends Module> getJacksonModules()
{
@@ -64,9 +62,11 @@ public void configure(Binder binder)
// Set up the EXTERN macro.
SqlBindings.addOperatorConversion(binder, ExternalOperatorConversion.class);
+ SqlBindings.addOperatorConversion(binder, HttpOperatorConversion.class);
+ SqlBindings.addOperatorConversion(binder, InlineOperatorConversion.class);
+ SqlBindings.addOperatorConversion(binder, LocalOperatorConversion.class);
}
-
@Provides
@MSQ
@LazySingleton
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java
index 1857d83708c7..8fa04ce6d90b 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/ControllerChatHandler.java
@@ -26,7 +26,8 @@
import org.apache.druid.msq.exec.Controller;
import org.apache.druid.msq.exec.ControllerClient;
import org.apache.druid.msq.indexing.error.MSQErrorReport;
-import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.kernel.StageId;
+import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlers;
import org.apache.druid.server.security.Action;
@@ -58,16 +59,17 @@ public ControllerChatHandler(TaskToolbox toolbox, Controller controller)
}
/**
- * Used by subtasks to post {@link ClusterByStatisticsSnapshot} for shuffling stages.
+ * Used by subtasks to post {@link PartialKeyStatisticsInformation} for shuffling stages.
*
- * See {@link ControllerClient#postKeyStatistics} for the client-side code that calls this API.
+ * See {@link ControllerClient#postPartialKeyStatistics(StageId, int, PartialKeyStatisticsInformation)}
+ * for the client-side code that calls this API.
*/
@POST
- @Path("/keyStatistics/{queryId}/{stageNumber}/{workerNumber}")
+ @Path("/partialKeyStatisticsInformation/{queryId}/{stageNumber}/{workerNumber}")
@Produces(MediaType.APPLICATION_JSON)
@Consumes(MediaType.APPLICATION_JSON)
- public Response httpPostKeyStatistics(
- final Object keyStatisticsObject,
+ public Response httpPostPartialKeyStatistics(
+ final Object partialKeyStatisticsObject,
@PathParam("queryId") final String queryId,
@PathParam("stageNumber") final int stageNumber,
@PathParam("workerNumber") final int workerNumber,
@@ -75,7 +77,7 @@ public Response httpPostKeyStatistics(
)
{
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
- controller.updateStatus(stageNumber, workerNumber, keyStatisticsObject);
+ controller.updatePartialKeyStatisticsInformation(stageNumber, workerNumber, partialKeyStatisticsObject);
return Response.status(Response.Status.ACCEPTED).build();
}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java
index 3a6c9e78797b..1ff5952a8958 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerClient.java
@@ -29,7 +29,7 @@
import org.apache.druid.msq.exec.ControllerClient;
import org.apache.druid.msq.indexing.error.MSQErrorReport;
import org.apache.druid.msq.kernel.StageId;
-import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
+import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
import org.apache.druid.rpc.IgnoreHttpResponseHandler;
import org.apache.druid.rpc.RequestBuilder;
import org.apache.druid.rpc.ServiceClient;
@@ -59,14 +59,14 @@ public IndexerControllerClient(
}
@Override
- public void postKeyStatistics(
+ public void postPartialKeyStatistics(
StageId stageId,
int workerNumber,
- ClusterByStatisticsSnapshot keyStatistics
+ PartialKeyStatisticsInformation partialKeyStatisticsInformation
) throws IOException
{
final String path = StringUtils.format(
- "/keyStatistics/%s/%s/%d",
+ "/partialKeyStatisticsInformation/%s/%d/%d",
StringUtils.urlEncode(stageId.getQueryId()),
stageId.getStageNumber(),
workerNumber
@@ -74,7 +74,7 @@ public void postKeyStatistics(
doRequest(
new RequestBuilder(HttpMethod.POST, path)
- .jsonContent(jsonMapper, keyStatistics),
+ .jsonContent(jsonMapper, partialKeyStatisticsInformation),
IgnoreHttpResponseHandler.INSTANCE
);
}
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java
index 430c264291f8..980c7f97bee5 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerClient.java
@@ -41,6 +41,7 @@
import org.apache.druid.msq.exec.WorkerClient;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.WorkOrder;
+import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import org.apache.druid.rpc.IgnoreHttpResponseHandler;
import org.apache.druid.rpc.RequestBuilder;
import org.apache.druid.rpc.ServiceClient;
@@ -103,6 +104,48 @@ public ListenableFuture