From 6430ef8e1b62515576f5c7ec090bcd79da7eb8ee Mon Sep 17 00:00:00 2001
From: Justin Borromeo
Date: Fri, 1 Feb 2019 14:21:13 -0800
Subject: [PATCH 01/13] lol (#6985)
---
docs/content/ingestion/schema-design.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/content/ingestion/schema-design.md b/docs/content/ingestion/schema-design.md
index 0408a39330a..58f0faeca08 100644
--- a/docs/content/ingestion/schema-design.md
+++ b/docs/content/ingestion/schema-design.md
@@ -86,7 +86,7 @@ rollup and load your existing data as-is. Rollup in Druid is similar to creating
(Like OpenTSDB or InfluxDB.)
Similar to time series databases, Druid's data model requires a timestamp. Druid is not a timeseries database, but
-it is a natural choice for storing timeseries data. Its flexible data mdoel allows it to store both timeseries and
+it is a natural choice for storing timeseries data. Its flexible data model allows it to store both timeseries and
non-timeseries data, even in the same datasource.
To achieve best-case compression and query performance in Druid for timeseries data, it is important to partition and
From 61f165c23f33381f2ea05ae61feeef6a940ec0f0 Mon Sep 17 00:00:00 2001
From: Furkan KAMACI
Date: Sun, 3 Feb 2019 05:42:28 +0300
Subject: [PATCH 02/13] Try-with-resources should be used since the new syntax
is more readable. (#6944)
* Try-with-resources should be used since the new syntax is more readable.
* Fixed checkstyle error.
---
.../org/apache/druid/common/utils/SocketUtil.java | 14 +-------------
1 file changed, 1 insertion(+), 13 deletions(-)
diff --git a/core/src/main/java/org/apache/druid/common/utils/SocketUtil.java b/core/src/main/java/org/apache/druid/common/utils/SocketUtil.java
index c02351d7504..8eb38776e9e 100644
--- a/core/src/main/java/org/apache/druid/common/utils/SocketUtil.java
+++ b/core/src/main/java/org/apache/druid/common/utils/SocketUtil.java
@@ -41,24 +41,12 @@ public class SocketUtil
int currPort = startPort;
while (currPort < 0xffff) {
- ServerSocket socket = null;
- try {
- socket = new ServerSocket(currPort);
+ try (ServerSocket socket = new ServerSocket(currPort)) {
return currPort;
}
catch (IOException e) {
++currPort;
}
- finally {
- if (socket != null) {
- try {
- socket.close();
- }
- catch (IOException e) {
-
- }
- }
- }
}
throw new ISE("Unable to find open port between [%d] and [%d]", startPort, currPort);
From 7baa33049cde3c3751037f014f468db4beb41a4c Mon Sep 17 00:00:00 2001
From: Surekha
Date: Sat, 2 Feb 2019 22:27:13 -0800
Subject: [PATCH 03/13] Introduce published segment cache in broker (#6901)
* Add published segment cache in broker
* Change the DataSegment interner so it's not based on DataSEgment's equals only and size is preserved if set
* Added a trueEquals to DataSegment class
* Use separate interner for realtime and historical segments
* Remove trueEquals as it's not used anymore, change log message
* PR comments
* PR comments
* Fix tests
* PR comments
* Few more modification to
* change the coordinator api
* removeall segments at once from MetadataSegmentView in order to serve a more consistent view of published segments
* Change the poll behaviour to avoid multiple poll execution at same time
* minor changes
* PR comments
* PR comments
* Make the segment cache in broker off by default
* Added a config to PlannerConfig
* Moved MetadataSegmentView to sql module
* Add doc for new planner config
* Update documentation
* PR comments
* some more changes
* PR comments
* fix test
* remove unintentional change, whether to synchronize on lifecycleLock is still in discussion in PR
* minor changes
* some changes to initialization
* use pollPeriodInMS
* Add boolean cachePopulated to check if first poll succeeds
* Remove poll from start()
* take the log message out of condition in stop()
---
.../druid/benchmark/query/SqlBenchmark.java | 2 +-
docs/content/querying/sql.md | 2 +
.../sql/BloomFilterSqlAggregatorTest.java | 2 +-
.../sql/QuantileSqlAggregatorTest.java | 2 +-
.../druid/client/DataSegmentInterner.java | 49 ++++
.../druid/client/selector/ServerSelector.java | 3 +-
.../druid/server/http/MetadataResource.java | 15 +-
.../client/CachingClusteredClientTest.java | 5 +-
.../sql/calcite/planner/PlannerConfig.java | 23 ++
.../calcite/schema/MetadataSegmentView.java | 255 ++++++++++++++++++
.../sql/calcite/schema/SystemSchema.java | 104 ++-----
.../sql/avatica/DruidAvaticaHandlerTest.java | 4 +-
.../druid/sql/avatica/DruidStatementTest.java | 2 +-
.../sql/calcite/BaseCalciteQueryTest.java | 2 +-
.../sql/calcite/http/SqlResourceTest.java | 2 +-
.../sql/calcite/schema/SystemSchemaTest.java | 184 +++++--------
.../druid/sql/calcite/util/CalciteTests.java | 13 +-
17 files changed, 460 insertions(+), 209 deletions(-)
create mode 100644 server/src/main/java/org/apache/druid/client/DataSegmentInterner.java
create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
index bb19188fc4e..1f515882753 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
@@ -114,7 +114,7 @@ public class SqlBenchmark
final QueryRunnerFactoryConglomerate conglomerate = conglomerateCloserPair.lhs;
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index);
final PlannerFactory plannerFactory = new PlannerFactory(
druidSchema,
diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md
index 358c1ec3f1a..9a3f03e36ac 100644
--- a/docs/content/querying/sql.md
+++ b/docs/content/querying/sql.md
@@ -698,6 +698,8 @@ The Druid SQL server is configured through the following properties on the Broke
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the Broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
+|`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST api will be invoked when broker needs published segments info.|false|
+|`druid.sql.planner.metadataSegmentPollPeriod`|How often to poll coordinator for published segments list if `druid.sql.planner.metadataSegmentCacheEnable` is set to true. Poll period is in milliseconds. |60000|
## SQL Metrics
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 4641e274899..6b218bbcb44 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
@@ -192,7 +192,7 @@ public class BloomFilterSqlAggregatorTest
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new BloomFilterSqlAggregator()),
ImmutableSet.of()
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 69337b568f6..7aa4e6733ea 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
@@ -157,7 +157,7 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = new DruidOperatorTable(
ImmutableSet.of(new QuantileSqlAggregator()),
ImmutableSet.of()
diff --git a/server/src/main/java/org/apache/druid/client/DataSegmentInterner.java b/server/src/main/java/org/apache/druid/client/DataSegmentInterner.java
new file mode 100644
index 00000000000..11d104d8604
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/client/DataSegmentInterner.java
@@ -0,0 +1,49 @@
+/*
+ * 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.client;
+
+import com.google.common.collect.Interner;
+import com.google.common.collect.Interners;
+import org.apache.druid.timeline.DataSegment;
+
+/**
+ * Interns the DataSegment object in order to share the reference for same DataSegment.
+ * It uses two separate interners for realtime and historical segments to prevent
+ * overwriting the size of a segment which was served by a historical and later served
+ * by another realtime server, since realtime server always publishes with size 0.
+ */
+public class DataSegmentInterner
+{
+ private static final Interner REALTIME_INTERNER = Interners.newWeakInterner();
+ private static final Interner HISTORICAL_INTERNER = Interners.newWeakInterner();
+
+ private DataSegmentInterner()
+ {
+ //No instantiation
+ }
+
+ public static DataSegment intern(DataSegment segment)
+ {
+ // A segment learns it's size and dimensions when it moves from a relatime to historical server
+ // for that reason, we are using it's size as the indicator to decide whether to use REALTIME or
+ // HISTORICAL interner.
+ return segment.getSize() > 0 ? HISTORICAL_INTERNER.intern(segment) : REALTIME_INTERNER.intern(segment);
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java
index de6b58be385..a485dbaa955 100644
--- a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java
+++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java
@@ -20,6 +20,7 @@
package org.apache.druid.client.selector;
import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap;
+import org.apache.druid.client.DataSegmentInterner;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.timeline.DataSegment;
@@ -50,7 +51,7 @@ public class ServerSelector implements DiscoverySelector
TierSelectorStrategy strategy
)
{
- this.segment = new AtomicReference<>(segment);
+ this.segment = new AtomicReference<>(DataSegmentInterner.intern(segment));
this.strategy = strategy;
this.historicalServers = new Int2ObjectRBTreeMap<>(strategy.getComparator());
this.realtimeServers = new Int2ObjectRBTreeMap<>(strategy.getComparator());
diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java
index d1e83a91f4e..c7e270214ff 100644
--- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java
+++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java
@@ -57,6 +57,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
+import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
@@ -148,14 +149,22 @@ public class MetadataResource
@GET
@Path("/segments")
@Produces(MediaType.APPLICATION_JSON)
- public Response getDatabaseSegments(@Context final HttpServletRequest req)
+ public Response getDatabaseSegments(
+ @Context final HttpServletRequest req,
+ @QueryParam("datasources") final Set datasources
+ )
{
- final Collection druidDataSources = metadataSegmentManager.getDataSources();
+ Collection druidDataSources = metadataSegmentManager.getDataSources();
+ if (datasources != null && !datasources.isEmpty()) {
+ druidDataSources = druidDataSources.stream()
+ .filter(src -> datasources.contains(src.getName()))
+ .collect(Collectors.toSet());
+ }
final Stream metadataSegments = druidDataSources
.stream()
.flatMap(t -> t.getSegments().stream());
- Function> raGenerator = segment -> Collections.singletonList(
+ final Function> raGenerator = segment -> Collections.singletonList(
AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));
final Iterable authorizedSegments =
diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
index 2eff321d78f..4f043eeaede 100644
--- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
+++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
@@ -2213,13 +2213,13 @@ public class CachingClusteredClientTest
expectedResults.get(k).get(j)
);
serverExpectations.get(lastServer).addExpectation(expectation);
-
+ EasyMock.expect(mockSegment.getSize()).andReturn(0L).anyTimes();
+ EasyMock.replay(mockSegment);
ServerSelector selector = new ServerSelector(
expectation.getSegment(),
new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy())
);
selector.addServerAndUpdateSegment(new QueryableDruidServer(lastServer, null), selector.getSegment());
-
final ShardSpec shardSpec;
if (numChunks == 1) {
shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0);
@@ -2234,6 +2234,7 @@ public class CachingClusteredClientTest
}
shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j);
}
+ EasyMock.reset(mockSegment);
EasyMock.expect(mockSegment.getShardSpec())
.andReturn(shardSpec)
.anyTimes();
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java
index 766bf92e6d4..acc4d08986c 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java
@@ -67,6 +67,21 @@ public class PlannerConfig
private DateTimeZone sqlTimeZone = DateTimeZone.UTC;
@JsonProperty
+ private boolean metadataSegmentCacheEnable = false;
+
+ @JsonProperty
+ private long metadataSegmentPollPeriod = 60000;
+
+ public long getMetadataSegmentPollPeriod()
+ {
+ return metadataSegmentPollPeriod;
+ }
+
+ public boolean isMetadataSegmentCacheEnable()
+ {
+ return metadataSegmentCacheEnable;
+ }
+
private boolean serializeComplexValues = true;
public Period getMetadataRefreshPeriod()
@@ -159,6 +174,8 @@ public class PlannerConfig
newConfig.requireTimeCondition = isRequireTimeCondition();
newConfig.sqlTimeZone = getSqlTimeZone();
newConfig.awaitInitializationOnStart = isAwaitInitializationOnStart();
+ newConfig.metadataSegmentCacheEnable = isMetadataSegmentCacheEnable();
+ newConfig.metadataSegmentPollPeriod = getMetadataSegmentPollPeriod();
newConfig.serializeComplexValues = shouldSerializeComplexValues();
return newConfig;
}
@@ -200,6 +217,8 @@ public class PlannerConfig
useFallback == that.useFallback &&
requireTimeCondition == that.requireTimeCondition &&
awaitInitializationOnStart == that.awaitInitializationOnStart &&
+ metadataSegmentCacheEnable == that.metadataSegmentCacheEnable &&
+ metadataSegmentPollPeriod == that.metadataSegmentPollPeriod &&
serializeComplexValues == that.serializeComplexValues &&
Objects.equals(metadataRefreshPeriod, that.metadataRefreshPeriod) &&
Objects.equals(sqlTimeZone, that.sqlTimeZone);
@@ -221,6 +240,8 @@ public class PlannerConfig
requireTimeCondition,
awaitInitializationOnStart,
sqlTimeZone,
+ metadataSegmentCacheEnable,
+ metadataSegmentPollPeriod,
serializeComplexValues
);
}
@@ -239,6 +260,8 @@ public class PlannerConfig
", useFallback=" + useFallback +
", requireTimeCondition=" + requireTimeCondition +
", awaitInitializationOnStart=" + awaitInitializationOnStart +
+ ", metadataSegmentCacheEnable=" + metadataSegmentCacheEnable +
+ ", metadataSegmentPollPeriod=" + metadataSegmentPollPeriod +
", sqlTimeZone=" + sqlTimeZone +
", serializeComplexValues=" + serializeComplexValues +
'}';
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java
new file mode 100644
index 00000000000..50fe3133cd2
--- /dev/null
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.sql.calcite.schema;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.inject.Inject;
+import org.apache.druid.client.BrokerSegmentWatcherConfig;
+import org.apache.druid.client.DataSegmentInterner;
+import org.apache.druid.client.JsonParserIterator;
+import org.apache.druid.client.coordinator.Coordinator;
+import org.apache.druid.concurrent.LifecycleLock;
+import org.apache.druid.discovery.DruidLeaderClient;
+import org.apache.druid.guice.ManageLifecycle;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
+import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.http.client.Request;
+import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler;
+import org.apache.druid.sql.calcite.planner.PlannerConfig;
+import org.apache.druid.timeline.DataSegment;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.joda.time.DateTime;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This class polls the coordinator in background to keep the latest published segments.
+ * Provides {@link #getPublishedSegments()} for others to get segments in metadata store.
+ */
+@ManageLifecycle
+public class MetadataSegmentView
+{
+
+ private static final EmittingLogger log = new EmittingLogger(MetadataSegmentView.class);
+
+ private final DruidLeaderClient coordinatorDruidLeaderClient;
+ private final ObjectMapper jsonMapper;
+ private final BytesAccumulatingResponseHandler responseHandler;
+ private final BrokerSegmentWatcherConfig segmentWatcherConfig;
+
+ private final boolean isCacheEnabled;
+ @Nullable
+ private final ConcurrentMap publishedSegments;
+ private final ScheduledExecutorService scheduledExec;
+ private final long pollPeriodInMS;
+ private final LifecycleLock lifecycleLock = new LifecycleLock();
+ private final AtomicBoolean cachePopulated = new AtomicBoolean(false);
+
+ @Inject
+ public MetadataSegmentView(
+ final @Coordinator DruidLeaderClient druidLeaderClient,
+ final ObjectMapper jsonMapper,
+ final BytesAccumulatingResponseHandler responseHandler,
+ final BrokerSegmentWatcherConfig segmentWatcherConfig,
+ final PlannerConfig plannerConfig
+ )
+ {
+ Preconditions.checkNotNull(plannerConfig, "plannerConfig");
+ this.coordinatorDruidLeaderClient = druidLeaderClient;
+ this.jsonMapper = jsonMapper;
+ this.responseHandler = responseHandler;
+ this.segmentWatcherConfig = segmentWatcherConfig;
+ this.isCacheEnabled = plannerConfig.isMetadataSegmentCacheEnable();
+ this.pollPeriodInMS = plannerConfig.getMetadataSegmentPollPeriod();
+ this.publishedSegments = isCacheEnabled ? new ConcurrentHashMap<>(1000) : null;
+ this.scheduledExec = Execs.scheduledSingleThreaded("MetadataSegmentView-Cache--%d");
+ }
+
+ @LifecycleStart
+ public void start()
+ {
+ if (!lifecycleLock.canStart()) {
+ throw new ISE("can't start.");
+ }
+ try {
+ if (isCacheEnabled) {
+ scheduledExec.schedule(new PollTask(), pollPeriodInMS, TimeUnit.MILLISECONDS);
+ }
+ lifecycleLock.started();
+ log.info("MetadataSegmentView Started.");
+ }
+ finally {
+ lifecycleLock.exitStart();
+ }
+ }
+
+ @LifecycleStop
+ public void stop()
+ {
+ if (!lifecycleLock.canStop()) {
+ throw new ISE("can't stop.");
+ }
+ log.info("MetadataSegmentView is stopping.");
+ if (isCacheEnabled) {
+ scheduledExec.shutdown();
+ }
+ log.info("MetadataSegmentView Stopped.");
+ }
+
+ private void poll()
+ {
+ log.info("polling published segments from coordinator");
+ final JsonParserIterator metadataSegments = getMetadataSegments(
+ coordinatorDruidLeaderClient,
+ jsonMapper,
+ responseHandler,
+ segmentWatcherConfig.getWatchedDataSources()
+ );
+
+ final DateTime timestamp = DateTimes.nowUtc();
+ while (metadataSegments.hasNext()) {
+ final DataSegment interned = DataSegmentInterner.intern(metadataSegments.next());
+ // timestamp is used to filter deleted segments
+ publishedSegments.put(interned, timestamp);
+ }
+ // filter the segments from cache whose timestamp is not equal to latest timestamp stored,
+ // since the presence of a segment with an earlier timestamp indicates that
+ // "that" segment is not returned by coordinator in latest poll, so it's
+ // likely deleted and therefore we remove it from publishedSegments
+ // Since segments are not atomically replaced because it can cause high
+ // memory footprint due to large number of published segments, so
+ // we are incrementally removing deleted segments from the map
+ // This means publishedSegments will be eventually consistent with
+ // the segments in coordinator
+ publishedSegments.entrySet().removeIf(e -> e.getValue() != timestamp);
+ cachePopulated.set(true);
+ }
+
+ public Iterator getPublishedSegments()
+ {
+ if (isCacheEnabled) {
+ Preconditions.checkState(
+ lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS) && cachePopulated.get(),
+ "hold on, still syncing published segments"
+ );
+ return publishedSegments.keySet().iterator();
+ } else {
+ return getMetadataSegments(
+ coordinatorDruidLeaderClient,
+ jsonMapper,
+ responseHandler,
+ segmentWatcherConfig.getWatchedDataSources()
+ );
+ }
+ }
+
+ // Note that coordinator must be up to get segments
+ private JsonParserIterator getMetadataSegments(
+ DruidLeaderClient coordinatorClient,
+ ObjectMapper jsonMapper,
+ BytesAccumulatingResponseHandler responseHandler,
+ Set watchedDataSources
+ )
+ {
+ String query = "/druid/coordinator/v1/metadata/segments";
+ if (watchedDataSources != null && !watchedDataSources.isEmpty()) {
+ log.debug(
+ "filtering datasources in published segments based on broker's watchedDataSources[%s]", watchedDataSources);
+ final StringBuilder sb = new StringBuilder();
+ for (String ds : watchedDataSources) {
+ sb.append("datasources=").append(ds).append("&");
+ }
+ sb.setLength(sb.length() - 1);
+ query = "/druid/coordinator/v1/metadata/segments?" + sb;
+ }
+ Request request;
+ try {
+ request = coordinatorClient.makeRequest(
+ HttpMethod.GET,
+ StringUtils.format(query),
+ false
+ );
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ ListenableFuture future = coordinatorClient.goAsync(
+ request,
+ responseHandler
+ );
+
+ final JavaType typeRef = jsonMapper.getTypeFactory().constructType(new TypeReference()
+ {
+ });
+ return new JsonParserIterator<>(
+ typeRef,
+ future,
+ request.getUrl().toString(),
+ null,
+ request.getUrl().getHost(),
+ jsonMapper,
+ responseHandler
+ );
+ }
+
+ private class PollTask implements Runnable
+ {
+ @Override
+ public void run()
+ {
+ long delayMS = pollPeriodInMS;
+ try {
+ final long pollStartTime = System.nanoTime();
+ poll();
+ final long pollEndTime = System.nanoTime();
+ final long pollTimeNS = pollEndTime - pollStartTime;
+ final long pollTimeMS = TimeUnit.NANOSECONDS.toMillis(pollTimeNS);
+ delayMS = Math.max(pollPeriodInMS - pollTimeMS, 0);
+ }
+ catch (Exception e) {
+ log.makeAlert(e, "Problem polling Coordinator.").emit();
+ }
+ finally {
+ if (!Thread.currentThread().isInterrupted()) {
+ scheduledExec.schedule(new PollTask(), delayMS, TimeUnit.MILLISECONDS);
+ }
+ }
+ }
+ }
+
+}
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
index 772c62886f4..d0599f86190 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java
@@ -149,6 +149,7 @@ public class SystemSchema extends AbstractSchema
@Inject
public SystemSchema(
final DruidSchema druidSchema,
+ final MetadataSegmentView metadataView,
final TimelineServerView serverView,
final AuthorizerMapper authorizerMapper,
final @Coordinator DruidLeaderClient coordinatorDruidLeaderClient,
@@ -158,11 +159,10 @@ public class SystemSchema extends AbstractSchema
{
Preconditions.checkNotNull(serverView, "serverView");
BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler();
- SegmentsTable segmentsTable = new SegmentsTable(
+ final SegmentsTable segmentsTable = new SegmentsTable(
druidSchema,
- coordinatorDruidLeaderClient,
+ metadataView,
jsonMapper,
- responseHandler,
authorizerMapper
);
this.tableMap = ImmutableMap.of(
@@ -182,23 +182,20 @@ public class SystemSchema extends AbstractSchema
static class SegmentsTable extends AbstractTable implements ScannableTable
{
private final DruidSchema druidSchema;
- private final DruidLeaderClient druidLeaderClient;
private final ObjectMapper jsonMapper;
- private final BytesAccumulatingResponseHandler responseHandler;
private final AuthorizerMapper authorizerMapper;
+ private final MetadataSegmentView metadataView;
public SegmentsTable(
DruidSchema druidSchemna,
- DruidLeaderClient druidLeaderClient,
+ MetadataSegmentView metadataView,
ObjectMapper jsonMapper,
- BytesAccumulatingResponseHandler responseHandler,
AuthorizerMapper authorizerMapper
)
{
this.druidSchema = druidSchemna;
- this.druidLeaderClient = druidLeaderClient;
+ this.metadataView = metadataView;
this.jsonMapper = jsonMapper;
- this.responseHandler = responseHandler;
this.authorizerMapper = authorizerMapper;
}
@@ -231,12 +228,8 @@ public class SystemSchema extends AbstractSchema
partialSegmentDataMap.put(h.getSegmentId(), partialSegmentData);
}
- //get published segments from coordinator
- final JsonParserIterator metadataSegments = getMetadataSegments(
- druidLeaderClient,
- jsonMapper,
- responseHandler
- );
+ //get published segments from metadata segment cache (if enabled in sql planner config), else directly from coordinator
+ final Iterator metadataSegments = metadataView.getPublishedSegments();
final Set segmentsAlreadySeen = new HashSet<>();
@@ -245,7 +238,7 @@ public class SystemSchema extends AbstractSchema
metadataSegments,
root
))
- .transform((DataSegment val) -> {
+ .transform(val -> {
try {
segmentsAlreadySeen.add(val.getId());
final PartialSegmentData partialSegmentData = partialSegmentDataMap.get(val.getId());
@@ -318,6 +311,26 @@ public class SystemSchema extends AbstractSchema
}
+ private Iterator getAuthorizedPublishedSegments(
+ Iterator it,
+ DataContext root
+ )
+ {
+ final AuthenticationResult authenticationResult =
+ (AuthenticationResult) root.get(PlannerContext.DATA_CTX_AUTHENTICATION_RESULT);
+
+ Function> raGenerator = segment -> Collections.singletonList(
+ AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));
+
+ final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources(
+ authenticationResult,
+ () -> it,
+ raGenerator,
+ authorizerMapper
+ );
+ return authorizedSegments.iterator();
+ }
+
private Iterator> getAuthorizedAvailableSegments(
Iterator> availableSegmentEntries,
DataContext root
@@ -340,27 +353,6 @@ public class SystemSchema extends AbstractSchema
return authorizedSegments.iterator();
}
- private CloseableIterator getAuthorizedPublishedSegments(
- JsonParserIterator it,
- DataContext root
- )
- {
- final AuthenticationResult authenticationResult =
- (AuthenticationResult) root.get(PlannerContext.DATA_CTX_AUTHENTICATION_RESULT);
-
- Function> raGenerator = segment -> Collections.singletonList(
- AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));
-
- final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources(
- authenticationResult,
- () -> it,
- raGenerator,
- authorizerMapper
- );
-
- return wrap(authorizedSegments.iterator(), it);
- }
-
private static class PartialSegmentData
{
private final long isAvailable;
@@ -404,44 +396,6 @@ public class SystemSchema extends AbstractSchema
}
}
- // Note that coordinator must be up to get segments
- private static JsonParserIterator getMetadataSegments(
- DruidLeaderClient coordinatorClient,
- ObjectMapper jsonMapper,
- BytesAccumulatingResponseHandler responseHandler
- )
- {
-
- Request request;
- try {
- request = coordinatorClient.makeRequest(
- HttpMethod.GET,
- StringUtils.format("/druid/coordinator/v1/metadata/segments"),
- false
- );
- }
- catch (IOException e) {
- throw new RuntimeException(e);
- }
- ListenableFuture future = coordinatorClient.goAsync(
- request,
- responseHandler
- );
-
- final JavaType typeRef = jsonMapper.getTypeFactory().constructType(new TypeReference()
- {
- });
- return new JsonParserIterator<>(
- typeRef,
- future,
- request.getUrl().toString(),
- null,
- request.getUrl().getHost(),
- jsonMapper,
- responseHandler
- );
- }
-
static class ServersTable extends AbstractTable implements ScannableTable
{
private final TimelineServerView serverView;
diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
index c1c43b5ad1e..4f25a66c113 100644
--- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java
@@ -159,7 +159,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder());
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
@@ -790,7 +790,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
final List frames = new ArrayList<>();
diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java
index 74386125ef1..9cc2ecf4cab 100644
--- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java
@@ -88,7 +88,7 @@ public class DruidStatementTest extends CalciteTestBase
walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder());
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
final PlannerFactory plannerFactory = new PlannerFactory(
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
index 6b47b42993b..18c1ac478c7 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
@@ -547,7 +547,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
{
final InProcessViewManager viewManager = new InProcessViewManager(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR);
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig, viewManager);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final PlannerFactory plannerFactory = new PlannerFactory(
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/http/SqlResourceTest.java
index 14b226896d2..d525fffde4c 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/http/SqlResourceTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/http/SqlResourceTest.java
@@ -121,7 +121,7 @@ public class SqlResourceTest extends CalciteTestBase
}
};
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
- final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
req = EasyMock.createStrictMock(HttpServletRequest.class);
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
index 407ff69ed95..7d8cdaad572 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java
@@ -98,6 +98,9 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
public class SystemSchemaTest extends CalciteTestBase
{
@@ -127,6 +130,7 @@ public class SystemSchemaTest extends CalciteTestBase
private AuthorizerMapper authMapper;
private static QueryRunnerFactoryConglomerate conglomerate;
private static Closer resourceCloser;
+ private MetadataSegmentView metadataView;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -215,8 +219,10 @@ public class SystemSchemaTest extends CalciteTestBase
);
druidSchema.start();
druidSchema.awaitInitialization();
+ metadataView = EasyMock.createMock(MetadataSegmentView.class);
schema = new SystemSchema(
druidSchema,
+ metadataView,
serverView,
EasyMock.createStrictMock(AuthorizerMapper.class),
client,
@@ -225,6 +231,44 @@ public class SystemSchemaTest extends CalciteTestBase
);
}
+
+ private final DataSegment publishedSegment1 = new DataSegment(
+ "wikipedia1",
+ Intervals.of("2007/2008"),
+ "version1",
+ null,
+ ImmutableList.of("dim1", "dim2"),
+ ImmutableList.of("met1", "met2"),
+ null,
+ 1,
+ 53000L,
+ DataSegment.PruneLoadSpecHolder.DEFAULT
+ );
+ private final DataSegment publishedSegment2 = new DataSegment(
+ "wikipedia2",
+ Intervals.of("2008/2009"),
+ "version2",
+ null,
+ ImmutableList.of("dim1", "dim2"),
+ ImmutableList.of("met1", "met2"),
+ null,
+ 1,
+ 83000L,
+ DataSegment.PruneLoadSpecHolder.DEFAULT
+ );
+ private final DataSegment publishedSegment3 = new DataSegment(
+ "wikipedia3",
+ Intervals.of("2009/2010"),
+ "version3",
+ null,
+ ImmutableList.of("dim1", "dim2"),
+ ImmutableList.of("met1", "met2"),
+ null,
+ 1,
+ 47000L,
+ DataSegment.PruneLoadSpecHolder.DEFAULT
+ );
+
private final DataSegment segment1 = new DataSegment(
"test1",
Intervals.of("2010/2011"),
@@ -263,7 +307,7 @@ public class SystemSchemaTest extends CalciteTestBase
);
private final DataSegment segment4 = new DataSegment(
"test4",
- Intervals.of("2017/2018"),
+ Intervals.of("2014/2015"),
"version4",
null,
ImmutableList.of("dim1", "dim2"),
@@ -275,7 +319,7 @@ public class SystemSchemaTest extends CalciteTestBase
);
private final DataSegment segment5 = new DataSegment(
"test5",
- Intervals.of("2017/2018"),
+ Intervals.of("2015/2016"),
"version5",
null,
ImmutableList.of("dim1", "dim2"),
@@ -340,120 +384,22 @@ public class SystemSchemaTest extends CalciteTestBase
}
@Test
- public void testSegmentsTable() throws Exception
+ public void testSegmentsTable()
{
final SystemSchema.SegmentsTable segmentsTable = EasyMock
.createMockBuilder(SystemSchema.SegmentsTable.class)
- .withConstructor(druidSchema, client, mapper, responseHandler, authMapper)
+ .withConstructor(druidSchema, metadataView, mapper, authMapper)
.createMock();
EasyMock.replay(segmentsTable);
+ final Set publishedSegments = Stream.of(publishedSegment1,
+ publishedSegment2,
+ publishedSegment3,
+ segment1,
+ segment2).collect(Collectors.toSet());
+ EasyMock.expect(metadataView.getPublishedSegments()).andReturn(publishedSegments.iterator()).once();
- EasyMock
- .expect(client.makeRequest(HttpMethod.GET, "/druid/coordinator/v1/metadata/segments", false))
- .andReturn(request)
- .anyTimes();
- SettableFuture future = SettableFuture.create();
- EasyMock.expect(client.goAsync(request, responseHandler)).andReturn(future).once();
- final int ok = HttpServletResponse.SC_OK;
- EasyMock.expect(responseHandler.getStatus()).andReturn(ok).anyTimes();
-
- EasyMock
- .expect(request.getUrl())
- .andReturn(new URL("http://test-host:1234/druid/coordinator/v1/metadata/segments"))
- .anyTimes();
-
- AppendableByteArrayInputStream in = new AppendableByteArrayInputStream();
- //segments in metadata store : wikipedia1, wikipedia2, wikipedia3, test1, test2
- final String json = "[{\n"
- + "\t\"dataSource\": \"wikipedia1\",\n"
- + "\t\"interval\": \"2018-08-07T23:00:00.000Z/2018-08-08T00:00:00.000Z\",\n"
- + "\t\"version\": \"2018-08-07T23:00:00.059Z\",\n"
- + "\t\"loadSpec\": {\n"
- + "\t\t\"type\": \"local\",\n"
- + "\t\t\"path\": \"/var/druid/segments/wikipedia-kafka/2018-08-07T23:00:00.000Z_2018-08-08T00:00:00.000Z/2018-08-07T23:00:00.059Z/51/1578eb79-0e44-4b41-a87b-65e40c52be53/index.zip\"\n"
- + "\t},\n"
- + "\t\"dimensions\": \"isRobot,channel,flags,isUnpatrolled,page,diffUrl,comment,isNew,isMinor,user,namespace,commentLength,deltaBucket,cityName,countryIsoCode,countryName,isAnonymous,regionIsoCode,regionName,added,deleted,delta\",\n"
- + "\t\"metrics\": \"count,user_unique\",\n"
- + "\t\"shardSpec\": {\n"
- + "\t\t\"type\": \"none\",\n"
- + "\t\t\"partitionNum\": 51,\n"
- + "\t\t\"partitions\": 0\n"
- + "\t},\n"
- + "\t\"binaryVersion\": 9,\n"
- + "\t\"size\": 47406,\n"
- + "\t\"identifier\": \"wikipedia-kafka_2018-08-07T23:00:00.000Z_2018-08-08T00:00:00.000Z_2018-08-07T23:00:00.059Z_51\"\n"
- + "}, {\n"
- + "\t\"dataSource\": \"wikipedia2\",\n"
- + "\t\"interval\": \"2018-08-07T18:00:00.000Z/2018-08-07T19:00:00.000Z\",\n"
- + "\t\"version\": \"2018-08-07T18:00:00.117Z\",\n"
- + "\t\"loadSpec\": {\n"
- + "\t\t\"type\": \"local\",\n"
- + "\t\t\"path\": \"/var/druid/segments/wikipedia-kafka/2018-08-07T18:00:00.000Z_2018-08-07T19:00:00.000Z/2018-08-07T18:00:00.117Z/9/a2646827-b782-424c-9eed-e48aa448d2c5/index.zip\"\n"
- + "\t},\n"
- + "\t\"dimensions\": \"isRobot,channel,flags,isUnpatrolled,page,diffUrl,comment,isNew,isMinor,user,namespace,commentLength,deltaBucket,cityName,countryIsoCode,countryName,isAnonymous,metroCode,regionIsoCode,regionName,added,deleted,delta\",\n"
- + "\t\"metrics\": \"count,user_unique\",\n"
- + "\t\"shardSpec\": {\n"
- + "\t\t\"type\": \"none\",\n"
- + "\t\t\"partitionNum\": 9,\n"
- + "\t\t\"partitions\": 0\n"
- + "\t},\n"
- + "\t\"binaryVersion\": 9,\n"
- + "\t\"size\": 83846,\n"
- + "\t\"identifier\": \"wikipedia-kafka_2018-08-07T18:00:00.000Z_2018-08-07T19:00:00.000Z_2018-08-07T18:00:00.117Z_9\"\n"
- + "}, {\n"
- + "\t\"dataSource\": \"wikipedia3\",\n"
- + "\t\"interval\": \"2018-08-07T23:00:00.000Z/2018-08-08T00:00:00.000Z\",\n"
- + "\t\"version\": \"2018-08-07T23:00:00.059Z\",\n"
- + "\t\"loadSpec\": {\n"
- + "\t\t\"type\": \"local\",\n"
- + "\t\t\"path\": \"/var/druid/segments/wikipedia-kafka/2018-08-07T23:00:00.000Z_2018-08-08T00:00:00.000Z/2018-08-07T23:00:00.059Z/50/87c5457e-c39b-4c03-9df8-e2b20b210dfc/index.zip\"\n"
- + "\t},\n"
- + "\t\"dimensions\": \"isRobot,channel,flags,isUnpatrolled,page,diffUrl,comment,isNew,isMinor,user,namespace,commentLength,deltaBucket,cityName,countryIsoCode,countryName,isAnonymous,metroCode,regionIsoCode,regionName,added,deleted,delta\",\n"
- + "\t\"metrics\": \"count,user_unique\",\n"
- + "\t\"shardSpec\": {\n"
- + "\t\t\"type\": \"none\",\n"
- + "\t\t\"partitionNum\": 50,\n"
- + "\t\t\"partitions\": 0\n"
- + "\t},\n"
- + "\t\"binaryVersion\": 9,\n"
- + "\t\"size\": 53527,\n"
- + "\t\"identifier\": \"wikipedia-kafka_2018-08-07T23:00:00.000Z_2018-08-08T00:00:00.000Z_2018-08-07T23:00:00.059Z_50\"\n"
- + "}, {\n"
- + "\t\"dataSource\": \"test1\",\n"
- + "\t\"interval\": \"2010-01-01T00:00:00.000Z/2011-01-01T00:00:00.000Z\",\n"
- + "\t\"version\": \"version1\",\n"
- + "\t\"loadSpec\": null,\n"
- + "\t\"dimensions\": \"dim1,dim2\",\n"
- + "\t\"metrics\": \"met1,met2\",\n"
- + "\t\"shardSpec\": {\n"
- + "\t\t\"type\": \"none\",\n"
- + "\t\t\"domainDimensions\": []\n"
- + "\t},\n"
- + "\t\"binaryVersion\": 1,\n"
- + "\t\"size\": 100,\n"
- + "\t\"identifier\": \"test1_2010-01-01T00:00:00.000Z_2011-01-01T00:00:00.000Z_version1\"\n"
- + "}, {\n"
- + "\t\"dataSource\": \"test2\",\n"
- + "\t\"interval\": \"2011-01-01T00:00:00.000Z/2012-01-01T00:00:00.000Z\",\n"
- + "\t\"version\": \"version2\",\n"
- + "\t\"loadSpec\": null,\n"
- + "\t\"dimensions\": \"dim1,dim2\",\n"
- + "\t\"metrics\": \"met1,met2\",\n"
- + "\t\"shardSpec\": {\n"
- + "\t\t\"type\": \"none\",\n"
- + "\t\t\"domainDimensions\": []\n"
- + "\t},\n"
- + "\t\"binaryVersion\": 1,\n"
- + "\t\"size\": 100,\n"
- + "\t\"identifier\": \"test2_2011-01-01T00:00:00.000Z_2012-01-01T00:00:00.000Z_version2\"\n"
- + "}]";
- byte[] bytesToWrite = json.getBytes(StandardCharsets.UTF_8);
- in.add(bytesToWrite);
- in.done();
- future.set(in);
-
- EasyMock.replay(client, request, responseHolder, responseHandler);
+ EasyMock.replay(client, request, responseHolder, responseHandler, metadataView);
DataContext dataContext = new DataContext()
{
@Override
@@ -531,7 +477,7 @@ public class SystemSchemaTest extends CalciteTestBase
verifyRow(
rows.get(3),
- "test4_2017-01-01T00:00:00.000Z_2018-01-01T00:00:00.000Z_version4",
+ "test4_2014-01-01T00:00:00.000Z_2015-01-01T00:00:00.000Z_version4",
100L,
0L, //partition_num
1L, //num_replicas
@@ -543,7 +489,7 @@ public class SystemSchemaTest extends CalciteTestBase
verifyRow(
rows.get(4),
- "test5_2017-01-01T00:00:00.000Z_2018-01-01T00:00:00.000Z_version5",
+ "test5_2015-01-01T00:00:00.000Z_2016-01-01T00:00:00.000Z_version5",
100L,
0L, //partition_num
1L, //num_replicas
@@ -556,8 +502,8 @@ public class SystemSchemaTest extends CalciteTestBase
// wikipedia segments are published and unavailable, num_replicas is 0
verifyRow(
rows.get(5),
- "wikipedia1_2018-08-07T23:00:00.000Z_2018-08-08T00:00:00.000Z_2018-08-07T23:00:00.059Z",
- 47406L,
+ "wikipedia1_2007-01-01T00:00:00.000Z_2008-01-01T00:00:00.000Z_version1",
+ 53000L,
0L, //partition_num
0L, //num_replicas
0L, //numRows
@@ -568,8 +514,8 @@ public class SystemSchemaTest extends CalciteTestBase
verifyRow(
rows.get(6),
- "wikipedia2_2018-08-07T18:00:00.000Z_2018-08-07T19:00:00.000Z_2018-08-07T18:00:00.117Z",
- 83846L,
+ "wikipedia2_2008-01-01T00:00:00.000Z_2009-01-01T00:00:00.000Z_version2",
+ 83000L,
0L, //partition_num
0L, //num_replicas
0L, //numRows
@@ -580,8 +526,8 @@ public class SystemSchemaTest extends CalciteTestBase
verifyRow(
rows.get(7),
- "wikipedia3_2018-08-07T23:00:00.000Z_2018-08-08T00:00:00.000Z_2018-08-07T23:00:00.059Z",
- 53527L,
+ "wikipedia3_2009-01-01T00:00:00.000Z_2010-01-01T00:00:00.000Z_version3",
+ 47000L,
0L, //partition_num
0L, //num_replicas
0L, //numRows
@@ -736,11 +682,11 @@ public class SystemSchemaTest extends CalciteTestBase
Object[] row3 = rows.get(3);
Assert.assertEquals("server2:1234", row3[0]);
- Assert.assertEquals("test4_2017-01-01T00:00:00.000Z_2018-01-01T00:00:00.000Z_version4", row3[1].toString());
+ Assert.assertEquals("test4_2014-01-01T00:00:00.000Z_2015-01-01T00:00:00.000Z_version4", row3[1].toString());
Object[] row4 = rows.get(4);
Assert.assertEquals("server2:1234", row4[0]);
- Assert.assertEquals("test5_2017-01-01T00:00:00.000Z_2018-01-01T00:00:00.000Z_version5", row4[1].toString());
+ Assert.assertEquals("test5_2015-01-01T00:00:00.000Z_2016-01-01T00:00:00.000Z_version5", row4[1].toString());
// Verify value types.
verifyTypes(rows, SystemSchema.SERVER_SEGMENTS_SIGNATURE);
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java
index 359ca25bd06..5beafb7bf23 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java
@@ -31,6 +31,7 @@ import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import org.apache.curator.x.discovery.ServiceProvider;
+import org.apache.druid.client.BrokerSegmentWatcherConfig;
import org.apache.druid.collections.CloseableStupidPool;
import org.apache.druid.curator.discovery.ServerDiscoverySelector;
import org.apache.druid.data.input.InputRow;
@@ -104,6 +105,7 @@ import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.server.QueryLifecycleFactory;
+import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler;
import org.apache.druid.server.log.NoopRequestLogger;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.AllowAllAuthenticator;
@@ -123,6 +125,7 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.sql.calcite.schema.MetadataSegmentView;
import org.apache.druid.sql.calcite.schema.SystemSchema;
import org.apache.druid.sql.calcite.view.NoopViewManager;
import org.apache.druid.sql.calcite.view.ViewManager;
@@ -739,7 +742,8 @@ public class CalciteTests
public static SystemSchema createMockSystemSchema(
final DruidSchema druidSchema,
- final SpecificSegmentsQuerySegmentWalker walker
+ final SpecificSegmentsQuerySegmentWalker walker,
+ final PlannerConfig plannerConfig
)
{
final DruidLeaderClient druidLeaderClient = new DruidLeaderClient(
@@ -753,6 +757,13 @@ public class CalciteTests
};
final SystemSchema schema = new SystemSchema(
druidSchema,
+ new MetadataSegmentView(
+ druidLeaderClient,
+ getJsonMapper(),
+ new BytesAccumulatingResponseHandler(),
+ new BrokerSegmentWatcherConfig(),
+ plannerConfig
+ ),
new TestServerInventoryView(walker.getSegments()),
TEST_AUTHORIZER_MAPPER,
druidLeaderClient,
From 953b96d0a436b2bd979c2aa6eecc3f636387633c Mon Sep 17 00:00:00 2001
From: Jonathan Wei
Date: Sat, 2 Feb 2019 22:34:53 -0800
Subject: [PATCH 04/13] Add more sketch aggregator support in Druid SQL (#6951)
* Add more sketch aggregator support in Druid SQL
* Add docs
* Tweak module serde register
* Fix tests
* Checkstyle
* Test fix
* PR comment
* PR comment
* PR comments
---
docs/content/querying/sql.md | 6 +-
extensions-core/datasketches/pom.xml | 20 +
.../hll/HllSketchAggregatorFactory.java | 6 +-
.../datasketches/hll/HllSketchModule.java | 27 +-
.../hll/sql/HllSketchSqlAggregator.java | 202 ++++++++
.../DoublesSketchAggregatorFactory.java | 2 +-
.../quantiles/DoublesSketchModule.java | 15 +-
.../sql/DoublesSketchSqlAggregator.java | 237 +++++++++
.../theta/SketchMergeAggregatorFactory.java | 9 +-
.../datasketches/theta/SketchModule.java | 32 +-
.../theta/sql/ThetaSketchSqlAggregator.java | 192 ++++++++
.../hll/HllSketchAggregatorTest.java | 6 +-
.../hll/sql/HllSketchSqlAggregatorTest.java | 393 +++++++++++++++
.../DoublesSketchAggregatorTest.java | 5 +-
.../sql/DoublesSketchSqlAggregatorTest.java | 408 ++++++++++++++++
.../theta/SketchAggregationTest.java | 5 +-
.../SketchAggregationWithSimpleDataTest.java | 4 +-
.../sql/ThetaSketchSqlAggregatorTest.java | 402 ++++++++++++++++
.../ApproximateHistogramDruidModule.java | 15 +-
...BucketsHistogramQuantileSqlAggregator.java | 322 +++++++++++++
.../ApproximateHistogramAggregationTest.java | 5 +-
.../ApproximateHistogramGroupByQueryTest.java | 4 +-
.../FixedBucketsHistogramAggregationTest.java | 5 +-
...FixedBucketsHistogramGroupByQueryTest.java | 3 +-
...etsHistogramQuantileSqlAggregatorTest.java | 452 ++++++++++++++++++
.../sql/QuantileSqlAggregatorTest.java | 390 ++++++++-------
26 files changed, 2908 insertions(+), 259 deletions(-)
create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregator.java
create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregator.java
create mode 100644 extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregator.java
create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java
create mode 100644 extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java
create mode 100644 extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java
create mode 100644 extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java
diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md
index 9a3f03e36ac..3f274918569 100644
--- a/docs/content/querying/sql.md
+++ b/docs/content/querying/sql.md
@@ -119,7 +119,11 @@ Only the COUNT aggregation can accept DISTINCT.
|`MAX(expr)`|Takes the maximum of numbers.|
|`AVG(expr)`|Averages numbers.|
|`APPROX_COUNT_DISTINCT(expr)`|Counts distinct values of expr, which can be a regular column or a hyperUnique column. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`.|
-|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or approxHistogram exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
+|`APPROX_COUNT_DISTINCT_DS_HLL(expr, [lgK, tgtHllType])`|Counts distinct values of expr, which can be a regular column or an [HLL sketch](../development/extensions-core/datasketches-hll.html) column. The `lgK` and `tgtHllType` parameters are described in the HLL sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to use this function.|
+|`APPROX_COUNT_DISTINCT_DS_THETA(expr, [size])`|Counts distinct values of expr, which can be a regular column or a [Theta sketch](../development/extensions-core/datasketches-theta.html) column. The `size` parameter is described in the Theta sketch documentation. This is always approximate, regardless of the value of "useApproximateCountDistinct". See also `COUNT(DISTINCT expr)`. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to use this function.|
+|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.html#approximate-histogram-aggregator) exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
+|`APPROX_QUANTILE_DS(expr, probability, [k])`|Computes approximate quantiles on numeric or [Quantiles sketch](../development/extensions-core/datasketches-quantiles.html) exprs. The "probability" should be between 0 and 1 (exclusive). The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extensions.html) must be loaded to use this function.|
+|`APPROX_QUANTILE_FIXED_BUCKETS(expr, probability, numBuckets, lowerLimit, upperLimit, [outlierHandlingMode])`|Computes approximate quantiles on numeric or [fixed buckets histogram](../development/extensions-core/approximate-histograms.html#fixed-buckets-histogram) exprs. The "probability" should be between 0 and 1 (exclusive). The `numBuckets`, `lowerLimit`, `upperLimit`, and `outlierHandlingMode` parameters are described in the fixed buckets histogram documentation. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
|`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positve rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.html) documentation for additional details.|
### Numeric functions
diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml
index e3c10e9c1fc..47017673f75 100644
--- a/extensions-core/datasketches/pom.xml
+++ b/extensions-core/datasketches/pom.xml
@@ -56,6 +56,12 @@
${project.parent.version}provided
+
+ org.apache.druid
+ druid-sql
+ ${project.parent.version}
+ provided
+ com.fasterxml.jackson.core
@@ -131,6 +137,20 @@
test-jartest
+
+ org.apache.druid
+ druid-server
+ ${project.parent.version}
+ test
+ test-jar
+
+
+ org.apache.druid
+ druid-sql
+ ${project.parent.version}
+ test-jar
+ test
+
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java
index 71a20decf69..2da37b840fc 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java
@@ -39,11 +39,11 @@ import java.util.Objects;
* Base class for both build and merge factories
* @author Alexander Saydakov
*/
-abstract class HllSketchAggregatorFactory extends AggregatorFactory
+public abstract class HllSketchAggregatorFactory extends AggregatorFactory
{
- static final int DEFAULT_LG_K = 12;
- static final TgtHllType DEFAULT_TGT_HLL_TYPE = TgtHllType.HLL_4;
+ public static final int DEFAULT_LG_K = 12;
+ public static final TgtHllType DEFAULT_TGT_HLL_TYPE = TgtHllType.HLL_4;
static final Comparator COMPARATOR =
Comparator.nullsFirst(Comparator.comparingDouble(HllSketch::getEstimate));
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 983b6e150f9..5a183d9ef6e 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
@@ -22,10 +22,13 @@ package org.apache.druid.query.aggregation.datasketches.hll;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Binder;
import com.yahoo.sketches.hll.HllSketch;
import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchSqlAggregator;
import org.apache.druid.segment.serde.ComplexMetrics;
+import org.apache.druid.sql.guice.SqlBindings;
import java.util.Collections;
import java.util.List;
@@ -48,15 +51,8 @@ public class HllSketchModule implements DruidModule
@Override
public void configure(final Binder binder)
{
- if (ComplexMetrics.getSerdeForType(TYPE_NAME) == null) {
- ComplexMetrics.registerSerde(TYPE_NAME, new HllSketchMergeComplexMetricSerde());
- }
- if (ComplexMetrics.getSerdeForType(BUILD_TYPE_NAME) == null) {
- ComplexMetrics.registerSerde(BUILD_TYPE_NAME, new HllSketchBuildComplexMetricSerde());
- }
- if (ComplexMetrics.getSerdeForType(MERGE_TYPE_NAME) == null) {
- ComplexMetrics.registerSerde(MERGE_TYPE_NAME, new HllSketchMergeComplexMetricSerde());
- }
+ registerSerde();
+ SqlBindings.addAggregator(binder, HllSketchSqlAggregator.class);
}
@Override
@@ -74,4 +70,17 @@ public class HllSketchModule implements DruidModule
);
}
+ @VisibleForTesting
+ public static void registerSerde()
+ {
+ if (ComplexMetrics.getSerdeForType(TYPE_NAME) == null) {
+ ComplexMetrics.registerSerde(TYPE_NAME, new HllSketchMergeComplexMetricSerde());
+ }
+ if (ComplexMetrics.getSerdeForType(BUILD_TYPE_NAME) == null) {
+ ComplexMetrics.registerSerde(BUILD_TYPE_NAME, new HllSketchBuildComplexMetricSerde());
+ }
+ if (ComplexMetrics.getSerdeForType(MERGE_TYPE_NAME) == null) {
+ ComplexMetrics.registerSerde(MERGE_TYPE_NAME, new HllSketchMergeComplexMetricSerde());
+ }
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregator.java
new file mode 100644
index 00000000000..13147ffdddf
--- /dev/null
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregator.java
@@ -0,0 +1,202 @@
+/*
+ * 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.sql;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.hll.HllSketchAggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory;
+import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.aggregation.Aggregation;
+import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.expression.Expressions;
+import org.apache.druid.sql.calcite.planner.Calcites;
+import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.table.RowSignature;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class HllSketchSqlAggregator implements SqlAggregator
+{
+ private static final SqlAggFunction FUNCTION_INSTANCE = new HllSketchSqlAggFunction();
+ private static final String NAME = "APPROX_COUNT_DISTINCT_DS_HLL";
+
+ @Override
+ public SqlAggFunction calciteFunction()
+ {
+ return FUNCTION_INSTANCE;
+ }
+
+ @Nullable
+ @Override
+ public Aggregation toDruidAggregation(
+ PlannerContext plannerContext,
+ RowSignature rowSignature,
+ RexBuilder rexBuilder,
+ String name,
+ AggregateCall aggregateCall,
+ Project project,
+ List existingAggregations,
+ boolean finalizeAggregations
+ )
+ {
+ // Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
+ // for string columns.
+ final RexNode columnRexNode = Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(0)
+ );
+
+ final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
+ if (columnArg == null) {
+ return null;
+ }
+
+ final int logK;
+ if (aggregateCall.getArgList().size() >= 2) {
+ final RexNode logKarg = Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(1)
+ );
+
+ if (!logKarg.isA(SqlKind.LITERAL)) {
+ // logK must be a literal in order to plan.
+ return null;
+ }
+
+ logK = ((Number) RexLiteral.value(logKarg)).intValue();
+ } else {
+ logK = HllSketchAggregatorFactory.DEFAULT_LG_K;
+ }
+
+ final String tgtHllType;
+ if (aggregateCall.getArgList().size() >= 3) {
+ final RexNode tgtHllTypeArg = Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(2)
+ );
+
+ if (!tgtHllTypeArg.isA(SqlKind.LITERAL)) {
+ // tgtHllType must be a literal in order to plan.
+ return null;
+ }
+
+ tgtHllType = RexLiteral.stringValue(tgtHllTypeArg);
+ } else {
+ tgtHllType = HllSketchAggregatorFactory.DEFAULT_TGT_HLL_TYPE.name();
+ }
+
+ final List virtualColumns = new ArrayList<>();
+ final AggregatorFactory aggregatorFactory;
+ final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
+
+ if (columnArg.isDirectColumnAccess() && rowSignature.getColumnType(columnArg.getDirectColumn()) == ValueType.COMPLEX) {
+ aggregatorFactory = new HllSketchMergeAggregatorFactory(aggregatorName, columnArg.getDirectColumn(), logK, tgtHllType);
+ } else {
+ final SqlTypeName sqlTypeName = columnRexNode.getType().getSqlTypeName();
+ final ValueType inputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName);
+ if (inputType == null) {
+ throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, aggregatorName);
+ }
+
+ final DimensionSpec dimensionSpec;
+
+ if (columnArg.isDirectColumnAccess()) {
+ dimensionSpec = columnArg.getSimpleExtraction().toDimensionSpec(null, inputType);
+ } else {
+ final ExpressionVirtualColumn virtualColumn = columnArg.toVirtualColumn(
+ Calcites.makePrefixedName(name, "v"),
+ inputType,
+ plannerContext.getExprMacroTable()
+ );
+ dimensionSpec = new DefaultDimensionSpec(virtualColumn.getOutputName(), null, inputType);
+ virtualColumns.add(virtualColumn);
+ }
+
+ aggregatorFactory = new HllSketchBuildAggregatorFactory(
+ aggregatorName,
+ dimensionSpec.getDimension(),
+ logK,
+ tgtHllType
+ );
+ }
+
+ return Aggregation.create(
+ virtualColumns,
+ Collections.singletonList(aggregatorFactory),
+ finalizeAggregations ? new FinalizingFieldAccessPostAggregator(
+ name,
+ aggregatorFactory.getName()
+ ) : null
+ );
+ }
+
+ private static class HllSketchSqlAggFunction extends SqlAggFunction
+ {
+ private static final String SIGNATURE = "'" + NAME + "(column, lgK, tgtHllType)'\n";
+
+ HllSketchSqlAggFunction()
+ {
+ super(
+ NAME,
+ null,
+ SqlKind.OTHER_FUNCTION,
+ ReturnTypes.explicit(SqlTypeName.BIGINT),
+ InferTypes.VARCHAR_1024,
+ OperandTypes.or(
+ OperandTypes.ANY,
+ OperandTypes.and(
+ OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL),
+ OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.STRING)
+ )
+ ),
+ SqlFunctionCategory.NUMERIC,
+ false,
+ false
+ );
+ }
+ }
+}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java
index 4b28e268a79..e8eb84ecf1a 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java
@@ -49,7 +49,7 @@ public class DoublesSketchAggregatorFactory extends AggregatorFactory
public static final Comparator COMPARATOR =
Comparator.nullsFirst(Comparator.comparingLong(DoublesSketch::getN));
- private static final int DEFAULT_K = 128;
+ public static final int DEFAULT_K = 128;
// Used for sketch size estimation.
private static final long MAX_STREAM_LENGTH = 1_000_000_000;
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 a3ad89e3edc..33f6949a584 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
@@ -22,10 +22,13 @@ package org.apache.druid.query.aggregation.datasketches.quantiles;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Binder;
import com.yahoo.sketches.quantiles.DoublesSketch;
import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchSqlAggregator;
import org.apache.druid.segment.serde.ComplexMetrics;
+import org.apache.druid.sql.guice.SqlBindings;
import java.util.Collections;
import java.util.List;
@@ -44,9 +47,8 @@ public class DoublesSketchModule implements DruidModule
@Override
public void configure(final Binder binder)
{
- if (ComplexMetrics.getSerdeForType(DOUBLES_SKETCH) == null) {
- ComplexMetrics.registerSerde(DOUBLES_SKETCH, new DoublesSketchComplexMetricSerde());
- }
+ registerSerde();
+ SqlBindings.addAggregator(binder, DoublesSketchSqlAggregator.class);
}
@Override
@@ -65,4 +67,11 @@ public class DoublesSketchModule implements DruidModule
);
}
+ @VisibleForTesting
+ public static void registerSerde()
+ {
+ if (ComplexMetrics.getSerdeForType(DOUBLES_SKETCH) == null) {
+ ComplexMetrics.registerSerde(DOUBLES_SKETCH, new DoublesSketchComplexMetricSerde());
+ }
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregator.java
new file mode 100644
index 00000000000..0a0a05ba272
--- /dev/null
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregator.java
@@ -0,0 +1,237 @@
+/*
+ * 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.sql;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.aggregation.Aggregation;
+import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.expression.Expressions;
+import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.table.RowSignature;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class DoublesSketchSqlAggregator implements SqlAggregator
+{
+ private static final SqlAggFunction FUNCTION_INSTANCE = new DoublesSketchSqlAggFunction();
+ private static final String NAME = "APPROX_QUANTILE_DS";
+
+ @Override
+ public SqlAggFunction calciteFunction()
+ {
+ return FUNCTION_INSTANCE;
+ }
+
+ @Nullable
+ @Override
+ public Aggregation toDruidAggregation(
+ final PlannerContext plannerContext,
+ final RowSignature rowSignature,
+ final RexBuilder rexBuilder,
+ final String name,
+ final AggregateCall aggregateCall,
+ final Project project,
+ final List existingAggregations,
+ final boolean finalizeAggregations
+ )
+ {
+ final DruidExpression input = Expressions.toDruidExpression(
+ plannerContext,
+ rowSignature,
+ Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(0)
+ )
+ );
+ if (input == null) {
+ return null;
+ }
+
+ final AggregatorFactory aggregatorFactory;
+ final String histogramName = StringUtils.format("%s:agg", name);
+ final RexNode probabilityArg = Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(1)
+ );
+
+ if (!probabilityArg.isA(SqlKind.LITERAL)) {
+ // Probability must be a literal in order to plan.
+ return null;
+ }
+
+ final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
+ final int k;
+
+ if (aggregateCall.getArgList().size() >= 3) {
+ final RexNode resolutionArg = Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(2)
+ );
+
+ if (!resolutionArg.isA(SqlKind.LITERAL)) {
+ // Resolution must be a literal in order to plan.
+ return null;
+ }
+
+ k = ((Number) RexLiteral.value(resolutionArg)).intValue();
+ } else {
+ k = DoublesSketchAggregatorFactory.DEFAULT_K;
+ }
+
+ // Look for existing matching aggregatorFactory.
+ for (final Aggregation existing : existingAggregations) {
+ for (AggregatorFactory factory : existing.getAggregatorFactories()) {
+ if (factory instanceof DoublesSketchAggregatorFactory) {
+ final DoublesSketchAggregatorFactory theFactory = (DoublesSketchAggregatorFactory) factory;
+
+ // Check input for equivalence.
+ final boolean inputMatches;
+ final VirtualColumn virtualInput = existing.getVirtualColumns()
+ .stream()
+ .filter(
+ virtualColumn ->
+ virtualColumn.getOutputName()
+ .equals(theFactory.getFieldName())
+ )
+ .findFirst()
+ .orElse(null);
+
+ if (virtualInput == null) {
+ inputMatches = input.isDirectColumnAccess()
+ && input.getDirectColumn().equals(theFactory.getFieldName());
+ } else {
+ inputMatches = ((ExpressionVirtualColumn) virtualInput).getExpression()
+ .equals(input.getExpression());
+ }
+
+ final boolean matches = inputMatches
+ && theFactory.getK() == k;
+
+ if (matches) {
+ // Found existing one. Use this.
+ return Aggregation.create(
+ ImmutableList.of(),
+ new DoublesSketchToQuantilePostAggregator(
+ name,
+ new FieldAccessPostAggregator(
+ factory.getName(),
+ factory.getName()
+ ),
+ probability
+ )
+ );
+ }
+ }
+ }
+ }
+
+ // No existing match found. Create a new one.
+ final List virtualColumns = new ArrayList<>();
+
+ if (input.isDirectColumnAccess()) {
+ aggregatorFactory = new DoublesSketchAggregatorFactory(
+ histogramName,
+ input.getDirectColumn(),
+ k
+ );
+ } else {
+ final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
+ StringUtils.format("%s:v", name),
+ ValueType.FLOAT,
+ plannerContext.getExprMacroTable()
+ );
+ virtualColumns.add(virtualColumn);
+ aggregatorFactory = new DoublesSketchAggregatorFactory(
+ histogramName,
+ virtualColumn.getOutputName(),
+ k
+ );
+ }
+
+ return Aggregation.create(
+ virtualColumns,
+ ImmutableList.of(aggregatorFactory),
+ new DoublesSketchToQuantilePostAggregator(
+ name,
+ new FieldAccessPostAggregator(
+ histogramName,
+ histogramName
+ ),
+ probability
+ )
+ );
+ }
+
+ private static class DoublesSketchSqlAggFunction extends SqlAggFunction
+ {
+ private static final String SIGNATURE1 = "'" + NAME + "(column, probability)'\n";
+ private static final String SIGNATURE2 = "'" + NAME + "(column, probability, k)'\n";
+
+ DoublesSketchSqlAggFunction()
+ {
+ super(
+ NAME,
+ null,
+ SqlKind.OTHER_FUNCTION,
+ ReturnTypes.explicit(SqlTypeName.DOUBLE),
+ null,
+ OperandTypes.or(
+ OperandTypes.and(
+ OperandTypes.sequence(SIGNATURE1, OperandTypes.ANY, OperandTypes.LITERAL),
+ OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
+ ),
+ OperandTypes.and(
+ OperandTypes.sequence(SIGNATURE2, OperandTypes.ANY, OperandTypes.LITERAL, OperandTypes.LITERAL),
+ OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC, SqlTypeFamily.EXACT_NUMERIC)
+ )
+ ),
+ SqlFunctionCategory.NUMERIC,
+ false,
+ false
+ );
+ }
+ }
+}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java
index 4701e8fbf0b..d172b82bb49 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java
@@ -25,6 +25,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
import org.apache.druid.query.aggregation.AggregatorUtil;
+import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
@@ -38,10 +39,10 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory
public SketchMergeAggregatorFactory(
@JsonProperty("name") String name,
@JsonProperty("fieldName") String fieldName,
- @JsonProperty("size") Integer size,
- @JsonProperty("shouldFinalize") Boolean shouldFinalize,
- @JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch,
- @JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev
+ @Nullable @JsonProperty("size") Integer size,
+ @Nullable @JsonProperty("shouldFinalize") Boolean shouldFinalize,
+ @Nullable @JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch,
+ @Nullable @JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev
)
{
super(name, fieldName, size, AggregatorUtil.SKETCH_MERGE_CACHE_TYPE_ID);
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java
index a2083e108f6..6264502d4a8 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java
@@ -22,9 +22,12 @@ package org.apache.druid.query.aggregation.datasketches.theta;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Binder;
import org.apache.druid.initialization.DruidModule;
+import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchSqlAggregator;
import org.apache.druid.segment.serde.ComplexMetrics;
+import org.apache.druid.sql.guice.SqlBindings;
import java.util.Collections;
import java.util.List;
@@ -44,17 +47,8 @@ public class SketchModule implements DruidModule
@Override
public void configure(Binder binder)
{
- if (ComplexMetrics.getSerdeForType(THETA_SKETCH) == null) {
- ComplexMetrics.registerSerde(THETA_SKETCH, new SketchMergeComplexMetricSerde());
- }
-
- if (ComplexMetrics.getSerdeForType(THETA_SKETCH_MERGE_AGG) == null) {
- ComplexMetrics.registerSerde(THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde());
- }
-
- if (ComplexMetrics.getSerdeForType(THETA_SKETCH_BUILD_AGG) == null) {
- ComplexMetrics.registerSerde(THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde());
- }
+ registerSerde();
+ SqlBindings.addAggregator(binder, ThetaSketchSqlAggregator.class);
}
@Override
@@ -71,4 +65,20 @@ public class SketchModule implements DruidModule
.addSerializer(SketchHolder.class, new SketchHolderJsonSerializer())
);
}
+
+ @VisibleForTesting
+ public static void registerSerde()
+ {
+ if (ComplexMetrics.getSerdeForType(THETA_SKETCH) == null) {
+ ComplexMetrics.registerSerde(THETA_SKETCH, new SketchMergeComplexMetricSerde());
+ }
+
+ if (ComplexMetrics.getSerdeForType(THETA_SKETCH_MERGE_AGG) == null) {
+ ComplexMetrics.registerSerde(THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde());
+ }
+
+ if (ComplexMetrics.getSerdeForType(THETA_SKETCH_BUILD_AGG) == null) {
+ ComplexMetrics.registerSerde(THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde());
+ }
+ }
}
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregator.java
new file mode 100644
index 00000000000..f202c1a9533
--- /dev/null
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregator.java
@@ -0,0 +1,192 @@
+/*
+ * 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.sql;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.theta.SketchAggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory;
+import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.segment.VirtualColumn;
+import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.sql.calcite.aggregation.Aggregation;
+import org.apache.druid.sql.calcite.aggregation.SqlAggregator;
+import org.apache.druid.sql.calcite.expression.DruidExpression;
+import org.apache.druid.sql.calcite.expression.Expressions;
+import org.apache.druid.sql.calcite.planner.Calcites;
+import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.table.RowSignature;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ThetaSketchSqlAggregator implements SqlAggregator
+{
+ private static final SqlAggFunction FUNCTION_INSTANCE = new ThetaSketchSqlAggFunction();
+ private static final String NAME = "APPROX_COUNT_DISTINCT_DS_THETA";
+
+ @Override
+ public SqlAggFunction calciteFunction()
+ {
+ return FUNCTION_INSTANCE;
+ }
+
+ @Nullable
+ @Override
+ public Aggregation toDruidAggregation(
+ PlannerContext plannerContext,
+ RowSignature rowSignature,
+ RexBuilder rexBuilder,
+ String name,
+ AggregateCall aggregateCall,
+ Project project,
+ List existingAggregations,
+ boolean finalizeAggregations
+ )
+ {
+ // Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access
+ // for string columns.
+ final RexNode columnRexNode = Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(0)
+ );
+
+ final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode);
+ if (columnArg == null) {
+ return null;
+ }
+
+ final int sketchSize;
+ if (aggregateCall.getArgList().size() >= 2) {
+ final RexNode sketchSizeArg = Expressions.fromFieldAccess(
+ rowSignature,
+ project,
+ aggregateCall.getArgList().get(1)
+ );
+
+ if (!sketchSizeArg.isA(SqlKind.LITERAL)) {
+ // logK must be a literal in order to plan.
+ return null;
+ }
+
+ sketchSize = ((Number) RexLiteral.value(sketchSizeArg)).intValue();
+ } else {
+ sketchSize = SketchAggregatorFactory.DEFAULT_MAX_SKETCH_SIZE;
+ }
+
+ final List virtualColumns = new ArrayList<>();
+ final AggregatorFactory aggregatorFactory;
+ final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name;
+
+ if (columnArg.isDirectColumnAccess() && rowSignature.getColumnType(columnArg.getDirectColumn()) == ValueType.COMPLEX) {
+ aggregatorFactory = new SketchMergeAggregatorFactory(
+ aggregatorName,
+ columnArg.getDirectColumn(),
+ sketchSize,
+ null,
+ null,
+ null
+ );
+ } else {
+ final SqlTypeName sqlTypeName = columnRexNode.getType().getSqlTypeName();
+ final ValueType inputType = Calcites.getValueTypeForSqlTypeName(sqlTypeName);
+ if (inputType == null) {
+ throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, aggregatorName);
+ }
+
+ final DimensionSpec dimensionSpec;
+
+ if (columnArg.isDirectColumnAccess()) {
+ dimensionSpec = columnArg.getSimpleExtraction().toDimensionSpec(null, inputType);
+ } else {
+ final ExpressionVirtualColumn virtualColumn = columnArg.toVirtualColumn(
+ Calcites.makePrefixedName(name, "v"),
+ inputType,
+ plannerContext.getExprMacroTable()
+ );
+ dimensionSpec = new DefaultDimensionSpec(virtualColumn.getOutputName(), null, inputType);
+ virtualColumns.add(virtualColumn);
+ }
+
+ aggregatorFactory = new SketchMergeAggregatorFactory(
+ aggregatorName,
+ dimensionSpec.getDimension(),
+ sketchSize,
+ null,
+ null,
+ null
+ );
+ }
+
+ return Aggregation.create(
+ virtualColumns,
+ Collections.singletonList(aggregatorFactory),
+ finalizeAggregations ? new FinalizingFieldAccessPostAggregator(
+ name,
+ aggregatorFactory.getName()
+ ) : null
+ );
+ }
+
+ private static class ThetaSketchSqlAggFunction extends SqlAggFunction
+ {
+ private static final String SIGNATURE = "'" + NAME + "(column, size)'\n";
+
+ ThetaSketchSqlAggFunction()
+ {
+ super(
+ NAME,
+ null,
+ SqlKind.OTHER_FUNCTION,
+ ReturnTypes.explicit(SqlTypeName.BIGINT),
+ InferTypes.VARCHAR_1024,
+ OperandTypes.or(
+ OperandTypes.ANY,
+ OperandTypes.and(
+ OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL),
+ OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC)
+ )
+ ),
+ SqlFunctionCategory.NUMERIC,
+ false,
+ false
+ );
+ }
+ }
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java
index 1eb21ed9cf5..f0b995e980a 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java
@@ -20,7 +20,6 @@
package org.apache.druid.query.aggregation.datasketches.hll;
import org.apache.druid.data.input.Row;
-import org.apache.druid.initialization.DruidModule;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.aggregation.AggregationTestHelper;
@@ -48,10 +47,9 @@ public class HllSketchAggregatorTest
public HllSketchAggregatorTest(GroupByQueryConfig config)
{
- DruidModule module = new HllSketchModule();
- module.configure(null);
+ HllSketchModule.registerSerde();
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
- module.getJacksonModules(), config, tempFolder);
+ new HllSketchModule().getJacksonModules(), config, tempFolder);
}
@Parameterized.Parameters(name = "{0}")
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
new file mode 100644
index 00000000000..46eb02c01c8
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java
@@ -0,0 +1,393 @@
+/*
+ * 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.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 org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryDataSource;
+import org.apache.druid.query.QueryRunnerFactoryConglomerate;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
+import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.hll.HllSketchMergeAggregatorFactory;
+import org.apache.druid.query.aggregation.datasketches.hll.HllSketchModule;
+import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.expression.TestExprMacroTable;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
+import org.apache.druid.segment.IndexBuilder;
+import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
+import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
+import org.apache.druid.server.security.AuthTestUtils;
+import org.apache.druid.server.security.AuthenticationResult;
+import org.apache.druid.sql.SqlLifecycle;
+import org.apache.druid.sql.SqlLifecycleFactory;
+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.planner.PlannerConfig;
+import org.apache.druid.sql.calcite.planner.PlannerContext;
+import org.apache.druid.sql.calcite.planner.PlannerFactory;
+import org.apache.druid.sql.calcite.schema.DruidSchema;
+import org.apache.druid.sql.calcite.schema.SystemSchema;
+import org.apache.druid.sql.calcite.util.CalciteTestBase;
+import org.apache.druid.sql.calcite.util.CalciteTests;
+import org.apache.druid.sql.calcite.util.QueryLogHook;
+import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class HllSketchSqlAggregatorTest extends CalciteTestBase
+{
+ private static final String DATA_SOURCE = "foo";
+
+ private static QueryRunnerFactoryConglomerate conglomerate;
+ private static Closer resourceCloser;
+ private static AuthenticationResult authenticationResult = CalciteTests.REGULAR_USER_AUTH_RESULT;
+ private static final Map QUERY_CONTEXT_DEFAULT = ImmutableMap.of(
+ PlannerContext.CTX_SQL_QUERY_ID, "dummy"
+ );
+
+ @BeforeClass
+ public static void setUpClass()
+ {
+ final Pair conglomerateCloserPair = CalciteTests
+ .createQueryRunnerFactoryConglomerate();
+ conglomerate = conglomerateCloserPair.lhs;
+ resourceCloser = conglomerateCloserPair.rhs;
+ }
+
+ @AfterClass
+ public static void tearDownClass() throws IOException
+ {
+ resourceCloser.close();
+ }
+
+ @Rule
+ public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+ @Rule
+ public QueryLogHook queryLogHook = QueryLogHook.create();
+
+ private SpecificSegmentsQuerySegmentWalker walker;
+ private SqlLifecycleFactory sqlLifecycleFactory;
+
+ @Before
+ public void setUp() throws Exception
+ {
+ HllSketchModule.registerSerde();
+ for (Module mod : new HllSketchModule().getJacksonModules()) {
+ CalciteTests.getJsonMapper().registerModule(mod);
+ }
+
+ final QueryableIndex index = IndexBuilder.create()
+ .tmpDir(temporaryFolder.newFolder())
+ .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
+ .schema(
+ new IncrementalIndexSchema.Builder()
+ .withMetrics(
+ new CountAggregatorFactory("cnt"),
+ new DoubleSumAggregatorFactory("m1", "m1"),
+ new HllSketchBuildAggregatorFactory(
+ "hllsketch_dim1",
+ "dim1",
+ null,
+ null
+ )
+ )
+ .withRollup(false)
+ .build()
+ )
+ .rows(CalciteTests.ROWS1)
+ .buildMMappedIndex();
+
+ walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
+ DataSegment.builder()
+ .dataSource(DATA_SOURCE)
+ .interval(index.getDataInterval())
+ .version("1")
+ .shardSpec(new LinearShardSpec(0))
+ .build(),
+ index
+ );
+
+ final PlannerConfig plannerConfig = new PlannerConfig();
+ final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
+ final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker);
+ final DruidOperatorTable operatorTable = new DruidOperatorTable(
+ ImmutableSet.of(new HllSketchSqlAggregator()),
+ ImmutableSet.of()
+ );
+
+ sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(
+ new PlannerFactory(
+ druidSchema,
+ systemSchema,
+ CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
+ operatorTable,
+ CalciteTests.createExprMacroTable(),
+ plannerConfig,
+ AuthTestUtils.TEST_AUTHORIZER_MAPPER,
+ CalciteTests.getJsonMapper()
+ )
+ );
+ }
+
+ @After
+ public void tearDown() throws Exception
+ {
+ walker.close();
+ walker = null;
+ }
+
+ @Test
+ public void testApproxCountDistinctHllSketch() throws Exception
+ {
+ SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
+
+ final String sql = "SELECT\n"
+ + " SUM(cnt),\n"
+ + " APPROX_COUNT_DISTINCT_DS_HLL(dim2),\n" // uppercase
+ + " APPROX_COUNT_DISTINCT_DS_HLL(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered
+ + " APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1)),\n" // on extractionFn
+ + " APPROX_COUNT_DISTINCT_DS_HLL(SUBSTRING(dim2, 1, 1) || 'x'),\n" // on expression
+ + " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1, 21, 'HLL_8'),\n" // on native HllSketch column
+ + " APPROX_COUNT_DISTINCT_DS_HLL(hllsketch_dim1)\n" // on native HllSketch column
+ + "FROM druid.foo";
+
+ // Verify results
+ final List
}
;
diff --git a/web-console/src/utils/general.tsx b/web-console/src/utils/general.tsx
index 32835f90cd6..873e634bc31 100644
--- a/web-console/src/utils/general.tsx
+++ b/web-console/src/utils/general.tsx
@@ -121,6 +121,11 @@ export function pluralIfNeeded(n: number, singular: string, plural?: string): st
return `${formatNumber(n)} ${n === 1 ? singular : plural}`;
}
+export function getHeadProp(results: Record[], prop: string): any {
+ if (!results || !results.length) return null;
+ return results[0][prop] || null;
+}
+
// ----------------------------
export function localStorageSet(key: string, value: string): void {
diff --git a/web-console/src/views/home-view.tsx b/web-console/src/views/home-view.tsx
index 4f8ed85b4f8..7258fe861ca 100644
--- a/web-console/src/views/home-view.tsx
+++ b/web-console/src/views/home-view.tsx
@@ -21,7 +21,7 @@ import * as React from 'react';
import * as classNames from 'classnames';
import { H5, Card, Icon } from "@blueprintjs/core";
import { IconName, IconNames } from "@blueprintjs/icons";
-import { QueryManager, pluralIfNeeded, queryDruidSql } from '../utils';
+import { QueryManager, pluralIfNeeded, queryDruidSql, getHeadProp } from '../utils';
import './home-view.scss';
export interface CardOptions {
@@ -147,7 +147,7 @@ export class HomeView extends React.Component {
this.segmentQueryManager = new QueryManager({
processQuery: async (query) => {
const segments = await queryDruidSql({ query });
- return segments[0].count;
+ return getHeadProp(segments, 'count') || 0;
},
onStateChange: ({ result, loading, error }) => {
this.setState({
@@ -211,7 +211,7 @@ GROUP BY 1`);
this.dataServerQueryManager = new QueryManager({
processQuery: async (query) => {
const dataServerCounts = await queryDruidSql({ query });
- return dataServerCounts[0].count;
+ return getHeadProp(dataServerCounts, 'count') || 0;
},
onStateChange: ({ result, loading, error }) => {
this.setState({
diff --git a/web-console/src/views/tasks-view.tsx b/web-console/src/views/tasks-view.tsx
index 78c69e599ba..8f394584f33 100644
--- a/web-console/src/views/tasks-view.tsx
+++ b/web-console/src/views/tasks-view.tsx
@@ -167,6 +167,7 @@ ORDER BY "rank" DESC, "created_time" DESC`);
message: 'Supervisor submitted successfully',
intent: Intent.SUCCESS
});
+ this.supervisorQueryManager.rerunLastQuery();
}
private async submitTask(spec: JSON) {
@@ -184,6 +185,7 @@ ORDER BY "rank" DESC, "created_time" DESC`);
message: 'Task submitted successfully',
intent: Intent.SUCCESS
});
+ this.taskQueryManager.rerunLastQuery();
}
renderResumeSupervisorAction() {
@@ -192,7 +194,7 @@ ORDER BY "rank" DESC, "created_time" DESC`);
return {
- const resp = await axios.post(`/druid/indexer/v1/supervisor/${resumeSupervisorId}/suspend`, {})
+ const resp = await axios.post(`/druid/indexer/v1/supervisor/${resumeSupervisorId}/suspend`, {});
return resp.data;
} : null
}
@@ -489,7 +491,7 @@ ORDER BY "rank" DESC, "created_time" DESC`);
Header: 'Actions',
id: 'actions',
accessor: 'task_id',
- width: 320,
+ width: 360,
filterable: false,
Cell: row => {
if (row.aggregated) return '';
From 0e926e865276f892ddc5b62685d331b822104022 Mon Sep 17 00:00:00 2001
From: Roman Leventov
Date: Tue, 5 Feb 2019 00:18:12 +0700
Subject: [PATCH 07/13] Prohibit assigning concurrent maps into Map-typed
variables and fields and fix a race condition in CoordinatorRuleManager
(#6898)
* Prohibit assigning concurrent maps into Map-types variables and fields; Fix a race condition in CoordinatorRuleManager; improve logic in DirectDruidClient and ResourcePool
* Enforce that if compute(), computeIfAbsent(), computeIfPresent() or merge() is called on a ConcurrentHashMap, it's stored in a ConcurrentHashMap-typed variable, not ConcurrentMap; add comments explaining get()-before-computeIfAbsent() optimization; refactor Counters; fix a race condition in Intialization.java
* Remove unnecessary comment
* Checkstyle
* Fix getFromExtensions()
* Add a reference to the comment about guarded computeIfAbsent() optimization; IdentityHashMap optimization
* Fix UriCacheGeneratorTest
* Workaround issue with MaterializedViewQueryQueryToolChest
* Strengthen Appenderator's contract regarding concurrency
---
.idea/inspectionProfiles/Druid.xml | 34 +++++++-
.../FilteredAggregatorBenchmark.java | 2 +-
.../GroupByTypeInterfaceBenchmark.java | 2 +-
.../benchmark/TopNTypeInterfaceBenchmark.java | 5 +-
.../benchmark/query/GroupByBenchmark.java | 2 +-
.../benchmark/query/QueryBenchmarkUtil.java | 2 +-
.../benchmark/query/SearchBenchmark.java | 2 +-
.../benchmark/query/SelectBenchmark.java | 2 +-
.../benchmark/query/TimeseriesBenchmark.java | 2 +-
.../druid/benchmark/query/TopNBenchmark.java | 5 +-
.../timecompare/TimeCompareBenchmark.java | 7 +-
.../emitter/core/ParametrizedUriEmitter.java | 10 +--
.../util/http/client/pool/ResourcePool.java | 26 +++++--
...terializedViewQueryQueryToolChestTest.java | 2 +-
.../segment/MapVirtualColumnGroupByTest.java | 2 +-
.../segment/MapVirtualColumnSelectTest.java | 2 +-
.../segment/MapVirtualColumnTopNTest.java | 2 +-
.../ApproximateHistogramTopNQueryTest.java | 4 +-
.../FixedBucketsHistogramTopNQueryTest.java | 4 +-
.../lookup/KafkaLookupExtractorFactory.java | 3 +-
.../kafka/LegacyKafkaIndexTaskRunner.java | 5 +-
.../indexing/kafka/KafkaIndexTaskTest.java | 4 +-
.../kinesis/KinesisRecordSupplier.java | 4 +-
.../kinesis/KinesisIndexTaskTest.java | 4 +-
.../namespace/cache/CacheScheduler.java | 6 +-
...OnHeapNamespaceExtractionCacheManager.java | 2 +-
.../namespace/UriCacheGeneratorTest.java | 11 +--
.../variance/VarianceTopNQueryTest.java | 2 +-
.../druid/indexing/common/Counters.java | 59 +++++---------
.../common/actions/TaskActionToolbox.java | 11 +--
.../parallel/ParallelIndexSupervisorTask.java | 6 +-
.../SinglePhaseParallelIndexTaskRunner.java | 5 +-
.../indexing/overlord/ForkingTaskRunner.java | 5 +-
.../druid/indexing/overlord/TaskQueue.java | 11 ++-
.../SeekableStreamIndexTaskRunner.java | 5 +-
.../indexing/worker/WorkerTaskManager.java | 4 +-
.../common/actions/TaskActionTestKit.java | 4 +-
...penderatorDriverRealtimeIndexTaskTest.java | 4 +-
.../common/task/IngestionTestBase.java | 4 +-
.../common/task/RealtimeIndexTaskTest.java | 4 +-
...rallelIndexSupervisorTaskResourceTest.java | 12 +--
.../IngestSegmentFirehoseFactoryTest.java | 4 +-
.../indexing/overlord/TaskLifecycleTest.java | 4 +-
.../indexing/overlord/TestTaskRunner.java | 3 +-
.../hrtr/HttpRemoteTaskRunnerTest.java | 9 ++-
...DefaultQueryRunnerFactoryConglomerate.java | 9 ++-
.../query/MapQueryToolChestWarehouse.java | 9 ++-
.../apache/druid/query/QueryToolChest.java | 38 +++++++++
.../druid/query/ReflectionLoaderThingy.java | 77 -------------------
.../ReflectionQueryToolChestWarehouse.java | 22 +++++-
.../DataSourceQueryQueryToolChest.java | 5 +-
.../SpecializationService.java | 7 +-
.../segment/incremental/IncrementalIndex.java | 4 +-
.../druid/query/MultiValuedDimensionTest.java | 2 +-
.../druid/query/QueryRunnerTestHelper.java | 4 +-
.../druid/query/RetryQueryRunnerTest.java | 11 +--
.../apache/druid/query/TestQueryRunners.java | 6 +-
.../aggregation/AggregationTestHelper.java | 8 +-
.../DataSourceMetadataQueryTest.java | 3 +-
.../groupby/GroupByQueryMergeBufferTest.java | 7 +-
.../GroupByQueryRunnerFailureTest.java | 7 +-
.../query/groupby/GroupByQueryRunnerTest.java | 8 +-
.../groupby/NestedQueryPushDownTest.java | 6 +-
.../query/search/SearchQueryRunnerTest.java | 2 +-
.../search/SearchQueryRunnerWithCaseTest.java | 4 +-
.../select/MultiSegmentSelectQueryTest.java | 2 +-
.../select/SelectQueryQueryToolChestTest.java | 2 +-
.../query/select/SelectQueryRunnerTest.java | 2 +-
.../TimeBoundaryQueryRunnerTest.java | 6 +-
.../TimeSeriesUnionQueryRunnerTest.java | 4 +-
.../TimeseriesQueryRunnerBonusTest.java | 2 +-
.../timeseries/TimeseriesQueryRunnerTest.java | 10 +--
.../topn/TopNQueryQueryToolChestTest.java | 2 +-
.../query/topn/TopNQueryRunnerBenchmark.java | 2 +-
.../druid/query/topn/TopNQueryRunnerTest.java | 4 +-
.../druid/query/topn/TopNUnionQueryTest.java | 4 +-
.../IndexMergerV9WithSpatialIndexTest.java | 6 +-
.../segment/data/IncrementalIndexTest.java | 4 +-
.../filter/SpatialFilterBonusTest.java | 6 +-
.../segment/filter/SpatialFilterTest.java | 6 +-
.../OnheapIncrementalIndexBenchmark.java | 2 +-
.../druid/client/DirectDruidClient.java | 38 ++-------
.../CuratorDruidNodeDiscoveryProvider.java | 8 +-
.../discovery/DruidNodeDiscoveryProvider.java | 3 +-
.../druid/initialization/Initialization.java | 21 ++---
.../realtime/appenderator/Appenderator.java | 52 +++++++------
.../appenderator/AppenderatorImpl.java | 23 +++---
.../appenderator/AppenderatorPlumber.java | 6 +-
.../appenderator/BaseAppenderatorDriver.java | 1 +
.../EventReceiverFirehoseFactory.java | 13 ++--
.../realtime/plumber/RealtimePlumber.java | 3 +-
.../apache/druid/server/QueryLifecycle.java | 3 +-
.../BatchDataSegmentAnnouncer.java | 4 +-
.../coordinator/ReplicationThrottler.java | 14 ++--
.../helper/DruidCoordinatorBalancer.java | 4 +-
.../server/http/DataSourcesResource.java | 6 +-
.../druid/server/http/TiersResource.java | 6 +-
.../server/router/CoordinatorRuleManager.java | 17 ++--
.../client/CachingClusteredClientTest.java | 30 ++++----
.../CachingClusteredClientTestUtils.java | 8 +-
.../druid/client/CachingQueryRunnerTest.java | 9 ++-
.../apache/druid/sql/avatica/DruidMeta.java | 11 ++-
.../druid/sql/calcite/util/CalciteTests.java | 8 +-
103 files changed, 442 insertions(+), 461 deletions(-)
delete mode 100644 processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java
diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml
index a79a44c4495..70a6a497bc7 100644
--- a/.idea/inspectionProfiles/Druid.xml
+++ b/.idea/inspectionProfiles/Druid.xml
@@ -147,7 +147,7 @@
-
+
@@ -236,6 +236,38 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
index 2116a14c86b..e156d24b38c 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
@@ -201,7 +201,7 @@ public class FilteredAggregatorBenchmark
factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
index 43a353cd2db..9a13f7c6845 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
@@ -407,7 +407,7 @@ public class GroupByTypeInterfaceBenchmark
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
- QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
)
);
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
index 2974441ae2a..3d01e1a3eec 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java
@@ -299,7 +299,10 @@ public class TopNTypeInterfaceBenchmark
0,
Integer.MAX_VALUE
),
- new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TopNQueryQueryToolChest(
+ new TopNQueryConfig(),
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
+ ),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
index 243564c0e29..f96aef616c1 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
@@ -495,7 +495,7 @@ public class GroupByBenchmark
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
- QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
)
);
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
index 411ca226e00..6e6ae2a82b7 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java
@@ -49,7 +49,7 @@ public class QueryBenchmarkUtil
);
}
- public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
+ public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{
return new IntervalChunkingQueryRunnerDecorator(null, null, null) {
@Override
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
index 5c84a6799bc..122914ba00e 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java
@@ -373,7 +373,7 @@ public class SearchBenchmark
new SearchStrategySelector(Suppliers.ofInstance(config)),
new SearchQueryQueryToolChest(
config,
- QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
index bd212646566..36fd251ba31 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java
@@ -233,7 +233,7 @@ public class SelectBenchmark
factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
JSON_MAPPER,
- QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator(),
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator(),
selectConfigSupplier
),
new SelectQueryEngine(),
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
index 94efa930e06..5c58c94ff87 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java
@@ -296,7 +296,7 @@ public class TimeseriesBenchmark
factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
index 717cd36968b..566ebe326b2 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java
@@ -278,7 +278,10 @@ public class TopNBenchmark
0,
Integer.MAX_VALUE
),
- new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TopNQueryQueryToolChest(
+ new TopNQueryConfig(),
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
+ ),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
}
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
index 179bbd7cb7a..85ef6007af8 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java
@@ -229,7 +229,10 @@ public class TimeCompareBenchmark
0,
Integer.MAX_VALUE
),
- new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TopNQueryQueryToolChest(
+ new TopNQueryConfig(),
+ QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
+ ),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
}
@@ -270,7 +273,7 @@ public class TimeCompareBenchmark
timeseriesQuery = timeseriesQueryBuilder.build();
timeseriesFactory = new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java b/core/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java
index 2588b32614b..8e4db9edf8b 100644
--- a/core/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java
+++ b/core/src/main/java/org/apache/druid/java/util/emitter/core/ParametrizedUriEmitter.java
@@ -120,21 +120,19 @@ public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
{
try {
URI uri = uriExtractor.apply(event);
+ // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed.
+ // See https://github.com/apache/incubator-druid/pull/6898#discussion_r251384586.
HttpPostEmitter emitter = emitters.get(uri);
if (emitter == null) {
try {
emitter = emitters.computeIfAbsent(uri, u -> {
try {
return innerLifecycle.addMaybeStartManagedInstance(
- new HttpPostEmitter(
- config.buildHttpEmitterConfig(u.toString()),
- client,
- jsonMapper
- )
+ new HttpPostEmitter(config.buildHttpEmitterConfig(u.toString()), client, jsonMapper)
);
}
catch (Exception e) {
- throw Throwables.propagate(e);
+ throw new RuntimeException(e);
}
});
}
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java b/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java
index bd8d5e8e5e1..40c491e1e1e 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java
@@ -24,13 +24,16 @@ import com.google.common.base.Throwables;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableSet;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import java.io.Closeable;
+import java.io.IOException;
import java.util.ArrayDeque;
+import java.util.Iterator;
import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -123,13 +126,23 @@ public class ResourcePool implements Closeable
public void close()
{
closed.set(true);
- final Map> mapView = pool.asMap();
- for (K k : ImmutableSet.copyOf(mapView.keySet())) {
- mapView.remove(k).close();
+ final ConcurrentMap> mapView = pool.asMap();
+ Closer closer = Closer.create();
+ for (Iterator>> iterator =
+ mapView.entrySet().iterator(); iterator.hasNext(); ) {
+ Map.Entry> e = iterator.next();
+ iterator.remove();
+ closer.register(e.getValue());
+ }
+ try {
+ closer.close();
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
}
}
- private static class ImmediateCreationResourceHolder
+ private static class ImmediateCreationResourceHolder implements Closeable
{
private final int maxSize;
private final K key;
@@ -265,7 +278,8 @@ public class ResourcePool implements Closeable
return resourceHolderList.stream().anyMatch(a -> a.getResource().equals(object));
}
- void close()
+ @Override
+ public void close()
{
synchronized (this) {
closed = true;
diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java
index 0c750c3f517..e3974eecb63 100644
--- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java
+++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java
@@ -58,7 +58,7 @@ public class MaterializedViewQueryQueryToolChestTest
.put(
TimeseriesQuery.class,
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
)
.build()
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
index 29f25fa83fa..5a28fbf98d7 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java
@@ -109,7 +109,7 @@ public class MapVirtualColumnGroupByTest
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
index ee15c7ec368..f248bec0d3b 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java
@@ -70,7 +70,7 @@ public class MapVirtualColumnSelectTest
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
selectConfigSupplier
),
new SelectQueryEngine(),
diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
index be1c10a43d7..761230d91cb 100644
--- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java
@@ -69,7 +69,7 @@ public class MapVirtualColumnTopNTest
new StupidPool<>("map-virtual-column-test", () -> ByteBuffer.allocate(1024)),
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
index 5bac2ac8a0a..4a5585da717 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
@@ -81,7 +81,7 @@ public class ApproximateHistogramTopNQueryTest
defaultPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
@@ -91,7 +91,7 @@ public class ApproximateHistogramTopNQueryTest
customPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java
index 42fc6487f3e..04bc90469b1 100644
--- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java
+++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java
@@ -81,7 +81,7 @@ public class FixedBucketsHistogramTopNQueryTest
defaultPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
@@ -91,7 +91,7 @@ public class FixedBucketsHistogramTopNQueryTest
customPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java
index e65c7a38218..01b3dd2aff4 100644
--- a/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java
+++ b/extensions-core/kafka-extraction-namespace/src/main/java/org/apache/druid/query/lookup/KafkaLookupExtractorFactory.java
@@ -54,6 +54,7 @@ import java.util.Map;
import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
@@ -184,7 +185,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
final String topic = getKafkaTopic();
LOG.debug("About to listen to topic [%s] with group.id [%s]", topic, factoryId);
cacheHandler = cacheManager.createCache();
- final Map map = cacheHandler.getCache();
+ final ConcurrentMap map = cacheHandler.getCache();
mapRef.set(map);
// Enable publish-subscribe
kafkaProperties.setProperty("auto.offset.reset", "smallest");
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
index dedb5221746..d081a0e0d79 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java
@@ -104,6 +104,7 @@ import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
@@ -123,8 +124,8 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner endOffsets = new ConcurrentHashMap<>();
- private final Map nextOffsets = new ConcurrentHashMap<>();
+ private final ConcurrentMap endOffsets = new ConcurrentHashMap<>();
+ private final ConcurrentMap nextOffsets = new ConcurrentHashMap<>();
// The pause lock and associated conditions are to support coordination between the Jetty threads and the main
// ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
index 7bf83ddd3a0..04d3802a66c 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -50,7 +50,6 @@ import org.apache.druid.discovery.DruidNodeAnnouncer;
import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskLock;
@@ -2393,8 +2392,7 @@ public class KafkaIndexTaskTest
);
return true;
}
- },
- new Counters()
+ }
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java
index 622c1f37bbc..a05eade5f4c 100644
--- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java
@@ -68,6 +68,7 @@ import java.util.Set;
import java.util.UUID;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.RejectedExecutionException;
@@ -341,7 +342,8 @@ public class KinesisRecordSupplier implements RecordSupplier
private ScheduledExecutorService scheduledExec;
- private final Map, PartitionResource> partitionResources = new ConcurrentHashMap<>();
+ private final ConcurrentMap, PartitionResource> partitionResources =
+ new ConcurrentHashMap<>();
private BlockingQueue> records;
private volatile boolean checkPartitionsStarted = false;
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
index a62d45ae451..a8130b9c0e5 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
@@ -55,7 +55,6 @@ import org.apache.druid.discovery.DruidNodeAnnouncer;
import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskLock;
@@ -2705,8 +2704,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
return true;
}
- },
- new Counters()
+ }
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,
diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
index 658cd2ce105..40a05992114 100644
--- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
+++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/CacheScheduler.java
@@ -34,6 +34,7 @@ import org.apache.druid.query.lookup.namespace.ExtractionNamespace;
import sun.misc.Cleaner;
import javax.annotation.Nullable;
+import java.util.IdentityHashMap;
import java.util.Map;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CountDownLatch;
@@ -424,7 +425,9 @@ public final class CacheScheduler
NamespaceExtractionCacheManager cacheManager
)
{
- this.namespaceGeneratorMap = namespaceGeneratorMap;
+ // Accesses to IdentityHashMap should be faster than to HashMap or ImmutableMap.
+ // Class doesn't override Object.equals().
+ this.namespaceGeneratorMap = new IdentityHashMap<>(namespaceGeneratorMap);
this.cacheManager = cacheManager;
cacheManager.scheduledExecutorService().scheduleAtFixedRate(
new Runnable()
@@ -506,6 +509,7 @@ public final class CacheScheduler
public Entry schedule(final T namespace)
{
+ @SuppressWarnings("unchecked")
final CacheGenerator generator = (CacheGenerator) namespaceGeneratorMap.get(namespace.getClass());
if (generator == null) {
throw new ISE("Cannot find generator for namespace [%s]", namespace);
diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java
index 3612ce54565..9c6871efc9c 100644
--- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java
+++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/cache/OnHeapNamespaceExtractionCacheManager.java
@@ -114,7 +114,7 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa
long size = 0;
expungeCollectedCaches();
for (WeakReference> cacheRef : caches) {
- final Map cache = cacheRef.get();
+ final ConcurrentMap cache = cacheRef.get();
if (cache == null) {
continue;
}
diff --git a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/UriCacheGeneratorTest.java b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/UriCacheGeneratorTest.java
index 87cae6af656..8f8b507b315 100644
--- a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/UriCacheGeneratorTest.java
+++ b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/server/lookup/namespace/UriCacheGeneratorTest.java
@@ -29,8 +29,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
-import org.apache.druid.query.lookup.namespace.CacheGenerator;
-import org.apache.druid.query.lookup.namespace.ExtractionNamespace;
import org.apache.druid.query.lookup.namespace.UriExtractionNamespace;
import org.apache.druid.query.lookup.namespace.UriExtractionNamespaceTest;
import org.apache.druid.segment.loading.LocalFileTimestampVersionFinder;
@@ -64,7 +62,6 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -246,20 +243,14 @@ public class UriCacheGeneratorTest
Function cacheManagerCreator
)
{
- final Map, CacheGenerator>> namespaceFunctionFactoryMap = new HashMap<>();
this.suffix = suffix;
this.outStreamSupplier = outStreamSupplier;
this.lifecycle = new Lifecycle();
this.scheduler = new CacheScheduler(
new NoopServiceEmitter(),
- namespaceFunctionFactoryMap,
+ ImmutableMap.of(UriExtractionNamespace.class, new UriCacheGenerator(FINDERS)),
cacheManagerCreator.apply(lifecycle)
);
- namespaceFunctionFactoryMap.put(
- UriExtractionNamespace.class,
-
- new UriCacheGenerator(FINDERS)
- );
}
@Rule
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
index e634b67984b..7b5406bc4b3 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java
@@ -137,7 +137,7 @@ public class VarianceTopNQueryTest
{
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
final QueryRunner> mergeRunner = chest.mergeResults(runner);
final Sequence> retval = mergeRunner.run(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/Counters.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/Counters.java
index 3cf4ac8ba78..e46389083fb 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/Counters.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/Counters.java
@@ -19,54 +19,33 @@
package org.apache.druid.indexing.common;
-import com.google.common.util.concurrent.AtomicDouble;
-
-import javax.annotation.Nullable;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.BinaryOperator;
+import java.util.concurrent.atomic.AtomicLong;
-public class Counters
+public final class Counters
{
- private final ConcurrentMap intCounters = new ConcurrentHashMap<>();
- private final ConcurrentMap doubleCounters = new ConcurrentHashMap<>();
- private final ConcurrentMap objectCounters = new ConcurrentHashMap<>();
-
- public int increment(String key, int val)
+ public static int incrementAndGetInt(ConcurrentHashMap counters, K key)
{
- return intCounters.computeIfAbsent(key, k -> new AtomicInteger()).addAndGet(val);
+ // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed.
+ // See https://github.com/apache/incubator-druid/pull/6898#discussion_r251384586.
+ AtomicInteger counter = counters.get(key);
+ if (counter == null) {
+ counter = counters.computeIfAbsent(key, k -> new AtomicInteger());
+ }
+ return counter.incrementAndGet();
}
- public double increment(String key, double val)
+ public static long incrementAndGetLong(ConcurrentHashMap counters, K key)
{
- return doubleCounters.computeIfAbsent(key, k -> new AtomicDouble()).addAndGet(val);
+ // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed.
+ // See https://github.com/apache/incubator-druid/pull/6898#discussion_r251384586.
+ AtomicLong counter = counters.get(key);
+ if (counter == null) {
+ counter = counters.computeIfAbsent(key, k -> new AtomicLong());
+ }
+ return counter.incrementAndGet();
}
- public Object increment(String key, Object obj, BinaryOperator mergeFunction)
- {
- return objectCounters.computeIfAbsent(key, k -> new AtomicReference()).accumulateAndGet(obj, mergeFunction);
- }
-
- @Nullable
- public Integer getIntCounter(String key)
- {
- final AtomicInteger atomicInteger = intCounters.get(key);
- return atomicInteger == null ? null : atomicInteger.get();
- }
-
- @Nullable
- public Double getDoubleCounter(String key)
- {
- final AtomicDouble atomicDouble = doubleCounters.get(key);
- return atomicDouble == null ? null : atomicDouble.get();
- }
-
- @Nullable
- public Object getObjectCounter(String key)
- {
- final AtomicReference atomicReference = objectCounters.get(key);
- return atomicReference == null ? null : atomicReference.get();
- }
+ private Counters() {}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java
index bb04d8dffcd..e81a8140369 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java
@@ -20,7 +20,6 @@
package org.apache.druid.indexing.common.actions;
import com.google.inject.Inject;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.indexing.overlord.TaskLockbox;
import org.apache.druid.indexing.overlord.TaskStorage;
@@ -34,7 +33,6 @@ public class TaskActionToolbox
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
private final ServiceEmitter emitter;
private final SupervisorManager supervisorManager;
- private final Counters counters;
@Inject
public TaskActionToolbox(
@@ -42,8 +40,7 @@ public class TaskActionToolbox
TaskStorage taskStorage,
IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
ServiceEmitter emitter,
- SupervisorManager supervisorManager,
- Counters counters
+ SupervisorManager supervisorManager
)
{
this.taskLockbox = taskLockbox;
@@ -51,7 +48,6 @@ public class TaskActionToolbox
this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
this.emitter = emitter;
this.supervisorManager = supervisorManager;
- this.counters = counters;
}
public TaskLockbox getTaskLockbox()
@@ -78,9 +74,4 @@ public class TaskActionToolbox
{
return supervisorManager;
}
-
- public Counters getCounters()
- {
- return counters;
- }
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
index 2591987d1f3..b8a23097688 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
@@ -76,6 +76,8 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.SortedSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
/**
@@ -98,7 +100,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
private final AuthorizerMapper authorizerMapper;
private final RowIngestionMetersFactory rowIngestionMetersFactory;
- private final Counters counters = new Counters();
+ private final ConcurrentHashMap partitionNumCountersPerInterval = new ConcurrentHashMap<>();
private volatile ParallelIndexTaskRunner runner;
@@ -377,7 +379,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec);
}
- final int partitionNum = counters.increment(interval.toString(), 1);
+ final int partitionNum = Counters.incrementAndGetInt(partitionNumCountersPerInterval, interval);
return new SegmentIdWithShardSpec(
dataSource,
interval,
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java
index d04315668e6..e1919ec5609 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java
@@ -53,7 +53,6 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
@@ -82,8 +81,8 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn
private final BlockingQueue> taskCompleteEvents =
new LinkedBlockingDeque<>();
- // subTaskId -> report
- private final ConcurrentMap segmentsMap = new ConcurrentHashMap<>();
+ /** subTaskId -> report */
+ private final ConcurrentHashMap segmentsMap = new ConcurrentHashMap<>();
private volatile boolean stopped;
private volatile TaskMonitor taskMonitor;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java
index c3a0bc94ded..d3199834c63 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java
@@ -85,6 +85,7 @@ import java.util.Set;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
@@ -107,8 +108,8 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
private final PortFinder portFinder;
private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>();
- // Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting.
- private final Map tasks = new ConcurrentHashMap<>();
+ /** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */
+ private final ConcurrentMap tasks = new ConcurrentHashMap<>();
private volatile boolean stopping = false;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
index ef30f15f225..afcbdb1f5ad 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
@@ -33,6 +33,7 @@ import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Inject;
import org.apache.druid.indexer.TaskStatus;
+import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
@@ -103,8 +104,8 @@ public class TaskQueue
private static final EmittingLogger log = new EmittingLogger(TaskQueue.class);
- private final Map totalSuccessfulTaskCount = new ConcurrentHashMap<>();
- private final Map totalFailedTaskCount = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap totalSuccessfulTaskCount = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap totalFailedTaskCount = new ConcurrentHashMap<>();
private Map prevTotalSuccessfulTaskCount = new HashMap<>();
private Map prevTotalFailedTaskCount = new HashMap<>();
@@ -520,11 +521,9 @@ public class TaskQueue
);
if (status.isSuccess()) {
- totalSuccessfulTaskCount.computeIfAbsent(task.getDataSource(), k -> new AtomicLong())
- .incrementAndGet();
+ Counters.incrementAndGetLong(totalSuccessfulTaskCount, task.getDataSource());
} else {
- totalFailedTaskCount.computeIfAbsent(task.getDataSource(), k -> new AtomicLong())
- .incrementAndGet();
+ Counters.incrementAndGetLong(totalFailedTaskCount, task.getDataSource());
}
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
index 8e502f0387e..857264965f5 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
@@ -111,6 +111,7 @@ import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
@@ -145,8 +146,8 @@ public abstract class SeekableStreamIndexTaskRunner endOffsets;
- private final Map currOffsets = new ConcurrentHashMap<>();
- private final Map lastPersistedOffsets = new ConcurrentHashMap<>();
+ private final ConcurrentMap currOffsets = new ConcurrentHashMap<>();
+ private final ConcurrentMap lastPersistedOffsets = new ConcurrentHashMap<>();
// The pause lock and associated conditions are to support coordination between the Jetty threads and the main
// ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
index 0ec7b41089f..613c69b3a2e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java
@@ -87,8 +87,8 @@ public abstract class WorkerTaskManager
// ZK_CLEANUP_TODO : these are marked protected to be used in subclass WorkerTaskMonitor that updates ZK.
// should be marked private alongwith WorkerTaskMonitor removal.
- protected final Map runningTasks = new ConcurrentHashMap<>();
- protected final Map completedTasks = new ConcurrentHashMap<>();
+ protected final ConcurrentMap runningTasks = new ConcurrentHashMap<>();
+ protected final ConcurrentMap completedTasks = new ConcurrentHashMap<>();
private final ChangeRequestHistory changeHistory = new ChangeRequestHistory<>();
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java
index 98a3e65f713..ceb65d1f31e 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java
@@ -21,7 +21,6 @@ package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Suppliers;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskStorageConfig;
import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
@@ -98,8 +97,7 @@ public class TaskActionTestKit extends ExternalResource
taskStorage,
metadataStorageCoordinator,
new NoopServiceEmitter(),
- EasyMock.createMock(SupervisorManager.class),
- new Counters()
+ EasyMock.createMock(SupervisorManager.class)
);
testDerbyConnector.createDataSourceTable();
testDerbyConnector.createPendingSegmentsTable();
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
index 0add56bd6b5..b57397ab8cd 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
@@ -50,7 +50,6 @@ import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.indexer.IngestionState;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskReport;
@@ -1524,8 +1523,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
taskStorage,
mdc,
emitter,
- EasyMock.createMock(SupervisorManager.class),
- new Counters()
+ EasyMock.createMock(SupervisorManager.class)
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
index a9d68b5e1be..8406adb7e94 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
@@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.LocalTaskActionClient;
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
@@ -122,8 +121,7 @@ public abstract class IngestionTestBase
taskStorage,
storageCoordinator,
new NoopServiceEmitter(),
- null,
- new Counters()
+ null
);
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
index 9a6a8e98e6e..2c3d3c3d873 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
@@ -45,7 +45,6 @@ import org.apache.druid.discovery.DruidNodeAnnouncer;
import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.TaskToolboxFactory;
@@ -984,8 +983,7 @@ public class RealtimeIndexTaskTest
taskStorage,
mdc,
emitter,
- EasyMock.createMock(SupervisorManager.class),
- new Counters()
+ EasyMock.createMock(SupervisorManager.class)
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
index cc3ca979dcb..668c8ec1985 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java
@@ -85,19 +85,19 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd
{
private static final int NUM_SUB_TASKS = 10;
- // specId -> spec
+ /** specId -> spec */
private final ConcurrentMap subTaskSpecs = new ConcurrentHashMap<>();
- // specId -> taskStatusPlus
+ /** specId -> taskStatusPlus */
private final ConcurrentMap runningSpecs = new ConcurrentHashMap<>();
- // specId -> taskStatusPlus list
- private final ConcurrentMap> taskHistories = new ConcurrentHashMap<>();
+ /** specId -> taskStatusPlus list */
+ private final ConcurrentHashMap> taskHistories = new ConcurrentHashMap<>();
- // taskId -> subTaskSpec
+ /** taskId -> subTaskSpec */
private final ConcurrentMap taskIdToSpec = new ConcurrentHashMap<>();
- // taskId -> task
+ /** taskId -> task */
private final CopyOnWriteArrayList runningTasks = new CopyOnWriteArrayList<>();
private ExecutorService service;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
index 9db06583a80..3329419cf33 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
@@ -39,7 +39,6 @@ import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.guice.GuiceAnnotationIntrospector;
import org.apache.druid.guice.GuiceInjectableValues;
import org.apache.druid.guice.GuiceInjectors;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskToolboxFactory;
import org.apache.druid.indexing.common.TestUtils;
@@ -219,8 +218,7 @@ public class IngestSegmentFirehoseFactoryTest
TASK_STORAGE,
mdc,
newMockEmitter(),
- EasyMock.createMock(SupervisorManager.class),
- new Counters()
+ EasyMock.createMock(SupervisorManager.class)
),
new TaskAuditLogConfig(false)
);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
index 31cbf2c580f..265150c6056 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
@@ -45,7 +45,6 @@ import org.apache.druid.discovery.DruidNodeAnnouncer;
import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox;
@@ -538,8 +537,7 @@ public class TaskLifecycleTest
taskStorage,
mdc,
emitter,
- EasyMock.createMock(SupervisorManager.class),
- new Counters()
+ EasyMock.createMock(SupervisorManager.class)
),
new TaskAuditLogConfig(true)
);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskRunner.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskRunner.java
index c9163611c61..41a4366feac 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskRunner.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskRunner.java
@@ -54,7 +54,6 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executor;
@@ -68,7 +67,7 @@ public class TestTaskRunner implements TaskRunner, QuerySegmentWalker
{
private static final EmittingLogger log = new EmittingLogger(TestTaskRunner.class);
- private final ConcurrentMap exec = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap exec = new ConcurrentHashMap<>();
private final Set runningItems = new ConcurrentSkipListSet<>();
private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>();
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java
index 814a20f5e8c..0adcf266014 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java
@@ -67,6 +67,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicInteger;
@@ -271,7 +272,7 @@ public class HttpRemoteTaskRunnerTest
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
- Map workerHolders = new ConcurrentHashMap<>();
+ ConcurrentMap workerHolders = new ConcurrentHashMap<>();
Task task1 = NoopTask.create("task-id-1", 0);
Task task2 = NoopTask.create("task-id-2", 0);
@@ -424,7 +425,7 @@ public class HttpRemoteTaskRunnerTest
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
- Map workerHolders = new ConcurrentHashMap<>();
+ ConcurrentMap workerHolders = new ConcurrentHashMap<>();
HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
TestHelper.makeJsonMapper(),
@@ -600,7 +601,7 @@ public class HttpRemoteTaskRunnerTest
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
- Map workerHolders = new ConcurrentHashMap<>();
+ ConcurrentMap workerHolders = new ConcurrentHashMap<>();
HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
TestHelper.makeJsonMapper(),
@@ -777,7 +778,7 @@ public class HttpRemoteTaskRunnerTest
Task task1 = NoopTask.create("task-id-1", 0);
Task task2 = NoopTask.create("task-id-2", 0);
- Map workerHolders = new ConcurrentHashMap<>();
+ ConcurrentMap workerHolders = new ConcurrentHashMap<>();
HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner(
TestHelper.makeJsonMapper(),
diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java
index ecd77b70021..7f004ffa634 100644
--- a/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java
+++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java
@@ -21,6 +21,7 @@ package org.apache.druid.query;
import com.google.inject.Inject;
+import java.util.IdentityHashMap;
import java.util.Map;
/**
@@ -30,11 +31,11 @@ public class DefaultQueryRunnerFactoryConglomerate implements QueryRunnerFactory
private final Map, QueryRunnerFactory> factories;
@Inject
- public DefaultQueryRunnerFactoryConglomerate(
- Map, QueryRunnerFactory> factories
- )
+ public DefaultQueryRunnerFactoryConglomerate(Map, QueryRunnerFactory> factories)
{
- this.factories = factories;
+ // Accesses to IdentityHashMap should be faster than to HashMap or ImmutableMap.
+ // Class doesn't override Object.equals().
+ this.factories = new IdentityHashMap<>(factories);
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/query/MapQueryToolChestWarehouse.java b/processing/src/main/java/org/apache/druid/query/MapQueryToolChestWarehouse.java
index db88db14dbe..7199c957c4a 100644
--- a/processing/src/main/java/org/apache/druid/query/MapQueryToolChestWarehouse.java
+++ b/processing/src/main/java/org/apache/druid/query/MapQueryToolChestWarehouse.java
@@ -21,6 +21,7 @@ package org.apache.druid.query;
import com.google.inject.Inject;
+import java.util.IdentityHashMap;
import java.util.Map;
/**
@@ -30,11 +31,11 @@ public class MapQueryToolChestWarehouse implements QueryToolChestWarehouse
private final Map, QueryToolChest> toolchests;
@Inject
- public MapQueryToolChestWarehouse(
- Map, QueryToolChest> toolchests
- )
+ public MapQueryToolChestWarehouse(Map, QueryToolChest> toolchests)
{
- this.toolchests = toolchests;
+ // Accesses to IdentityHashMap should be faster than to HashMap or ImmutableMap.
+ // Class doesn't override Object.equals().
+ this.toolchests = new IdentityHashMap<>(toolchests);
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
index 0b890c06133..81f59e43f2e 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java
@@ -20,6 +20,8 @@
package org.apache.druid.query;
import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.type.TypeFactory;
import com.google.common.base.Function;
import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.query.aggregation.MetricManipulationFn;
@@ -34,6 +36,42 @@ import java.util.List;
@ExtensionPoint
public abstract class QueryToolChest>
{
+ private final JavaType baseResultType;
+ private final JavaType bySegmentResultType;
+
+ protected QueryToolChest()
+ {
+ final TypeFactory typeFactory = TypeFactory.defaultInstance();
+ TypeReference resultTypeReference = getResultTypeReference();
+ // resultTypeReference is null in MaterializedViewQueryQueryToolChest.
+ // See https://github.com/apache/incubator-druid/issues/6977
+ if (resultTypeReference != null) {
+ baseResultType = typeFactory.constructType(resultTypeReference);
+ bySegmentResultType = typeFactory.constructParametrizedType(
+ Result.class,
+ Result.class,
+ typeFactory.constructParametrizedType(
+ BySegmentResultValueClass.class,
+ BySegmentResultValueClass.class,
+ baseResultType
+ )
+ );
+ } else {
+ baseResultType = null;
+ bySegmentResultType = null;
+ }
+ }
+
+ public final JavaType getBaseResultType()
+ {
+ return baseResultType;
+ }
+
+ public final JavaType getBySegmentResultType()
+ {
+ return bySegmentResultType;
+ }
+
/**
* This method wraps a QueryRunner. The input QueryRunner, by contract, will provide a series of
* ResultType objects in time order (ascending or descending). This method should return a new QueryRunner that
diff --git a/processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java b/processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java
deleted file mode 100644
index a976b07e999..00000000000
--- a/processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.query;
-
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.logger.Logger;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- */
-@Deprecated
-public class ReflectionLoaderThingy
-{
- private static final Logger log = new Logger(ReflectionLoaderThingy.class);
-
- public static ReflectionLoaderThingy create(Class interfaceClass)
- {
- return new ReflectionLoaderThingy(interfaceClass);
- }
-
- Map, AtomicReference> toolChestMap = new ConcurrentHashMap<>();
-
- private final Class interfaceClass;
-
- public ReflectionLoaderThingy(
- Class interfaceClass
- )
- {
- this.interfaceClass = interfaceClass;
- }
-
- public T getForObject(Object keyObject)
- {
- Class> clazz = keyObject.getClass();
-
- AtomicReference retVal = toolChestMap.get(clazz);
-
- if (retVal == null) {
- String interfaceName = interfaceClass.getSimpleName();
-
- AtomicReference retVal1;
- try {
- final Class> queryToolChestClass = Class.forName(StringUtils.format("%s%s", clazz.getName(), interfaceName));
- retVal1 = new AtomicReference(interfaceClass.cast(queryToolChestClass.newInstance()));
- }
- catch (Exception e) {
- log.warn(e, "Unable to load interface[%s] for input class[%s]", interfaceClass, clazz);
- retVal1 = new AtomicReference(null);
- }
- retVal = retVal1;
-
- toolChestMap.put(clazz, retVal);
- }
-
- return retVal.get();
- }
-}
diff --git a/processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java b/processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java
index 4c7aab72d43..e2b047c1b2d 100644
--- a/processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java
+++ b/processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java
@@ -19,17 +19,35 @@
package org.apache.druid.query;
+import org.apache.druid.java.util.common.logger.Logger;
+
/**
*/
@Deprecated
public class ReflectionQueryToolChestWarehouse implements QueryToolChestWarehouse
{
- ReflectionLoaderThingy loader = ReflectionLoaderThingy.create(QueryToolChest.class);
+ private static final Logger log = new Logger(ReflectionQueryToolChestWarehouse.class);
+
+ private final ClassValue> toolChests = new ClassValue>()
+ {
+ @Override
+ protected QueryToolChest, ?> computeValue(Class> type)
+ {
+ try {
+ final Class> queryToolChestClass = Class.forName(type.getName() + "QueryToolChest");
+ return (QueryToolChest, ?>) queryToolChestClass.newInstance();
+ }
+ catch (Exception e) {
+ log.warn(e, "Unable to load interface[QueryToolChest] for input class[%s]", type);
+ throw new RuntimeException(e);
+ }
+ }
+ };
@Override
@SuppressWarnings("unchecked")
public > QueryToolChest getToolChest(QueryType query)
{
- return (QueryToolChest) loader.getForObject(query);
+ return (QueryToolChest) toolChests.get(query.getClass());
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java
index 13d265186b4..e1befab71ad 100644
--- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java
@@ -48,9 +48,8 @@ import java.util.Map;
public class DataSourceQueryQueryToolChest
extends QueryToolChest, DataSourceMetadataQuery>
{
- private static final TypeReference> TYPE_REFERENCE = new TypeReference>()
- {
- };
+ private static final TypeReference> TYPE_REFERENCE =
+ new TypeReference>() {};
private final GenericQueryMetricsFactory queryMetricsFactory;
diff --git a/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java b/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java
index 43f1a1a5afd..7f0089d73a0 100644
--- a/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java
+++ b/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java
@@ -38,7 +38,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -146,7 +145,7 @@ public final class SpecializationService
static class PerPrototypeClassState
{
private final Class prototypeClass;
- private final ConcurrentMap> specializationStates =
+ private final ConcurrentHashMap> specializationStates =
new ConcurrentHashMap<>();
private final String prototypeClassBytecodeName;
private final String specializedClassNamePrefix;
@@ -164,6 +163,8 @@ public final class SpecializationService
SpecializationState getSpecializationState(String runtimeShape, ImmutableMap, Class>> classRemapping)
{
SpecializationId specializationId = new SpecializationId(runtimeShape, classRemapping);
+ // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed.
+ // See https://github.com/apache/incubator-druid/pull/6898#discussion_r251384586.
SpecializationState alreadyExistingState = specializationStates.get(specializationId);
if (alreadyExistingState != null) {
return alreadyExistingState;
@@ -278,7 +279,7 @@ public final class SpecializationService
private final PerPrototypeClassState perPrototypeClassState;
private final SpecializationId specializationId;
/** A map with the number of iterations per each minute during the last hour */
- private final ConcurrentMap perMinuteIterations = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap perMinuteIterations = new ConcurrentHashMap<>();
private final AtomicBoolean specializationScheduled = new AtomicBoolean(false);
WindowedLoopIterationCounter(
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
index f20484ef3b3..865cbf45e7b 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
@@ -1304,7 +1304,7 @@ public abstract class IncrementalIndex extends AbstractIndex imp
IncrementalIndexRow end = new IncrementalIndexRow(timeEnd, new Object[]{}, dimensionDescsList);
ConcurrentNavigableMap subMap =
((ConcurrentNavigableMap) facts).subMap(start, end);
- final Map rangeMap = descending ? subMap.descendingMap() : subMap;
+ ConcurrentMap rangeMap = descending ? subMap.descendingMap() : subMap;
return rangeMap.keySet();
}
@@ -1397,7 +1397,7 @@ public abstract class IncrementalIndex extends AbstractIndex imp
{
ConcurrentNavigableMap> subMap =
((ConcurrentNavigableMap>) facts).subMap(timeStart, timeEnd);
- final Map> rangeMap = descending ? subMap.descendingMap() : subMap;
+ final ConcurrentMap> rangeMap = descending ? subMap.descendingMap() : subMap;
return timeOrderedConcat(rangeMap.values(), descending);
}
diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
index 877043bcf72..c4c4a0b99af 100644
--- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
+++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
@@ -271,7 +271,7 @@ public class MultiValuedDimensionTest
pool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
index 1b7b8900a6f..094e8fc2c32 100644
--- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java
@@ -475,7 +475,7 @@ public class QueryRunnerTestHelper
.applyPostMergeDecoration();
}
- public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
+ public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{
return new IntervalChunkingQueryRunnerDecorator(null, null, null)
{
@@ -518,7 +518,7 @@ public class QueryRunnerTestHelper
public static TimeseriesQueryRunnerFactory newTimeseriesQueryRunnerFactory()
{
return new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
index 6e921527aa7..378c024bbf0 100644
--- a/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java
@@ -40,6 +40,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
public class RetryQueryRunnerTest
{
@@ -89,7 +90,7 @@ public class RetryQueryRunnerTest
@Test
public void testRunWithMissingSegments()
{
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
RetryQueryRunner> runner = new RetryQueryRunner<>(
new QueryRunner>()
@@ -133,7 +134,7 @@ public class RetryQueryRunnerTest
@Test
public void testRetry()
{
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put("count", 0);
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
RetryQueryRunner> runner = new RetryQueryRunner<>(
@@ -181,7 +182,7 @@ public class RetryQueryRunnerTest
@Test
public void testRetryMultiple()
{
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put("count", 0);
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
RetryQueryRunner> runner = new RetryQueryRunner<>(
@@ -229,7 +230,7 @@ public class RetryQueryRunnerTest
@Test(expected = SegmentMissingException.class)
public void testException()
{
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
RetryQueryRunner> runner = new RetryQueryRunner<>(
new QueryRunner>()
@@ -261,7 +262,7 @@ public class RetryQueryRunnerTest
@Test
public void testNoDuplicateRetry()
{
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put("count", 0);
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
RetryQueryRunner> runner = new RetryQueryRunner<>(
diff --git a/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java b/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java
index 24b104da87c..ecba0055ef2 100644
--- a/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java
+++ b/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java
@@ -57,7 +57,7 @@ public class TestQueryRunners
pool,
new TopNQueryQueryToolChest(
topNConfig,
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
@@ -71,7 +71,7 @@ public class TestQueryRunners
{
QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
@@ -89,7 +89,7 @@ public class TestQueryRunners
new SearchStrategySelector(Suppliers.ofInstance(config)),
new SearchQueryQueryToolChest(
config,
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
index f1503cb1c31..cff1b4f689a 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
@@ -195,14 +195,14 @@ public class AggregationTestHelper implements Closeable
SelectQueryQueryToolChest toolchest = new SelectQueryQueryToolChest(
TestHelper.makeJsonMapper(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
configSupplier
);
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
TestHelper.makeJsonMapper(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
configSupplier
),
new SelectQueryEngine(
@@ -242,7 +242,7 @@ public class AggregationTestHelper implements Closeable
ObjectMapper mapper = TestHelper.makeJsonMapper();
TimeseriesQueryQueryToolChest toolchest = new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
@@ -284,7 +284,7 @@ public class AggregationTestHelper implements Closeable
TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
final CloseableStupidPool pool = new CloseableStupidPool<>(
diff --git a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
index b2d6b87f242..2a23b547f66 100644
--- a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java
@@ -52,6 +52,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
public class DataSourceMetadataQueryTest
{
@@ -137,7 +138,7 @@ public class DataSourceMetadataQueryTest
DataSourceMetadataQuery dataSourceMetadataQuery = Druids.newDataSourceMetadataQueryBuilder()
.dataSource("testing")
.build();
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
Iterable> results =
runner.run(QueryPlus.wrap(dataSourceMetadataQuery), context).toList();
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java
index 19f3a586267..36e55051e5f 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java
@@ -157,12 +157,9 @@ public class GroupByQueryMergeBufferTest
);
final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(
strategySelector,
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
- );
- return new GroupByQueryRunnerFactory(
- strategySelector,
- toolChest
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
+ return new GroupByQueryRunnerFactory(strategySelector, toolChest);
}
private static final CloseableStupidPool bufferPool = new CloseableStupidPool<>(
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java
index 9222ba5c7ac..e804de9cf49 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java
@@ -119,12 +119,9 @@ public class GroupByQueryRunnerFailureTest
);
final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(
strategySelector,
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
- );
- return new GroupByQueryRunnerFactory(
- strategySelector,
- toolChest
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
+ return new GroupByQueryRunnerFactory(strategySelector, toolChest);
}
private static final CloseableStupidPool bufferPool = new CloseableStupidPool<>(
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
index 3a2d5324e53..96e369966db 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -396,13 +396,7 @@ public class GroupByQueryRunnerTest
final Closer closer = Closer.create();
closer.register(bufferPool);
closer.register(mergeBufferPool);
- return Pair.of(
- new GroupByQueryRunnerFactory(
- strategySelector,
- toolChest
- ),
- closer
- );
+ return Pair.of(new GroupByQueryRunnerFactory(strategySelector, toolChest), closer);
}
@Parameterized.Parameters(name = "{0}")
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
index 0eb5e9afd92..e730abf1d41 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java
@@ -354,7 +354,7 @@ public class NestedQueryPushDownTest
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
)
);
@@ -362,7 +362,7 @@ public class NestedQueryPushDownTest
strategySelector2,
new GroupByQueryQueryToolChest(
strategySelector2,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
)
);
}
@@ -919,7 +919,7 @@ public class NestedQueryPushDownTest
}
};
- public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
+ public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{
return new IntervalChunkingQueryRunnerDecorator(null, null, null)
{
diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
index 4d3d1e05905..af2bdf8b28f 100644
--- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java
@@ -79,7 +79,7 @@ public class SearchQueryRunnerTest
private static final SearchQueryConfig config = new SearchQueryConfig();
private static final SearchQueryQueryToolChest toolChest = new SearchQueryQueryToolChest(
config,
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
private static final SearchStrategySelector selector = new SearchStrategySelector(Suppliers.ofInstance(config));
diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
index 98d3568b3fb..b80253890ea 100644
--- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
+++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java
@@ -51,12 +51,12 @@ import java.util.Set;
import java.util.TreeMap;
import static org.apache.druid.query.QueryRunnerTestHelper.NOOP_QUERYWATCHER;
-import static org.apache.druid.query.QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator;
import static org.apache.druid.query.QueryRunnerTestHelper.allGran;
import static org.apache.druid.query.QueryRunnerTestHelper.dataSource;
import static org.apache.druid.query.QueryRunnerTestHelper.fullOnIntervalSpec;
import static org.apache.druid.query.QueryRunnerTestHelper.makeQueryRunner;
import static org.apache.druid.query.QueryRunnerTestHelper.marketDimension;
+import static org.apache.druid.query.QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator;
import static org.apache.druid.query.QueryRunnerTestHelper.placementDimension;
import static org.apache.druid.query.QueryRunnerTestHelper.placementishDimension;
import static org.apache.druid.query.QueryRunnerTestHelper.qualityDimension;
@@ -130,7 +130,7 @@ public class SearchQueryRunnerWithCaseTest
new SearchStrategySelector(Suppliers.ofInstance(config)),
new SearchQueryQueryToolChest(
config,
- NoopIntervalChunkingQueryRunnerDecorator()
+ noopIntervalChunkingQueryRunnerDecorator()
),
NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
index 2edc2b6c8e5..1b7eacc449f 100644
--- a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java
@@ -75,7 +75,7 @@ public class MultiSegmentSelectQueryTest
private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
configSupplier
);
diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java
index 5d9da392a89..46f12daad0f 100644
--- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryQueryToolChestTest.java
@@ -38,7 +38,7 @@ public class SelectQueryQueryToolChestTest
private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
configSupplier
);
diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
index a5ce234c68b..3034315c598 100644
--- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java
@@ -129,7 +129,7 @@ public class SelectQueryRunnerTest
private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
configSupplier
);
diff --git a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
index 094b732dfce..769d5cf0e3e 100644
--- a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java
@@ -54,8 +54,8 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
/**
*/
@@ -219,7 +219,7 @@ public class TimeBoundaryQueryRunnerTest
.dataSource("testing")
.bound(TimeBoundaryQuery.MAX_TIME)
.build();
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
Iterable> results = runner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
TimeBoundaryResultValue val = results.iterator().next().getValue();
@@ -238,7 +238,7 @@ public class TimeBoundaryQueryRunnerTest
.dataSource("testing")
.bound(TimeBoundaryQuery.MIN_TIME)
.build();
- Map context = new ConcurrentHashMap<>();
+ ConcurrentMap context = new ConcurrentHashMap<>();
context.put(Result.MISSING_SEGMENTS_KEY, new ArrayList<>());
Iterable> results = runner.run(QueryPlus.wrap(timeBoundaryQuery), context).toList();
TimeBoundaryResultValue val = results.iterator().next().getValue();
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java
index 4efcddb990e..48a322b08c8 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java
@@ -62,7 +62,7 @@ public class TimeSeriesUnionQueryRunnerTest
return QueryRunnerTestHelper.cartesian(
QueryRunnerTestHelper.makeUnionQueryRunners(
new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
@@ -145,7 +145,7 @@ public class TimeSeriesUnionQueryRunnerTest
)
.descending(descending)
.build();
- QueryToolChest toolChest = new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator());
+ QueryToolChest toolChest = new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator());
final List> ds1 = Lists.newArrayList(
new Result<>(
DateTimes.of("2011-04-02"),
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
index 824552203d9..22134a10e86 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java
@@ -112,7 +112,7 @@ public class TimeseriesQueryRunnerBonusTest
private List> runTimeseriesCount(IncrementalIndex index)
{
final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
index d59ddf12e2f..c24fc72c0da 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
@@ -93,7 +93,7 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.makeQueryRunners(
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
@@ -508,7 +508,7 @@ public class TimeseriesQueryRunnerTest
// Must create a toolChest so we can run mergeResults (which applies grand totals).
QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
// Must wrapped in a results finalizer to stop the runner's builtin finalizer from being called.
@@ -559,7 +559,7 @@ public class TimeseriesQueryRunnerTest
// Must create a toolChest so we can run mergeResults (which creates the zeroed-out row).
QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
// Must wrapped in a results finalizer to stop the runner's builtin finalizer from being called.
@@ -2425,7 +2425,7 @@ public class TimeseriesQueryRunnerTest
TestHelper.assertExpectedResults(expectedResults, results);
QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
QueryRunner> optimizedRunner = toolChest.postMergeQueryDecoration(
toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)));
@@ -2455,7 +2455,7 @@ public class TimeseriesQueryRunnerTest
// Must create a toolChest so we can run mergeResults.
QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
// Must wrapped in a results finalizer to stop the runner's builtin finalizer from being called.
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
index 8212dc7f7a7..e263c1eef2b 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java
@@ -136,7 +136,7 @@ public class TopNQueryQueryToolChestTest
TopNQueryConfig config = new TopNQueryConfig();
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
config,
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
try (CloseableStupidPool pool = TestQueryRunners.createDefaultNonBlockingPool()) {
QueryRunnerFactory factory = new TopNQueryRunnerFactory(
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
index a43957e755c..839ee6d5aba 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java
@@ -99,7 +99,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
}
}
),
- new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
testCaseMap.put(
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
index da23ca673a7..0244372dc30 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java
@@ -260,7 +260,7 @@ public class TopNQueryRunnerTest
{
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
final QueryRunner> mergeRunner = new FinalizeResultsQueryRunner(
chest.mergeResults(runner),
@@ -4281,7 +4281,7 @@ public class TopNQueryRunnerTest
{
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
final QueryRunner> Runner = new FinalizeResultsQueryRunner(
chest.mergeResults(chest.preMergeQueryDecoration(runner)),
diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
index 0aa1fcde882..076bbda48fe 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java
@@ -73,7 +73,7 @@ public class TopNUnionQueryTest
defaultPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
@@ -83,7 +83,7 @@ public class TopNUnionQueryTest
customPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
index 1f94d71b5b7..e2064e0d89e 100644
--- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java
@@ -574,7 +574,7 @@ public class IndexMergerV9WithSpatialIndexTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
@@ -628,7 +628,7 @@ public class IndexMergerV9WithSpatialIndexTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
@@ -717,7 +717,7 @@ public class IndexMergerV9WithSpatialIndexTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java
index a9c60350f4c..a3c3478ebed 100644
--- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java
@@ -433,7 +433,7 @@ public class IncrementalIndexTest
final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null);
final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
@@ -530,7 +530,7 @@ public class IncrementalIndexTest
final List> queryFutures = Lists.newArrayListWithExpectedSize(concurrentThreads);
final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null);
final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
index ff54321b5d0..69857f5c5ec 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java
@@ -508,7 +508,7 @@ public class SpatialFilterBonusTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
@@ -596,7 +596,7 @@ public class SpatialFilterBonusTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
@@ -690,7 +690,7 @@ public class SpatialFilterBonusTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
index ee745a50f1c..22e3e683829 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java
@@ -564,7 +564,7 @@ public class SpatialFilterTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
@@ -617,7 +617,7 @@ public class SpatialFilterTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
@@ -705,7 +705,7 @@ public class SpatialFilterTest
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
- QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java
index eb4609cb688..096e4d3de43 100644
--- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java
+++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java
@@ -350,7 +350,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
final List> queryFutures = new ArrayList<>();
final Segment incrementalIndexSegment = new IncrementalIndexSegment(incrementalIndex, null);
final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
- new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
+ new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
index 3f86efd18c6..7f73abf97c8 100644
--- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
+++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java
@@ -21,7 +21,6 @@ package org.apache.druid.client;
import com.fasterxml.jackson.databind.JavaType;
import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.type.TypeFactory;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
import com.google.common.base.Preconditions;
@@ -29,7 +28,6 @@ import com.google.common.base.Throwables;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
-import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.guava.BaseSequence;
@@ -45,7 +43,6 @@ import org.apache.druid.java.util.http.client.response.ClientResponse;
import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
-import org.apache.druid.query.BySegmentResultValueClass;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryMetrics;
@@ -54,7 +51,6 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryWatcher;
-import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
@@ -74,6 +70,7 @@ import java.util.Enumeration;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
@@ -92,8 +89,6 @@ public class DirectDruidClient implements QueryRunner
private static final Logger log = new Logger(DirectDruidClient.class);
- private static final Map, Pair> typesMap = new ConcurrentHashMap<>();
-
private final QueryToolChestWarehouse warehouse;
private final QueryWatcher queryWatcher;
private final ObjectMapper objectMapper;
@@ -113,13 +108,10 @@ public class DirectDruidClient implements QueryRunner
responseContext.remove(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED);
}
- public static Map makeResponseContextForQuery()
+ public static ConcurrentMap makeResponseContextForQuery()
{
- final Map responseContext = new ConcurrentHashMap<>();
- responseContext.put(
- DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED,
- new AtomicLong()
- );
+ final ConcurrentMap responseContext = new ConcurrentHashMap<>();
+ responseContext.put(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());
return responseContext;
}
@@ -156,25 +148,7 @@ public class DirectDruidClient implements QueryRunner
final Query query = queryPlus.getQuery();
QueryToolChest> toolChest = warehouse.getToolChest(query);
boolean isBySegment = QueryContexts.isBySegment(query);
-
- Pair types = typesMap.get(query.getClass());
- if (types == null) {
- final TypeFactory typeFactory = objectMapper.getTypeFactory();
- JavaType baseType = typeFactory.constructType(toolChest.getResultTypeReference());
- JavaType bySegmentType = typeFactory.constructParametricType(
- Result.class,
- typeFactory.constructParametricType(BySegmentResultValueClass.class, baseType)
- );
- types = Pair.of(baseType, bySegmentType);
- typesMap.put(query.getClass(), types);
- }
-
- final JavaType typeRef;
- if (isBySegment) {
- typeRef = types.rhs;
- } else {
- typeRef = types.lhs;
- }
+ final JavaType queryResultType = isBySegment ? toolChest.getBySegmentResultType() : toolChest.getBaseResultType();
final ListenableFuture future;
final String url = StringUtils.format("%s://%s/druid/v2/", scheme, host);
@@ -543,7 +517,7 @@ public class DirectDruidClient implements QueryRunner
@Override
public JsonParserIterator make()
{
- return new JsonParserIterator(typeRef, future, url, query, host, objectMapper, null);
+ return new JsonParserIterator(queryResultType, future, url, query, host, objectMapper, null);
}
@Override
diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java
index 7025319d2de..86fe0c32a6f 100644
--- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java
+++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java
@@ -48,8 +48,8 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
-import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@@ -67,7 +67,7 @@ public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvide
private ExecutorService listenerExecutor;
- private final Map nodeTypeWatchers = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap nodeTypeWatchers = new ConcurrentHashMap<>();
private final LifecycleLock lifecycleLock = new LifecycleLock();
@@ -155,8 +155,8 @@ public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvide
private final NodeType nodeType;
private final ObjectMapper jsonMapper;
- // hostAndPort -> DiscoveryDruidNode
- private final Map nodes = new ConcurrentHashMap<>();
+ /** hostAndPort -> DiscoveryDruidNode */
+ private final ConcurrentMap nodes = new ConcurrentHashMap<>();
private final Collection unmodifiableNodes = Collections.unmodifiableCollection(nodes.values());
private final PathChildrenCache cache;
diff --git a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java
index 79e73c158fc..4dcb77759a8 100644
--- a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java
+++ b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java
@@ -32,6 +32,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
/**
* Provider of {@link DruidNodeDiscovery} instances.
@@ -81,7 +82,7 @@ public abstract class DruidNodeDiscoveryProvider
private static final Logger log = new Logger(ServiceDruidNodeDiscovery.class);
private final String service;
- private final Map nodes = new ConcurrentHashMap<>();
+ private final ConcurrentMap nodes = new ConcurrentHashMap<>();
private final Collection unmodifiableNodes = Collections.unmodifiableCollection(nodes.values());
private final List listeners = new ArrayList<>();
diff --git a/server/src/main/java/org/apache/druid/initialization/Initialization.java b/server/src/main/java/org/apache/druid/initialization/Initialization.java
index fcf288ac6a3..bee01d16fff 100644
--- a/server/src/main/java/org/apache/druid/initialization/Initialization.java
+++ b/server/src/main/java/org/apache/druid/initialization/Initialization.java
@@ -79,7 +79,6 @@ import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
@@ -87,16 +86,15 @@ import java.util.Map;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
/**
*/
public class Initialization
{
private static final Logger log = new Logger(Initialization.class);
- private static final ConcurrentMap loadersMap = new ConcurrentHashMap<>();
+ private static final ConcurrentHashMap loadersMap = new ConcurrentHashMap<>();
- private static final Map extensionsMap = new HashMap<>();
+ private static final ConcurrentHashMap, Collection>> extensionsMap = new ConcurrentHashMap<>();
/**
* @param clazz service class
@@ -107,7 +105,7 @@ public class Initialization
public static Collection getLoadedImplementations(Class clazz)
{
@SuppressWarnings("unchecked")
- Collection retVal = extensionsMap.get(clazz);
+ Collection retVal = (Collection) extensionsMap.get(clazz);
if (retVal == null) {
return new HashSet<>();
}
@@ -138,11 +136,16 @@ public class Initialization
* elements in the returned collection is not specified and not guaranteed to be the same for different calls to
* getFromExtensions().
*/
- public static synchronized Collection getFromExtensions(ExtensionsConfig config, Class serviceClass)
+ public static Collection getFromExtensions(ExtensionsConfig config, Class serviceClass)
{
- Collection modulesToLoad = new ServiceLoadingFromExtensions<>(config, serviceClass).implsToLoad;
- extensionsMap.put(serviceClass, modulesToLoad);
- return modulesToLoad;
+ // It's not clear whether we should recompute modules even if they have been computed already for the serviceClass,
+ // but that's how it used to be an preserving the old behaviour here.
+ Collection> modules = extensionsMap.compute(
+ serviceClass,
+ (serviceC, ignored) -> new ServiceLoadingFromExtensions<>(config, serviceC).implsToLoad
+ );
+ //noinspection unchecked
+ return (Collection) modules;
}
private static class ServiceLoadingFromExtensions
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java
index 7f07ea74030..7ad67ecc8af 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderator.java
@@ -19,6 +19,7 @@
package org.apache.druid.segment.realtime.appenderator;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Supplier;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.data.input.Committer;
@@ -40,6 +41,10 @@ import java.util.List;
* You can provide a {@link Committer} or a Supplier of one when you call one of the methods that {@link #add},
* {@link #persistAll}, or {@link #push}. The Committer should represent all data you have given to the Appenderator so
* far. This Committer will be used when that data has been persisted to disk.
+ *
+ * Concurrency: all methods defined in this class directly, including {@link #close()} and {@link #closeNow()}, i. e.
+ * all methods of the data appending and indexing lifecycle except {@link #drop} must be called from a single thread.
+ * Methods inherited from {@link QuerySegmentWalker} can be called concurrently from multiple threads.
*/
public interface Appenderator extends QuerySegmentWalker, Closeable
{
@@ -56,10 +61,14 @@ public interface Appenderator extends QuerySegmentWalker, Closeable
Object startJob();
/**
- * Same as {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)}, with allowIncrementalPersists set to true
+ * Same as {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)}, with allowIncrementalPersists set to
+ * true
*/
- default AppenderatorAddResult add(SegmentIdWithShardSpec identifier, InputRow row, Supplier committerSupplier)
- throws IndexSizeExceededException, SegmentNotWritableException
+ default AppenderatorAddResult add(
+ SegmentIdWithShardSpec identifier,
+ InputRow row,
+ Supplier committerSupplier
+ ) throws IndexSizeExceededException, SegmentNotWritableException
{
return add(identifier, row, committerSupplier, true);
}
@@ -74,15 +83,13 @@ public interface Appenderator extends QuerySegmentWalker, Closeable
* Committer is guaranteed to be *created* synchronously with the call to add, but will actually be used
* asynchronously.
*
- * If committer is not provided, no metadata is persisted. If it's provided, {@link #add}, {@link #clear},
- * {@link #persistAll}, and {@link #push} methods should all be called from the same thread to keep the metadata
- * committed by Committer in sync.
+ * If committer is not provided, no metadata is persisted.
*
* @param identifier the segment into which this row should be added
* @param row the row to add
- * @param committerSupplier supplier of a committer associated with all data that has been added, including this row
- * if {@param allowIncrementalPersists} is set to false then this will not be used as no
- * persist will be done automatically
+ * @param committerSupplier supplier of a committer associated with all data that has been added, including
+ * this row if {@code allowIncrementalPersists} is set to false then this will not be
+ * used as no persist will be done automatically
* @param allowIncrementalPersists indicate whether automatic persist should be performed or not if required.
* If this flag is set to false then the return value should have
* {@link AppenderatorAddResult#isPersistRequired} set to true if persist was skipped
@@ -116,6 +123,7 @@ public interface Appenderator extends QuerySegmentWalker, Closeable
*
* @throws IllegalStateException if the segment is unknown
*/
+ @VisibleForTesting
int getRowCount(SegmentIdWithShardSpec identifier);
/**
@@ -129,20 +137,23 @@ public interface Appenderator extends QuerySegmentWalker, Closeable
* Drop all in-memory and on-disk data, and forget any previously-remembered commit metadata. This could be useful if,
* for some reason, rows have been added that we do not actually want to hand off. Blocks until all data has been
* cleared. This may take some time, since all pending persists must finish first.
- *
- * {@link #add}, {@link #clear}, {@link #persistAll}, and {@link #push} methods should all be called from the same
- * thread to keep the metadata committed by Committer in sync.
*/
+ @VisibleForTesting
void clear() throws InterruptedException;
/**
- * Drop all data associated with a particular pending segment. Unlike {@link #clear()}), any on-disk commit
- * metadata will remain unchanged. If there is no pending segment with this identifier, then this method will
+ * Schedule dropping all data associated with a particular pending segment. Unlike {@link #clear()}), any on-disk
+ * commit metadata will remain unchanged. If there is no pending segment with this identifier, then this method will
* do nothing.
*
* You should not write to the dropped segment after calling "drop". If you need to drop all your data and
* re-write it, consider {@link #clear()} instead.
*
+ * This method might be called concurrently from a thread different from the "main data appending / indexing thread",
+ * from where all other methods in this class (except those inherited from {@link QuerySegmentWalker}) are called.
+ * This typically happens when {@code drop()} is called in an async future callback. drop() itself is cheap
+ * and relays heavy dropping work to an internal executor of this Appenderator.
+ *
* @param identifier the pending segment to drop
*
* @return future that resolves when data is dropped
@@ -155,9 +166,7 @@ public interface Appenderator extends QuerySegmentWalker, Closeable
* be used asynchronously. Any metadata returned by the committer will be associated with the data persisted to
* disk.
*
- * If committer is not provided, no metadata is persisted. If it's provided, {@link #add}, {@link #clear},
- * {@link #persistAll}, and {@link #push} methods should all be called from the same thread to keep the metadata
- * committed by Committer in sync.
+ * If committer is not provided, no metadata is persisted.
*
* @param committer a committer associated with all data that has been added so far
*
@@ -171,9 +180,7 @@ public interface Appenderator extends QuerySegmentWalker, Closeable
*
* After this method is called, you cannot add new data to any segments that were previously under construction.
*
- * If committer is not provided, no metadata is persisted. If it's provided, {@link #add}, {@link #clear},
- * {@link #persistAll}, and {@link #push} methods should all be called from the same thread to keep the metadata
- * committed by Committer in sync.
+ * If committer is not provided, no metadata is persisted.
*
* @param identifiers list of segments to push
* @param committer a committer associated with all data that has been added so far
@@ -189,8 +196,9 @@ public interface Appenderator extends QuerySegmentWalker, Closeable
);
/**
- * Stop any currently-running processing and clean up after ourselves. This allows currently running persists and pushes
- * to finish. This will not remove any on-disk persisted data, but it will drop any data that has not yet been persisted.
+ * Stop any currently-running processing and clean up after ourselves. This allows currently running persists and
+ * pushes to finish. This will not remove any on-disk persisted data, but it will drop any data that has not yet been
+ * persisted.
*/
@Override
void close();
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java
index 83be7dc2431..351013474c1 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java
@@ -94,6 +94,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@@ -121,7 +122,13 @@ public class AppenderatorImpl implements Appenderator
private final IndexIO indexIO;
private final IndexMerger indexMerger;
private final Cache cache;
- private final Map sinks = new ConcurrentHashMap<>();
+ /**
+ * This map needs to be concurrent because it's accessed and mutated from multiple threads: both the thread from where
+ * this Appenderator is used (and methods like {@link #add(SegmentIdWithShardSpec, InputRow, Supplier, boolean)} are
+ * called) and from {@link #persistExecutor}. It could also be accessed (but not mutated) potentially in the context
+ * of any thread from {@link #drop}.
+ */
+ private final ConcurrentMap sinks = new ConcurrentHashMap<>();
private final Set droppingSinks = Sets.newConcurrentHashSet();
private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>(
String.CASE_INSENSITIVE_ORDER
@@ -1105,17 +1112,18 @@ public class AppenderatorImpl implements Appenderator
// Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread.
return Futures.transform(
pushBarrier(),
- new Function()
+ new Function()
{
@Nullable
@Override
- public Object apply(@Nullable Object input)
+ public Void apply(@Nullable Object input)
{
- if (sinks.get(identifier) != sink) {
- // Only abandon sink if it is the same one originally requested to be abandoned.
- log.warn("Sink for segment[%s] no longer valid, not abandoning.", identifier);
+ if (!sinks.remove(identifier, sink)) {
+ log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier);
return null;
}
+ log.info("Removing sink for segment[%s].", identifier);
+ metrics.setSinkCount(sinks.size());
if (removeOnDiskData) {
// Remove this segment from the committed list. This must be done from the persist thread.
@@ -1148,9 +1156,6 @@ public class AppenderatorImpl implements Appenderator
.emit();
}
- log.info("Removing sink for segment[%s].", identifier);
- sinks.remove(identifier);
- metrics.setSinkCount(sinks.size());
droppingSinks.remove(identifier);
sinkTimeline.remove(
sink.getInterval(),
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java
index ed0b8e43e7d..8eb27db0dd3 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java
@@ -66,6 +66,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
@@ -82,7 +83,7 @@ public class AppenderatorPlumber implements Plumber
private final SegmentPublisher segmentPublisher;
private final SegmentHandoffNotifier handoffNotifier;
private final Object handoffCondition = new Object();
- private final Map segments = new ConcurrentHashMap<>();
+ private final ConcurrentMap segments = new ConcurrentHashMap<>();
private final Appenderator appenderator;
private volatile boolean shuttingDown = false;
@@ -147,7 +148,8 @@ public class AppenderatorPlumber implements Plumber
}
@Override
- public IncrementalIndexAddResult add(InputRow row, Supplier committerSupplier) throws IndexSizeExceededException
+ public IncrementalIndexAddResult add(InputRow row, Supplier committerSupplier)
+ throws IndexSizeExceededException
{
final SegmentIdWithShardSpec identifier = getSegmentIdentifier(row.getTimestampFromEpoch());
if (identifier == null) {
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java
index a644e607fd3..cb6ba9085a7 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java
@@ -595,6 +595,7 @@ public abstract class BaseAppenderatorDriver implements Closeable
/**
* Clears out all our state and also calls {@link Appenderator#clear()} on the underlying Appenderator.
*/
+ @VisibleForTesting
public void clear() throws InterruptedException
{
synchronized (segments) {
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java
index 3abf0c4dcbf..09751f9d0fb 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java
@@ -74,7 +74,6 @@ import java.util.Optional;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
@@ -182,7 +181,7 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory producerSequences = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap producerSequences = new ConcurrentHashMap<>();
private final Stopwatch idleWatch = Stopwatch.createUnstarted();
public EventReceiverFirehose(InputRowParser