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-jar test + + 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 results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults; + + if (NullHandling.replaceWithDefault()) { + expectedResults = ImmutableList.of( + new Object[]{ + 6L, + 2L, + 2L, + 1L, + 2L, + 5L, + 5L + } + ); + } else { + expectedResults = ImmutableList.of( + new Object[]{ + 6L, + 2L, + 2L, + 1L, + 1L, + 5L, + 5L + } + ); + } + + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + // Verify query + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .virtualColumns( + new ExpressionVirtualColumn( + "a3:v", + "substring(\"dim2\", 0, 1)", + ValueType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "a4:v", + "concat(substring(\"dim2\", 0, 1),'x')", + ValueType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("a0", "cnt"), + new HllSketchBuildAggregatorFactory( + "a1", + "dim2", + null, + null + ), + new FilteredAggregatorFactory( + new HllSketchBuildAggregatorFactory( + "a2", + "dim2", + null, + null + ), + BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null)) + ), + new HllSketchBuildAggregatorFactory( + "a3", + "a3:v", + null, + null + ), + new HllSketchBuildAggregatorFactory( + "a4", + "a4:v", + null, + null + ), + new HllSketchMergeAggregatorFactory("a5", "hllsketch_dim1", 21, "HLL_8"), + new HllSketchMergeAggregatorFactory("a6", "hllsketch_dim1", null, null) + ) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); + } + + + @Test + public void testAvgDailyCountDistinctHllSketch() throws Exception + { + SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); + + final String sql = "SELECT\n" + + " AVG(u)\n" + + "FROM (SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_HLL(cnt) AS u FROM druid.foo GROUP BY 1)"; + + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{ + 1L + } + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + Query expected = GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of( + Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + new ExpressionVirtualColumn( + "d0:v", + "timestamp_floor(\"__time\",'P1D',null,'UTC')", + ValueType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + .setDimensions( + Collections.singletonList( + new DefaultDimensionSpec( + "d0:v", + "d0", + ValueType.LONG + ) + ) + ) + .setAggregatorSpecs( + Collections.singletonList( + new HllSketchBuildAggregatorFactory( + "a0:a", + "cnt", + null, + null + ) + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new FinalizingFieldAccessPostAggregator("a0", "a0:a") + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + ) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(Arrays.asList( + new LongSumAggregatorFactory("_a0:sum", "a0"), + new CountAggregatorFactory("_a0:count") + )) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "_a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "_a0:sum"), + new FieldAccessPostAggregator(null, "_a0:count") + ) + ) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build(); + + Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries()); + + // Verify query + Assert.assertEquals(expected, actual); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java index 5e321a32887..65d5717890b 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java @@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation.datasketches.quantiles; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.Row; -import org.apache.druid.initialization.DruidModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; @@ -55,8 +54,8 @@ public class DoublesSketchAggregatorTest public DoublesSketchAggregatorTest(final GroupByQueryConfig config) { - DruidModule module = new DoublesSketchModule(); - module.configure(null); + DoublesSketchModule.registerSerde(); + DoublesSketchModule module = new DoublesSketchModule(); helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( module.getJacksonModules(), config, tempFolder); timeSeriesHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper( diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java new file mode 100644 index 00000000000..ffbcc22757c --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -0,0 +1,408 @@ +/* + * 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.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.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.PostAggregator; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule; +import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +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.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.List; +import java.util.Map; + +public class DoublesSketchSqlAggregatorTest 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 + { + DoublesSketchModule.registerSerde(); + for (Module mod : new DoublesSketchModule().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 DoublesSketchAggregatorFactory( + "qsketch_m1", + "m1", + 128 + ) + ) + .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 DoublesSketchSqlAggregator()), + 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 testQuantileOnFloatAndLongs() throws Exception + { + SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); + final String sql = "SELECT\n" + + "APPROX_QUANTILE_DS(m1, 0.01),\n" + + "APPROX_QUANTILE_DS(m1, 0.5, 64),\n" + + "APPROX_QUANTILE_DS(m1, 0.98, 256),\n" + + "APPROX_QUANTILE_DS(m1, 0.99),\n" + + "APPROX_QUANTILE_DS(m1 * 2, 0.97),\n" + + "APPROX_QUANTILE_DS(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE_DS(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n" + + "APPROX_QUANTILE_DS(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE_DS(cnt, 0.5)\n" + + "FROM foo"; + + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{ + 1.0, + 4.0, + 6.0, + 6.0, + 12.0, + 6.0, + 5.0, + 6.0, + 1.0 + } + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + // Verify query + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .virtualColumns( + new ExpressionVirtualColumn( + "a4:v", + "(\"m1\" * 2)", + ValueType.FLOAT, + TestExprMacroTable.INSTANCE + ) + ) + .aggregators(ImmutableList.of( + new DoublesSketchAggregatorFactory("a0:agg", "m1", null), + new DoublesSketchAggregatorFactory("a1:agg", "m1", 64), + new DoublesSketchAggregatorFactory("a2:agg", "m1", 256), + new DoublesSketchAggregatorFactory("a4:agg", "a4:v", null), + new FilteredAggregatorFactory( + new DoublesSketchAggregatorFactory("a5:agg", "m1", null), + new SelectorDimFilter("dim1", "abc", null) + ), + new FilteredAggregatorFactory( + new DoublesSketchAggregatorFactory("a6:agg", "m1", null), + new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + ), + new DoublesSketchAggregatorFactory("a8:agg", "cnt", null) + )) + .postAggregators( + new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f), + new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.50f), + new DoublesSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 0.98f), + new DoublesSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 0.99f), + new DoublesSketchToQuantilePostAggregator("a4", makeFieldAccessPostAgg("a4:agg"), 0.97f), + new DoublesSketchToQuantilePostAggregator("a5", makeFieldAccessPostAgg("a5:agg"), 0.99f), + new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a6:agg"), 0.999f), + new DoublesSketchToQuantilePostAggregator("a7", makeFieldAccessPostAgg("a5:agg"), 0.999f), + new DoublesSketchToQuantilePostAggregator("a8", makeFieldAccessPostAgg("a8:agg"), 0.50f) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); + } + + @Test + public void testQuantileOnComplexColumn() throws Exception + { + SqlLifecycle lifecycle = sqlLifecycleFactory.factorize(); + final String sql = "SELECT\n" + + "APPROX_QUANTILE_DS(qsketch_m1, 0.01),\n" + + "APPROX_QUANTILE_DS(qsketch_m1, 0.5, 64),\n" + + "APPROX_QUANTILE_DS(qsketch_m1, 0.98, 256),\n" + + "APPROX_QUANTILE_DS(qsketch_m1, 0.99),\n" + + "APPROX_QUANTILE_DS(qsketch_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n" + + "APPROX_QUANTILE_DS(qsketch_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n" + + "FROM foo"; + + // Verify results + final List results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{ + 1.0, + 4.0, + 6.0, + 6.0, + 6.0, + 5.0, + 6.0 + } + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + // Verify query + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .aggregators(ImmutableList.of( + new DoublesSketchAggregatorFactory("a0:agg", "qsketch_m1", null), + new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", 64), + new DoublesSketchAggregatorFactory("a2:agg", "qsketch_m1", 256), + new FilteredAggregatorFactory( + new DoublesSketchAggregatorFactory("a4:agg", "qsketch_m1", null), + new SelectorDimFilter("dim1", "abc", null) + ), + new FilteredAggregatorFactory( + new DoublesSketchAggregatorFactory("a5:agg", "qsketch_m1", null), + new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + ) + )) + .postAggregators( + new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f), + new DoublesSketchToQuantilePostAggregator("a1", makeFieldAccessPostAgg("a1:agg"), 0.50f), + new DoublesSketchToQuantilePostAggregator("a2", makeFieldAccessPostAgg("a2:agg"), 0.98f), + new DoublesSketchToQuantilePostAggregator("a3", makeFieldAccessPostAgg("a0:agg"), 0.99f), + new DoublesSketchToQuantilePostAggregator("a4", makeFieldAccessPostAgg("a4:agg"), 0.99f), + new DoublesSketchToQuantilePostAggregator("a5", makeFieldAccessPostAgg("a5:agg"), 0.999f), + new DoublesSketchToQuantilePostAggregator("a6", makeFieldAccessPostAgg("a4:agg"), 0.999f) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); + } + + @Test + public void testQuantileOnInnerQuery() throws Exception + { + SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); + final String sql = "SELECT AVG(x), APPROX_QUANTILE_DS(x, 0.98)\n" + + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)"; + + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults; + if (NullHandling.replaceWithDefault()) { + expectedResults = ImmutableList.of(new Object[]{7.0, 11.0}); + } else { + expectedResults = ImmutableList.of(new Object[]{5.25, 8.0}); + } + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + // Verify query + Assert.assertEquals( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim2", "d0")) + .setAggregatorSpecs( + ImmutableList.of( + new DoubleSumAggregatorFactory("a0", "m1") + ) + ) + .setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build() + ) + ) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new DoubleSumAggregatorFactory("_a0:sum", "a0"), + new CountAggregatorFactory("_a0:count"), + new DoublesSketchAggregatorFactory( + "_a1:agg", + "a0", + null + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "_a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "_a0:sum"), + new FieldAccessPostAggregator(null, "_a0:count") + ) + ), + new DoublesSketchToQuantilePostAggregator("_a1", makeFieldAccessPostAgg("_a1:agg"), 0.98f) + ) + ) + .setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); + } + + private static PostAggregator makeFieldAccessPostAgg(String name) + { + return new FieldAccessPostAggregator(name, name); + } +} diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index 1e708128ddb..a2a4bad60a7 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -71,10 +71,9 @@ public class SketchAggregationTest public SketchAggregationTest(final GroupByQueryConfig config) { - SketchModule sm = new SketchModule(); - sm.configure(null); + SketchModule.registerSerde(); helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - sm.getJacksonModules(), + new SketchModule().getJacksonModules(), config, tempFolder ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java index cee95a6a451..0ee0e49ab1c 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java @@ -83,8 +83,8 @@ public class SketchAggregationWithSimpleDataTest @Before public void setup() throws Exception { + SketchModule.registerSerde(); sm = new SketchModule(); - sm.configure(null); try ( final AggregationTestHelper toolchest = AggregationTestHelper.createGroupByQueryAggregationTestHelper( sm.getJacksonModules(), @@ -264,8 +264,8 @@ public class SketchAggregationWithSimpleDataTest @Test public void testSimpleDataIngestAndSelectQuery() throws Exception { + SketchModule.registerSerde(); SketchModule sm = new SketchModule(); - sm.configure(null); AggregationTestHelper selectQueryAggregationTestHelper = AggregationTestHelper.createSelectQueryAggregationTestHelper( sm.getJacksonModules(), tempFolder diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java new file mode 100644 index 00000000000..e844864f6f7 --- /dev/null +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java @@ -0,0 +1,402 @@ +/* + * 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 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.theta.SketchMergeAggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; +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 ThetaSketchSqlAggregatorTest 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 + { + SketchModule.registerSerde(); + for (Module mod : new SketchModule().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 SketchMergeAggregatorFactory( + "thetasketch_dim1", + "dim1", + null, + false, + false, + 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 ThetaSketchSqlAggregator()), + 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 testApproxCountDistinctThetaSketch() throws Exception + { + SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); + final String sql = "SELECT\n" + + " SUM(cnt),\n" + + " APPROX_COUNT_DISTINCT_DS_THETA(dim2),\n" // uppercase + + " APPROX_COUNT_DISTINCT_DS_THETA(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered + + " APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1)),\n" // on extractionFn + + " APPROX_COUNT_DISTINCT_DS_THETA(SUBSTRING(dim2, 1, 1) || 'x'),\n" // on expression + + " APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1, 32768),\n" // on native theta sketch column + + " APPROX_COUNT_DISTINCT_DS_THETA(thetasketch_dim1)\n" // on native theta sketch column + + "FROM druid.foo"; + + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults; + + if (NullHandling.replaceWithDefault()) { + expectedResults = ImmutableList.of( + new Object[]{ + 6L, + 2L, + 2L, + 1L, + 2L, + 5L, + 5L + } + ); + } else { + expectedResults = ImmutableList.of( + new Object[]{ + 6L, + 2L, + 2L, + 1L, + 1L, + 5L, + 5L + } + ); + } + + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + // Verify query + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .virtualColumns( + new ExpressionVirtualColumn( + "a3:v", + "substring(\"dim2\", 0, 1)", + ValueType.STRING, + TestExprMacroTable.INSTANCE + ), + new ExpressionVirtualColumn( + "a4:v", + "concat(substring(\"dim2\", 0, 1),'x')", + ValueType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("a0", "cnt"), + new SketchMergeAggregatorFactory( + "a1", + "dim2", + null, + null, + null, + null + ), + new FilteredAggregatorFactory( + new SketchMergeAggregatorFactory( + "a2", + "dim2", + null, + null, + null, + null + ), + BaseCalciteQueryTest.NOT(BaseCalciteQueryTest.SELECTOR("dim2", "", null)) + ), + new SketchMergeAggregatorFactory( + "a3", + "a3:v", + null, + null, + null, + null + ), + new SketchMergeAggregatorFactory( + "a4", + "a4:v", + null, + null, + null, + null + ), + new SketchMergeAggregatorFactory("a5", "thetasketch_dim1", 32768, null, null, null), + new SketchMergeAggregatorFactory("a6", "thetasketch_dim1", null, null, null, null) + ) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); + } + + @Test + public void testAvgDailyCountDistinctThetaSketch() throws Exception + { + SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); + + final String sql = "SELECT\n" + + " AVG(u)\n" + + "FROM (SELECT FLOOR(__time TO DAY), APPROX_COUNT_DISTINCT_DS_THETA(cnt) AS u FROM druid.foo GROUP BY 1)"; + + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{ + 1L + } + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + Query expected = GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of( + Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + new ExpressionVirtualColumn( + "d0:v", + "timestamp_floor(\"__time\",'P1D',null,'UTC')", + ValueType.LONG, + TestExprMacroTable.INSTANCE + ) + ) + .setDimensions( + Collections.singletonList( + new DefaultDimensionSpec( + "d0:v", + "d0", + ValueType.LONG + ) + ) + ) + .setAggregatorSpecs( + Collections.singletonList( + new SketchMergeAggregatorFactory( + "a0:a", + "cnt", + null, + null, + null, + null + ) + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new FinalizingFieldAccessPostAggregator("a0", "a0:a") + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + ) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(Arrays.asList( + new LongSumAggregatorFactory("_a0:sum", "a0"), + new CountAggregatorFactory("_a0:count") + )) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "_a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "_a0:sum"), + new FieldAccessPostAggregator(null, "_a0:count") + ) + ) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build(); + + Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries()); + + // Verify query + Assert.assertEquals(expected, actual); + } +} diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java index 167f0c26afe..04d171e18e8 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java @@ -21,9 +21,11 @@ package org.apache.druid.query.aggregation.histogram; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import org.apache.druid.initialization.DruidModule; +import org.apache.druid.query.aggregation.histogram.sql.FixedBucketsHistogramQuantileSqlAggregator; import org.apache.druid.query.aggregation.histogram.sql.QuantileSqlAggregator; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.sql.guice.SqlBindings; @@ -55,6 +57,14 @@ public class ApproximateHistogramDruidModule implements DruidModule @Override public void configure(Binder binder) + { + registerSerde(); + SqlBindings.addAggregator(binder, QuantileSqlAggregator.class); + SqlBindings.addAggregator(binder, FixedBucketsHistogramQuantileSqlAggregator.class); + } + + @VisibleForTesting + public static void registerSerde() { if (ComplexMetrics.getSerdeForType("approximateHistogram") == null) { ComplexMetrics.registerSerde("approximateHistogram", new ApproximateHistogramFoldingSerde()); @@ -63,10 +73,5 @@ public class ApproximateHistogramDruidModule implements DruidModule if (ComplexMetrics.getSerdeForType(FixedBucketsHistogramAggregator.TYPE_NAME) == null) { ComplexMetrics.registerSerde(FixedBucketsHistogramAggregator.TYPE_NAME, new FixedBucketsHistogramSerde()); } - - if (binder != null) { - // Binder is null in some tests. - SqlBindings.addAggregator(binder, QuantileSqlAggregator.class); - } } } diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java new file mode 100644 index 00000000000..3a28edaeb9e --- /dev/null +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java @@ -0,0 +1,322 @@ +/* + * 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.histogram.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.histogram.FixedBucketsHistogram; +import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory; +import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator; +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 FixedBucketsHistogramQuantileSqlAggregator implements SqlAggregator +{ + private static final SqlAggFunction FUNCTION_INSTANCE = new FixedBucketsHistogramQuantileSqlAggFunction(); + private static final String NAME = "APPROX_QUANTILE_FIXED_BUCKETS"; + + @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 + ) + { + 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 numBuckets; + if (aggregateCall.getArgList().size() >= 3) { + final RexNode numBucketsArg = Expressions.fromFieldAccess( + rowSignature, + project, + aggregateCall.getArgList().get(2) + ); + + if (!numBucketsArg.isA(SqlKind.LITERAL)) { + // Resolution must be a literal in order to plan. + return null; + } + + numBuckets = ((Number) RexLiteral.value(numBucketsArg)).intValue(); + } else { + return null; + } + + final double lowerLimit; + if (aggregateCall.getArgList().size() >= 4) { + final RexNode lowerLimitArg = Expressions.fromFieldAccess( + rowSignature, + project, + aggregateCall.getArgList().get(3) + ); + + if (!lowerLimitArg.isA(SqlKind.LITERAL)) { + // Resolution must be a literal in order to plan. + return null; + } + + lowerLimit = ((Number) RexLiteral.value(lowerLimitArg)).doubleValue(); + } else { + return null; + } + + final double upperLimit; + if (aggregateCall.getArgList().size() >= 5) { + final RexNode upperLimitArg = Expressions.fromFieldAccess( + rowSignature, + project, + aggregateCall.getArgList().get(4) + ); + + if (!upperLimitArg.isA(SqlKind.LITERAL)) { + // Resolution must be a literal in order to plan. + return null; + } + + upperLimit = ((Number) RexLiteral.value(upperLimitArg)).doubleValue(); + } else { + return null; + } + + final FixedBucketsHistogram.OutlierHandlingMode outlierHandlingMode; + if (aggregateCall.getArgList().size() >= 6) { + final RexNode outlierHandlingModeArg = Expressions.fromFieldAccess( + rowSignature, + project, + aggregateCall.getArgList().get(5) + ); + + if (!outlierHandlingModeArg.isA(SqlKind.LITERAL)) { + // Resolution must be a literal in order to plan. + return null; + } + + outlierHandlingMode = FixedBucketsHistogram.OutlierHandlingMode.fromString( + RexLiteral.stringValue(outlierHandlingModeArg) + ); + } else { + outlierHandlingMode = FixedBucketsHistogram.OutlierHandlingMode.IGNORE; + } + + // Look for existing matching aggregatorFactory. + for (final Aggregation existing : existingAggregations) { + for (AggregatorFactory factory : existing.getAggregatorFactories()) { + if (factory instanceof FixedBucketsHistogramAggregatorFactory) { + final FixedBucketsHistogramAggregatorFactory theFactory = (FixedBucketsHistogramAggregatorFactory) 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.getOutlierHandlingMode() == outlierHandlingMode + && theFactory.getNumBuckets() == numBuckets + && theFactory.getLowerLimit() == lowerLimit + && theFactory.getUpperLimit() == upperLimit; + + if (matches) { + // Found existing one. Use this. + return Aggregation.create( + ImmutableList.of(), + new QuantilePostAggregator(name, factory.getName(), probability) + ); + } + } + } + } + + // No existing match found. Create a new one. + final List virtualColumns = new ArrayList<>(); + + if (input.isDirectColumnAccess()) { + aggregatorFactory = new FixedBucketsHistogramAggregatorFactory( + histogramName, + input.getDirectColumn(), + numBuckets, + lowerLimit, + upperLimit, + outlierHandlingMode + ); + } else { + final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn( + StringUtils.format("%s:v", name), + ValueType.FLOAT, + plannerContext.getExprMacroTable() + ); + virtualColumns.add(virtualColumn); + aggregatorFactory = new FixedBucketsHistogramAggregatorFactory( + histogramName, + virtualColumn.getOutputName(), + numBuckets, + lowerLimit, + upperLimit, + outlierHandlingMode + ); + } + + return Aggregation.create( + virtualColumns, + ImmutableList.of(aggregatorFactory), + new QuantilePostAggregator(name, histogramName, probability) + ); + } + + private static class FixedBucketsHistogramQuantileSqlAggFunction extends SqlAggFunction + { + private static final String SIGNATURE1 = + "'" + + NAME + + "(column, probability, numBuckets, lowerLimit, upperLimit)'\n"; + private static final String SIGNATURE2 = + "'" + + NAME + + "(column, probability, numBuckets, lowerLimit, upperLimit, outlierHandlingMode)'\n"; + + FixedBucketsHistogramQuantileSqlAggFunction() + { + super( + NAME, + null, + SqlKind.OTHER_FUNCTION, + ReturnTypes.explicit(SqlTypeName.DOUBLE), + null, + OperandTypes.or( + OperandTypes.and( + OperandTypes.sequence( + SIGNATURE1, + OperandTypes.ANY, + OperandTypes.LITERAL, + OperandTypes.LITERAL, + OperandTypes.LITERAL, + OperandTypes.LITERAL + ), + OperandTypes.family( + SqlTypeFamily.ANY, + SqlTypeFamily.NUMERIC, + SqlTypeFamily.NUMERIC, + SqlTypeFamily.NUMERIC, + SqlTypeFamily.NUMERIC + ) + ), + OperandTypes.and( + OperandTypes.sequence( + SIGNATURE2, + OperandTypes.ANY, + OperandTypes.LITERAL, + OperandTypes.LITERAL, + OperandTypes.LITERAL, + OperandTypes.LITERAL, + OperandTypes.LITERAL + ), + OperandTypes.family( + SqlTypeFamily.ANY, + SqlTypeFamily.NUMERIC, + SqlTypeFamily.NUMERIC, + SqlTypeFamily.NUMERIC, + SqlTypeFamily.NUMERIC, + SqlTypeFamily.STRING + ) + ) + ), + SqlFunctionCategory.NUMERIC, + false, + false + ); + } + } +} diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java index 793270a2be6..36103b94ceb 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java @@ -52,10 +52,9 @@ public class ApproximateHistogramAggregationTest public ApproximateHistogramAggregationTest(final GroupByQueryConfig config) { - ApproximateHistogramDruidModule module = new ApproximateHistogramDruidModule(); - module.configure(null); + ApproximateHistogramDruidModule.registerSerde(); helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - Lists.newArrayList(module.getJacksonModules()), + Lists.newArrayList(new ApproximateHistogramDruidModule().getJacksonModules()), config, tempFolder ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index 8f0acf5148d..08dd8b1d039 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -145,9 +145,7 @@ public class ApproximateHistogramGroupByQueryTest { this.factory = factory; this.runner = runner; - - //Note: this is needed in order to properly register the serde for Histogram. - new ApproximateHistogramDruidModule().configure(null); + ApproximateHistogramDruidModule.registerSerde(); } @After diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java index 838cae06b28..1d817375131 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregationTest.java @@ -52,10 +52,9 @@ public class FixedBucketsHistogramAggregationTest public FixedBucketsHistogramAggregationTest(final GroupByQueryConfig config) { - ApproximateHistogramDruidModule module = new ApproximateHistogramDruidModule(); - module.configure(null); + ApproximateHistogramDruidModule.registerSerde(); helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - Lists.newArrayList(module.getJacksonModules()), + Lists.newArrayList(new ApproximateHistogramDruidModule().getJacksonModules()), config, tempFolder ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java index 4ee5283502f..8c038a01aff 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java @@ -146,8 +146,7 @@ public class FixedBucketsHistogramGroupByQueryTest this.factory = factory; this.runner = runner; - //Note: this is needed in order to properly register the serde for Histogram. - new ApproximateHistogramDruidModule().configure(null); + ApproximateHistogramDruidModule.registerSerde(); } @After diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java new file mode 100644 index 00000000000..4ba94fe76d0 --- /dev/null +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java @@ -0,0 +1,452 @@ +/* + * 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.histogram.sql; + +import com.fasterxml.jackson.databind.Module; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import 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.histogram.ApproximateHistogramDruidModule; +import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogram; +import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory; +import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +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.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.List; +import java.util.Map; + +public class FixedBucketsHistogramQuantileSqlAggregatorTest 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 + { + ApproximateHistogramDruidModule.registerSerde(); + for (Module mod : new ApproximateHistogramDruidModule().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 FixedBucketsHistogramAggregatorFactory( + "fbhist_m1", + "m1", + 20, + 0, + 10, + FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ) + ) + .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 QuantileSqlAggregator(), new FixedBucketsHistogramQuantileSqlAggregator()), + 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 testQuantileOnFloatAndLongs() throws Exception + { + SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); + final String sql = "SELECT\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.01, 20, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.5, 20, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.98, 20, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1 * 2, 0.97, 40, 0.0, 20.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(cnt, 0.5, 20, 0.0, 10.0)\n" + + "FROM foo"; + + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{ + 1.0299999713897705, + 3.5, + 6.440000057220459, + 6.470000267028809, + 12.40999984741211, + 6.494999885559082, + 5.497499942779541, + 6.499499797821045, + 1.25 + } + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries()); + Query expected = Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .virtualColumns( + new ExpressionVirtualColumn( + "a4:v", + "(\"m1\" * 2)", + ValueType.FLOAT, + TestExprMacroTable.INSTANCE + ) + ) + .aggregators(ImmutableList.of( + new FixedBucketsHistogramAggregatorFactory( + "a0:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new FixedBucketsHistogramAggregatorFactory( + "a4:agg", "a4:v", 40, 0.0d, 20.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new FilteredAggregatorFactory( + new FixedBucketsHistogramAggregatorFactory( + "a5:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new SelectorDimFilter("dim1", "abc", null) + ), + new FilteredAggregatorFactory( + new FixedBucketsHistogramAggregatorFactory( + "a6:agg", "m1", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + ), + new FixedBucketsHistogramAggregatorFactory( + "a8:agg", "cnt", 20, 0.0d, 10.0d, FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ) + )) + .postAggregators( + new QuantilePostAggregator("a0", "a0:agg", 0.01f), + new QuantilePostAggregator("a1", "a0:agg", 0.50f), + new QuantilePostAggregator("a2", "a0:agg", 0.98f), + new QuantilePostAggregator("a3", "a0:agg", 0.99f), + new QuantilePostAggregator("a4", "a4:agg", 0.97f), + new QuantilePostAggregator("a5", "a5:agg", 0.99f), + new QuantilePostAggregator("a6", "a6:agg", 0.999f), + new QuantilePostAggregator("a7", "a5:agg", 0.999f), + new QuantilePostAggregator("a8", "a8:agg", 0.50f) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(); + + // Verify query + Assert.assertEquals( + expected, + actual + ); + } + + @Test + public void testQuantileOnComplexColumn() throws Exception + { + SqlLifecycle lifecycle = sqlLifecycleFactory.factorize(); + final String sql = "SELECT\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.01, 20, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.5, 20, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.98, 30, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.99, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 <> 'abc'),\n" + + "APPROX_QUANTILE_FIXED_BUCKETS(fbhist_m1, 0.999, 20, 0.0, 10.0) FILTER(WHERE dim1 = 'abc')\n" + + "FROM foo"; + + // Verify results + final List results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{ + 1.0299999713897705, + 3.5, + 6.293333530426025, + 6.470000267028809, + 6.494999885559082, + 5.497499942779541, + 6.499499797821045 + } + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries()); + Query expected = Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .aggregators(ImmutableList.of( + new FixedBucketsHistogramAggregatorFactory( + "a0:agg", + "fbhist_m1", + 20, + 0.0, + 10.0, + FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new FixedBucketsHistogramAggregatorFactory( + "a2:agg", + "fbhist_m1", + 30, + 0.0, + 10.0, + FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new FilteredAggregatorFactory( + new FixedBucketsHistogramAggregatorFactory( + "a4:agg", + "fbhist_m1", + 20, + 0.0, + 10.0, + FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new SelectorDimFilter("dim1", "abc", null) + ), + new FilteredAggregatorFactory( + new FixedBucketsHistogramAggregatorFactory( + "a5:agg", + "fbhist_m1", + 20, + 0.0, + 10.0, + FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ), + new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + ) + )) + .postAggregators( + new QuantilePostAggregator("a0", "a0:agg", 0.01f), + new QuantilePostAggregator("a1", "a0:agg", 0.50f), + new QuantilePostAggregator("a2", "a2:agg", 0.98f), + new QuantilePostAggregator("a3", "a0:agg", 0.99f), + new QuantilePostAggregator("a4", "a4:agg", 0.99f), + new QuantilePostAggregator("a5", "a5:agg", 0.999f), + new QuantilePostAggregator("a6", "a4:agg", 0.999f) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(); + + // Verify query + Assert.assertEquals(expected, actual); + } + + @Test + public void testQuantileOnInnerQuery() throws Exception + { + SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); + final String sql = "SELECT AVG(x), APPROX_QUANTILE_FIXED_BUCKETS(x, 0.98, 100, 0.0, 100.0)\n" + + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)"; + + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults; + if (NullHandling.replaceWithDefault()) { + expectedResults = ImmutableList.of(new Object[]{7.0, 11.940000534057617}); + } else { + expectedResults = ImmutableList.of(new Object[]{5.25, 8.920000076293945}); + } + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + Query actual = Iterables.getOnlyElement(queryLogHook.getRecordedQueries()); + Query expected = GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of( + Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim2", "d0")) + .setAggregatorSpecs( + ImmutableList.of( + new DoubleSumAggregatorFactory("a0", "m1") + ) + ) + .setContext(ImmutableMap.of( + PlannerContext.CTX_SQL_QUERY_ID, + "dummy" + )) + .build() + ) + ) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new DoubleSumAggregatorFactory("_a0:sum", "a0"), + new CountAggregatorFactory("_a0:count"), + new FixedBucketsHistogramAggregatorFactory( + "_a1:agg", + "a0", + 100, + 0, + 100.0d, + FixedBucketsHistogram.OutlierHandlingMode.IGNORE + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "_a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "_a0:sum"), + new FieldAccessPostAggregator(null, "_a0:count") + ) + ), + new QuantilePostAggregator("_a1", "_a1:agg", 0.98f) + ) + ) + .setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(); + + // Verify query + Assert.assertEquals(expected, actual); + } +} 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 7aa4e6733ea..ee354e5baab 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.aggregation.histogram.sql; +import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -119,8 +120,10 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase @Before public void setUp() throws Exception { - // Note: this is needed in order to properly register the serde for Histogram. - new ApproximateHistogramDruidModule().configure(null); + ApproximateHistogramDruidModule.registerSerde(); + for (Module mod : new ApproximateHistogramDruidModule().getJacksonModules()) { + CalciteTests.getJsonMapper().registerModule(mod); + } final QueryableIndex index = IndexBuilder.create() .tmpDir(temporaryFolder.newFolder()) @@ -188,221 +191,210 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase public void testQuantileOnFloatAndLongs() throws Exception { SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); - try { - final String sql = "SELECT\n" - + "APPROX_QUANTILE(m1, 0.01),\n" - + "APPROX_QUANTILE(m1, 0.5, 50),\n" - + "APPROX_QUANTILE(m1, 0.98, 200),\n" - + "APPROX_QUANTILE(m1, 0.99),\n" - + "APPROX_QUANTILE(m1 * 2, 0.97),\n" - + "APPROX_QUANTILE(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n" - + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n" - + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n" - + "APPROX_QUANTILE(cnt, 0.5)\n" - + "FROM foo"; - // Verify results - final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); - final List expectedResults = ImmutableList.of( - new Object[]{ - 1.0, - 3.0, - 5.880000114440918, - 5.940000057220459, - 11.640000343322754, - 6.0, - 4.994999885559082, - 6.0, - 1.0 - } - ); - Assert.assertEquals(expectedResults.size(), results.size()); - for (int i = 0; i < expectedResults.size(); i++) { - Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); - } + final String sql = "SELECT\n" + + "APPROX_QUANTILE(m1, 0.01),\n" + + "APPROX_QUANTILE(m1, 0.5, 50),\n" + + "APPROX_QUANTILE(m1, 0.98, 200),\n" + + "APPROX_QUANTILE(m1, 0.99),\n" + + "APPROX_QUANTILE(m1 * 2, 0.97),\n" + + "APPROX_QUANTILE(m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n" + + "APPROX_QUANTILE(m1, 0.999) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE(cnt, 0.5)\n" + + "FROM foo"; - // Verify query - Assert.assertEquals( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) - .granularity(Granularities.ALL) - .virtualColumns( - new ExpressionVirtualColumn( - "a4:v", - "(\"m1\" * 2)", - ValueType.FLOAT, - TestExprMacroTable.INSTANCE - ) - ) - .aggregators(ImmutableList.of( - new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null), - new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null), - new ApproximateHistogramAggregatorFactory("a4:agg", "a4:v", null, null, null, null), - new FilteredAggregatorFactory( - new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null), - new SelectorDimFilter("dim1", "abc", null) - ), - new FilteredAggregatorFactory( - new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) - ), - new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null) - )) - .postAggregators( - new QuantilePostAggregator("a0", "a0:agg", 0.01f), - new QuantilePostAggregator("a1", "a0:agg", 0.50f), - new QuantilePostAggregator("a2", "a2:agg", 0.98f), - new QuantilePostAggregator("a3", "a0:agg", 0.99f), - new QuantilePostAggregator("a4", "a4:agg", 0.97f), - new QuantilePostAggregator("a5", "a5:agg", 0.99f), - new QuantilePostAggregator("a6", "a6:agg", 0.999f), - new QuantilePostAggregator("a7", "a5:agg", 0.999f), - new QuantilePostAggregator("a8", "a8:agg", 0.50f) - ) - .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) - .build(), - Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) - ); - } - catch (Exception e) { - throw e; + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{ + 1.0, + 3.0, + 5.880000114440918, + 5.940000057220459, + 11.640000343322754, + 6.0, + 4.994999885559082, + 6.0, + 1.0 + } + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); } + + // Verify query + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .virtualColumns( + new ExpressionVirtualColumn( + "a4:v", + "(\"m1\" * 2)", + ValueType.FLOAT, + TestExprMacroTable.INSTANCE + ) + ) + .aggregators(ImmutableList.of( + new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null), + new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null), + new ApproximateHistogramAggregatorFactory("a4:agg", "a4:v", null, null, null, null), + new FilteredAggregatorFactory( + new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null), + new SelectorDimFilter("dim1", "abc", null) + ), + new FilteredAggregatorFactory( + new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null), + new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + ), + new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null) + )) + .postAggregators( + new QuantilePostAggregator("a0", "a0:agg", 0.01f), + new QuantilePostAggregator("a1", "a0:agg", 0.50f), + new QuantilePostAggregator("a2", "a2:agg", 0.98f), + new QuantilePostAggregator("a3", "a0:agg", 0.99f), + new QuantilePostAggregator("a4", "a4:agg", 0.97f), + new QuantilePostAggregator("a5", "a5:agg", 0.99f), + new QuantilePostAggregator("a6", "a6:agg", 0.999f), + new QuantilePostAggregator("a7", "a5:agg", 0.999f), + new QuantilePostAggregator("a8", "a8:agg", 0.50f) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); } @Test public void testQuantileOnComplexColumn() throws Exception { SqlLifecycle lifecycle = sqlLifecycleFactory.factorize(); - try { - final String sql = "SELECT\n" - + "APPROX_QUANTILE(hist_m1, 0.01),\n" - + "APPROX_QUANTILE(hist_m1, 0.5, 50),\n" - + "APPROX_QUANTILE(hist_m1, 0.98, 200),\n" - + "APPROX_QUANTILE(hist_m1, 0.99),\n" - + "APPROX_QUANTILE(hist_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n" - + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n" - + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n" - + "FROM foo"; + final String sql = "SELECT\n" + + "APPROX_QUANTILE(hist_m1, 0.01),\n" + + "APPROX_QUANTILE(hist_m1, 0.5, 50),\n" + + "APPROX_QUANTILE(hist_m1, 0.98, 200),\n" + + "APPROX_QUANTILE(hist_m1, 0.99),\n" + + "APPROX_QUANTILE(hist_m1, 0.99) FILTER(WHERE dim1 = 'abc'),\n" + + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 <> 'abc'),\n" + + "APPROX_QUANTILE(hist_m1, 0.999) FILTER(WHERE dim1 = 'abc')\n" + + "FROM foo"; - // Verify results - final List results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); - final List expectedResults = ImmutableList.of( - new Object[]{1.0, 3.0, 5.880000114440918, 5.940000057220459, 6.0, 4.994999885559082, 6.0} - ); - Assert.assertEquals(expectedResults.size(), results.size()); - for (int i = 0; i < expectedResults.size(); i++) { - Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); - } + // Verify results + final List results = lifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults = ImmutableList.of( + new Object[]{1.0, 3.0, 5.880000114440918, 5.940000057220459, 6.0, 4.994999885559082, 6.0} + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } - // Verify query - Assert.assertEquals( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) - .granularity(Granularities.ALL) - .aggregators(ImmutableList.of( - new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null), - new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null), - new FilteredAggregatorFactory( - new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null), - new SelectorDimFilter("dim1", "abc", null) - ), - new FilteredAggregatorFactory( - new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) - ) - )) - .postAggregators( - new QuantilePostAggregator("a0", "a0:agg", 0.01f), - new QuantilePostAggregator("a1", "a0:agg", 0.50f), - new QuantilePostAggregator("a2", "a2:agg", 0.98f), - new QuantilePostAggregator("a3", "a0:agg", 0.99f), - new QuantilePostAggregator("a4", "a4:agg", 0.99f), - new QuantilePostAggregator("a5", "a5:agg", 0.999f), - new QuantilePostAggregator("a6", "a4:agg", 0.999f) - ) - .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) - .build(), - Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) - ); - } - catch (Exception e) { - throw e; - } + // Verify query + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .aggregators(ImmutableList.of( + new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null), + new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null), + new FilteredAggregatorFactory( + new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null), + new SelectorDimFilter("dim1", "abc", null) + ), + new FilteredAggregatorFactory( + new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null), + new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + ) + )) + .postAggregators( + new QuantilePostAggregator("a0", "a0:agg", 0.01f), + new QuantilePostAggregator("a1", "a0:agg", 0.50f), + new QuantilePostAggregator("a2", "a2:agg", 0.98f), + new QuantilePostAggregator("a3", "a0:agg", 0.99f), + new QuantilePostAggregator("a4", "a4:agg", 0.99f), + new QuantilePostAggregator("a5", "a5:agg", 0.999f), + new QuantilePostAggregator("a6", "a4:agg", 0.999f) + ) + .context(ImmutableMap.of("skipEmptyBuckets", true, PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); } @Test public void testQuantileOnInnerQuery() throws Exception { SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize(); - try { - final String sql = "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n" - + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)"; + final String sql = "SELECT AVG(x), APPROX_QUANTILE(x, 0.98)\n" + + "FROM (SELECT dim2, SUM(m1) AS x FROM foo GROUP BY dim2)"; - // Verify results - final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); - final List expectedResults; - if (NullHandling.replaceWithDefault()) { - expectedResults = ImmutableList.of(new Object[]{7.0, 8.26386833190918}); - } else { - expectedResults = ImmutableList.of(new Object[]{5.25, 6.59091854095459}); - } - Assert.assertEquals(expectedResults.size(), results.size()); - for (int i = 0; i < expectedResults.size(); i++) { - Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); - } + // Verify results + final List results = sqlLifecycle.runSimple(sql, QUERY_CONTEXT_DEFAULT, authenticationResult).toList(); + final List expectedResults; + if (NullHandling.replaceWithDefault()) { + expectedResults = ImmutableList.of(new Object[]{7.0, 8.26386833190918}); + } else { + expectedResults = ImmutableList.of(new Object[]{5.25, 6.59091854095459}); + } + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } - // Verify query - Assert.assertEquals( - GroupByQuery.builder() - .setDataSource( - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) - .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("dim2", "d0")) - .setAggregatorSpecs( - ImmutableList.of( - new DoubleSumAggregatorFactory("a0", "m1") - ) - ) - .setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy")) - .build() - ) - ) - .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs(new DoubleSumAggregatorFactory("_a0:sum", "a0"), - new CountAggregatorFactory("_a0:count"), - new ApproximateHistogramAggregatorFactory("_a1:agg", - "a0", - null, - null, - null, - null - )) - .setPostAggregatorSpecs( - ImmutableList.of( - new ArithmeticPostAggregator( - "_a0", - "quotient", - ImmutableList.of( - new FieldAccessPostAggregator(null, "_a0:sum"), - new FieldAccessPostAggregator(null, "_a0:count") - ) - ), - new QuantilePostAggregator("_a1", "_a1:agg", 0.98f) - ) - ) - .setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy")) - .build(), - Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) - ); - } - catch (Exception e) { - throw e; - } + // Verify query + Assert.assertEquals( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim2", "d0")) + .setAggregatorSpecs( + ImmutableList.of( + new DoubleSumAggregatorFactory("a0", "m1") + ) + ) + .setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build() + ) + ) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new DoubleSumAggregatorFactory("_a0:sum", "a0"), + new CountAggregatorFactory("_a0:count"), + new ApproximateHistogramAggregatorFactory( + "_a1:agg", + "a0", + null, + null, + null, + null + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "_a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "_a0:sum"), + new FieldAccessPostAggregator(null, "_a0:count") + ) + ), + new QuantilePostAggregator("_a1", "_a1:agg", 0.98f) + ) + ) + .setContext(ImmutableMap.of(PlannerContext.CTX_SQL_QUERY_ID, "dummy")) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); } } From 6207b66e20a807c1cc74c914f2734c5dab5bfea7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 3 Feb 2019 09:38:51 -0800 Subject: [PATCH 05/13] fix build (#6994) --- .../datasketches/hll/sql/HllSketchSqlAggregatorTest.java | 2 +- .../quantiles/sql/DoublesSketchSqlAggregatorTest.java | 2 +- .../datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java | 2 +- .../sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index 46eb02c01c8..1690ef9b389 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -161,7 +161,7 @@ public class HllSketchSqlAggregatorTest 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 HllSketchSqlAggregator()), ImmutableSet.of() diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java index ffbcc22757c..2b4e5e0f723 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -157,7 +157,7 @@ public class DoublesSketchSqlAggregatorTest 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 DoublesSketchSqlAggregator()), ImmutableSet.of() diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java index e844864f6f7..919a59618a0 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java @@ -162,7 +162,7 @@ public class ThetaSketchSqlAggregatorTest 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 ThetaSketchSqlAggregator()), ImmutableSet.of() diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java index 4ba94fe76d0..2bac9e01a6f 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java @@ -161,7 +161,7 @@ public class FixedBucketsHistogramQuantileSqlAggregatorTest extends CalciteTestB 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(), new FixedBucketsHistogramQuantileSqlAggregator()), ImmutableSet.of() From 06d56c2abd603be4e0549bd96fcc696c87509805 Mon Sep 17 00:00:00 2001 From: Vadim Ogievetsky Date: Mon, 4 Feb 2019 09:04:58 -0800 Subject: [PATCH 06/13] Fixing little issues and CSS bugs in the new web console. (#6995) --- web-console/src/console-application.scss | 5 ++++- web-console/src/console-application.tsx | 8 ++++---- web-console/src/dialogs/coordinator-dynamic-config.scss | 1 - web-console/src/dialogs/retention-dialog.tsx | 2 +- web-console/src/utils/general.tsx | 5 +++++ web-console/src/views/home-view.tsx | 6 +++--- web-console/src/views/tasks-view.tsx | 6 ++++-- 7 files changed, 21 insertions(+), 12 deletions(-) diff --git a/web-console/src/console-application.scss b/web-console/src/console-application.scss index b1df645df8c..536e59c0d32 100644 --- a/web-console/src/console-application.scss +++ b/web-console/src/console-application.scss @@ -39,8 +39,11 @@ left: 0; right: 0; bottom: 0; - overflow-y: scroll; padding: $standard-padding; + + &.scrollable { + overflow-y: scroll; + } } .control-separator { diff --git a/web-console/src/console-application.tsx b/web-console/src/console-application.tsx index 7aa1975b84b..38c4c9d8990 100644 --- a/web-console/src/console-application.tsx +++ b/web-console/src/console-application.tsx @@ -131,10 +131,10 @@ export class ConsoleApplication extends React.Component { + const wrapInViewContainer = (active: HeaderActiveTab, el: JSX.Element, scrollable = false) => { return <> -
{el}
+
{el}
; }; @@ -148,10 +148,10 @@ export class ConsoleApplication extends React.Component); }} /> { - return wrapInViewContainer('tasks', ); + return wrapInViewContainer('tasks', , true); }} /> { - return wrapInViewContainer('servers', ); + return wrapInViewContainer('servers', , true); }} /> { return wrapInViewContainer('sql', ); diff --git a/web-console/src/dialogs/coordinator-dynamic-config.scss b/web-console/src/dialogs/coordinator-dynamic-config.scss index 663c029c384..a95edc25040 100644 --- a/web-console/src/dialogs/coordinator-dynamic-config.scss +++ b/web-console/src/dialogs/coordinator-dynamic-config.scss @@ -18,7 +18,6 @@ .coordinator-dynamic-config { .bp3-dialog-body { - overflow: scroll; max-height: 70vh; .auto-form { diff --git a/web-console/src/dialogs/retention-dialog.tsx b/web-console/src/dialogs/retention-dialog.tsx index 668b6887d1c..36a244a3076 100644 --- a/web-console/src/dialogs/retention-dialog.tsx +++ b/web-console/src/dialogs/retention-dialog.tsx @@ -167,7 +167,7 @@ export class RetentionDialog extends React.Component - This datasource currently has no rule, it will use the cluster defaults (edit cluster defaults) + This datasource currently has no rules, it will use the cluster defaults (edit cluster defaults)

} ; 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> parser) @@ -487,11 +486,11 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory sinks = new ConcurrentHashMap<>(); + private final ConcurrentMap sinks = new ConcurrentHashMap<>(); private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( String.CASE_INSENSITIVE_ORDER ); diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index 7db7cf2d7d9..fe42ad05c05 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -51,6 +51,7 @@ import javax.servlet.http.HttpServletRequest; import java.util.LinkedHashMap; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; /** @@ -248,7 +249,7 @@ public class QueryLifecycle { transition(State.AUTHORIZED, State.EXECUTING); - final Map responseContext = DirectDruidClient.makeResponseContextForQuery(); + final ConcurrentMap responseContext = DirectDruidClient.makeResponseContextForQuery(); final Sequence res = QueryPlus.wrap(baseQuery) .withIdentity(authenticationResult.getIdentity()) diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java index 07ad0efa005..d45c51cced6 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -45,9 +45,9 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.atomic.AtomicLong; @@ -67,7 +67,7 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer private final AtomicLong counter = new AtomicLong(0); private final Set availableZNodes = new ConcurrentSkipListSet(); - private final Map segmentLookup = new ConcurrentHashMap<>(); + private final ConcurrentMap segmentLookup = new ConcurrentHashMap<>(); private final Function segmentTransformer; private final ChangeRequestHistory changes = new ChangeRequestHistory(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java b/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java index 0129197482a..2c41cd1ddd9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ReplicationThrottler.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordinator; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.timeline.SegmentId; @@ -28,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; /** * The ReplicationThrottler is used to throttle the number of replicants that are created. @@ -123,7 +123,7 @@ public class ReplicationThrottler public void removeSegment(String tier, SegmentId segmentId) { - Map segments = currentlyProcessingSegments.get(tier); + ConcurrentMap segments = currentlyProcessingSegments.get(tier); if (segments != null) { segments.remove(segmentId); } @@ -131,7 +131,7 @@ public class ReplicationThrottler public int getNumProcessing(String tier) { - Map segments = currentlyProcessingSegments.get(tier); + ConcurrentMap segments = currentlyProcessingSegments.get(tier); return (segments == null) ? 0 : segments.size(); } @@ -161,10 +161,10 @@ public class ReplicationThrottler public List getCurrentlyProcessingSegmentsAndHosts(String tier) { - Map segments = currentlyProcessingSegments.get(tier); - List retVal = new ArrayList<>(); - segments.forEach((segmentId, serverId) -> retVal.add(StringUtils.format("%s ON %s", segmentId, serverId))); - return retVal; + ConcurrentMap segments = currentlyProcessingSegments.get(tier); + List segmentsAndHosts = new ArrayList<>(); + segments.forEach((segmentId, serverId) -> segmentsAndHosts.add(segmentId + " ON " + serverId)); + return segmentsAndHosts; } } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java index 7e978e879de..60fc376f07f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java @@ -41,6 +41,7 @@ import java.util.Map; import java.util.NavigableSet; import java.util.SortedSet; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; /** @@ -210,7 +211,8 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper LoadPeonCallback callback = null; try { - Map movingSegments = currentlyMovingSegments.get(toServer.getTier()); + ConcurrentMap movingSegments = + currentlyMovingSegments.get(toServer.getTier()); movingSegments.put(segmentId, segment); callback = () -> movingSegments.remove(segmentId); coordinator.moveSegment( diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 93bda0ba632..05535053f83 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -363,17 +363,17 @@ public class DataSourcesResource return Response.ok(retVal).build(); } else { - final Map> retVal = new TreeMap<>(comparator); + final Map> statsPerInterval = new TreeMap<>(comparator); for (DataSegment dataSegment : dataSource.getSegments()) { if (intervalFilter.test(dataSegment.getInterval())) { Map properties = - retVal.computeIfAbsent(dataSegment.getInterval(), i -> new EnumMap<>(SimpleProperties.class)); + statsPerInterval.computeIfAbsent(dataSegment.getInterval(), i -> new EnumMap<>(SimpleProperties.class)); properties.merge(SimpleProperties.size, dataSegment.getSize(), (a, b) -> (Long) a + (Long) b); properties.merge(SimpleProperties.count, 1, (a, b) -> (Integer) a + (Integer) b); } } - return Response.ok(retVal).build(); + return Response.ok(statsPerInterval).build(); } } diff --git a/server/src/main/java/org/apache/druid/server/http/TiersResource.java b/server/src/main/java/org/apache/druid/server/http/TiersResource.java index 7698b19e37d..4debc83d17e 100644 --- a/server/src/main/java/org/apache/druid/server/http/TiersResource.java +++ b/server/src/main/java/org/apache/druid/server/http/TiersResource.java @@ -101,11 +101,11 @@ public class TiersResource public Response getTierDataSources(@PathParam("tierName") String tierName, @QueryParam("simple") String simple) { if (simple != null) { - Map>> retVal = new HashMap<>(); + Map>> tierToStatsPerInterval = new HashMap<>(); for (DruidServer druidServer : serverInventoryView.getInventory()) { if (druidServer.getTier().equalsIgnoreCase(tierName)) { for (DataSegment dataSegment : druidServer.getSegments()) { - Map properties = retVal + Map properties = tierToStatsPerInterval .computeIfAbsent(dataSegment.getDataSource(), dsName -> new HashMap<>()) .computeIfAbsent(dataSegment.getInterval(), interval -> new EnumMap<>(IntervalProperties.class)); properties.merge(IntervalProperties.size, dataSegment.getSize(), (a, b) -> (Long) a + (Long) b); @@ -114,7 +114,7 @@ public class TiersResource } } - return Response.ok(retVal).build(); + return Response.ok(tierToStatsPerInterval).build(); } Set retVal = serverInventoryView diff --git a/server/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java b/server/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java index c8c3b56a104..49472be5437 100644 --- a/server/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java +++ b/server/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java @@ -44,6 +44,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.atomic.AtomicReference; @@ -167,14 +168,16 @@ public class CoordinatorRuleManager public List getRulesWithDefault(final String dataSource) { - List retVal = new ArrayList<>(); - Map> theRules = rules.get(); - if (theRules.get(dataSource) != null) { - retVal.addAll(theRules.get(dataSource)); + List rulesWithDefault = new ArrayList<>(); + ConcurrentMap> theRules = rules.get(); + List dataSourceRules = theRules.get(dataSource); + if (dataSourceRules != null) { + rulesWithDefault.addAll(dataSourceRules); } - if (theRules.get(config.get().getDefaultRule()) != null) { - retVal.addAll(theRules.get(config.get().getDefaultRule())); + List defaultRules = theRules.get(config.get().getDefaultRule()); + if (defaultRules != null) { + rulesWithDefault.addAll(defaultRules); } - return retVal; + return rulesWithDefault; } } 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 4f043eeaede..e1b7a886d64 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -444,7 +444,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -483,7 +483,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -610,7 +610,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()) + new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()) ); testQueryCaching( @@ -671,7 +671,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -716,7 +716,7 @@ public class CachingClusteredClientTest .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()) + new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()) ); testQueryCaching( runner, @@ -792,7 +792,7 @@ public class CachingClusteredClientTest getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -869,7 +869,7 @@ public class CachingClusteredClientTest getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -974,7 +974,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); testQueryCaching( @@ -1047,7 +1047,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); testQueryCaching( @@ -1148,7 +1148,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new SearchQueryQueryToolChest( new SearchQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); HashMap context = new HashMap(); @@ -1218,7 +1218,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new SearchQueryQueryToolChest( new SearchQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); HashMap context = new HashMap(); @@ -1316,7 +1316,7 @@ public class CachingClusteredClientTest getDefaultQueryRunner(), new SelectQueryQueryToolChest( JSON_MAPPER, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), SELECT_CONFIG_SUPPLIER ) ); @@ -1394,7 +1394,7 @@ public class CachingClusteredClientTest getDefaultQueryRunner(), new SelectQueryQueryToolChest( JSON_MAPPER, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), SELECT_CONFIG_SUPPLIER ) ); @@ -1638,7 +1638,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -1712,7 +1712,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java index 5a283f89397..8de4223c3e6 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java @@ -71,28 +71,28 @@ public final class CachingClusteredClientTestUtils .put( TimeseriesQuery.class, new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ) .put( TopNQuery.class, new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ) .put( SearchQuery.class, new SearchQueryQueryToolChest( new SearchQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ) .put( SelectQuery.class, new SelectQueryQueryToolChest( objectMapper, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), selectConfigSupplier ) ) diff --git a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java index 530fd05f983..630321a0247 100644 --- a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java @@ -78,6 +78,7 @@ import java.util.HashMap; 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.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -141,7 +142,7 @@ public class CachingQueryRunnerTest QueryToolChest toolchest = new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); testCloseAndPopulate(expectedRes, expectedCacheRes, builder.build(), toolchest); @@ -189,7 +190,7 @@ public class CachingQueryRunnerTest } QueryToolChest toolChest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); testCloseAndPopulate(expectedResults, expectedResults, query, toolChest); @@ -227,7 +228,7 @@ public class CachingQueryRunnerTest final CountDownLatch cacheMustBePutOnce = new CountDownLatch(1); Cache cache = new Cache() { - private final Map baseMap = new ConcurrentHashMap<>(); + private final ConcurrentMap baseMap = new ConcurrentHashMap<>(); @Override public byte[] get(NamedKey key) @@ -254,7 +255,7 @@ public class CachingQueryRunnerTest } @Override - public void close() throws IOException + public void close() { } diff --git a/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java index ba5a87f006e..814dcdd63f4 100644 --- a/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java @@ -54,6 +54,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -68,11 +69,13 @@ public class DruidMeta extends MetaImpl private final AvaticaServerConfig config; private final List authenticators; - // Used to track logical connections. - private final Map connections = new ConcurrentHashMap<>(); + /** Used to track logical connections. */ + private final ConcurrentMap connections = new ConcurrentHashMap<>(); - // Number of connections reserved in "connections". May be higher than the actual number of connections at times, - // such as when we're reserving space to open a new one. + /** + * Number of connections reserved in "connections". May be higher than the actual number of connections at times, + * such as when we're reserving space to open a new one. + */ private final AtomicInteger connectionCount = new AtomicInteger(); @Inject 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 5beafb7bf23..9ca3233f20d 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 @@ -517,7 +517,7 @@ public class CalciteTests new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( TestHelper.makeJsonMapper(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), SELECT_CONFIG_SUPPLIER ), new SelectQueryEngine(), @@ -527,9 +527,7 @@ public class CalciteTests .put( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -540,7 +538,7 @@ public class CalciteTests stupidPool, new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) From 7e48593b5780c3bc4852d183dc473be283c84d25 Mon Sep 17 00:00:00 2001 From: David Glasser Date: Mon, 4 Feb 2019 12:00:26 -0800 Subject: [PATCH 08/13] ParallelIndexSupervisorTask: don't warn about a default value (#6987) Native batch indexing doesn't yet support the maxParseExceptions, maxSavedParseExceptions, and logParseExceptions tuning config options, so ParallelIndexSupervisorTask logs if these are set. But the default value for maxParseExceptions is Integer.MAX_VALUE, which means that you'll get the maxParseExceptions flavor of this warning even if you don't configure maxParseExceptions. This PR changes all three warnings to occur if you change the settings from the default; this mostly affects the maxParseExceptions warning. --- .../task/batch/parallel/ParallelIndexSupervisorTask.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 b8a23097688..385797f751a 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 @@ -48,6 +48,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRun import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -140,13 +141,14 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan this.authorizerMapper = authorizerMapper; this.rowIngestionMetersFactory = rowIngestionMetersFactory; - if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { + if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() + != TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS) { log.warn("maxSavedParseExceptions is not supported yet"); } - if (ingestionSchema.getTuningConfig().getMaxParseExceptions() > 0) { + if (ingestionSchema.getTuningConfig().getMaxParseExceptions() != TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS) { log.warn("maxParseExceptions is not supported yet"); } - if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { + if (ingestionSchema.getTuningConfig().isLogParseExceptions() != TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS) { log.warn("logParseExceptions is not supported yet"); } } From 97b6407983f597fc039bb90339b41086bbaaea56 Mon Sep 17 00:00:00 2001 From: Egor Riashin Date: Tue, 5 Feb 2019 05:11:00 +0300 Subject: [PATCH 09/13] maintenance mode for Historical (#6349) * maintenance mode for Historical forbidden api fix, config deserialization fix logging fix, unit tests * addressed comments * addressed comments * a style fix * addressed comments * a unit-test fix due to recent code-refactoring * docs & refactoring * addressed comments * addressed a LoadRule drop flaw * post merge cleaning up --- docs/content/configuration/index.md | 6 +- server/pom.xml | 6 + .../coordinator/CoordinatorDynamicConfig.java | 144 +++++++-- .../server/coordinator/DruidCluster.java | 14 +- .../server/coordinator/DruidCoordinator.java | 9 +- ...ruidCoordinatorCleanupPendingSegments.java | 2 +- .../DruidCoordinatorRuntimeParams.java | 13 +- .../server/coordinator/ServerHolder.java | 18 ++ .../helper/DruidCoordinatorBalancer.java | 95 ++++-- .../helper/DruidCoordinatorSegmentKiller.java | 2 +- .../rules/BroadcastDistributionRule.java | 5 +- .../server/coordinator/rules/LoadRule.java | 42 ++- .../CoordinatorDynamicConfigsResource.java | 31 +- .../DruidCoordinatorBalancerProfiler.java | 7 +- .../DruidCoordinatorBalancerTest.java | 259 +++++++++++++++- .../DruidCoordinatorBalancerTester.java | 14 +- .../DruidCoordinatorRuleRunnerTest.java | 6 +- .../rules/BroadcastDistributionRuleTest.java | 113 ++++++- .../coordinator/rules/LoadRuleTest.java | 287 +++++++++++++++++- .../http/CoordinatorDynamicConfigTest.java | 92 ++++-- 20 files changed, 1012 insertions(+), 153 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 221cccdc2b9..c63eb41994e 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -779,7 +779,9 @@ A sample Coordinator dynamic config JSON object is shown below: "replicantLifetime": 15, "replicationThrottleLimit": 10, "emitBalancingStats": false, - "killDataSourceWhitelist": ["wikipedia", "testDatasource"] + "killDataSourceWhitelist": ["wikipedia", "testDatasource"], + "historicalNodesInMaintenance": ["localhost:8182", "localhost:8282"], + "nodesInMaintenancePriority": 7 } ``` @@ -799,6 +801,8 @@ Issuing a GET request at the same URL will return the spec that is currently in |`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false| |`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none| |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0| +|`historicalNodesInMaintenance`| List of Historical nodes in maintenance mode. Coordinator doesn't assign new segments on those nodes and moves segments from the nodes according to a specified priority.|none| +|`nodesInMaintenancePriority`| Priority of segments from servers in maintenance. Coordinator takes ceil(maxSegmentsToMove * (priority / 10)) from servers in maitenance during balancing phase, i.e.:
0 - no segments from servers in maintenance will be processed during balancing
5 - 50% segments from servers in maintenance
10 - 100% segments from servers in maintenance
By leveraging the priority an operator can prevent general nodes from overload or decrease maitenance time instead.|7| To view the audit history of Coordinator dynamic config issue a GET request to the URL - diff --git a/server/pom.xml b/server/pom.xml index b28013b614d..ab2ca24ec32 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -191,6 +191,12 @@ junit test + + org.hamcrest + hamcrest-all + 1.3 + test + com.carrotsearch junit-benchmarks diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index c51a04a8dea..de034af5dfc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -55,10 +55,12 @@ public class CoordinatorDynamicConfig private final int balancerComputeThreads; private final boolean emitBalancingStats; private final boolean killAllDataSources; - private final Set killDataSourceWhitelist; + private final Set killableDataSources; + private final Set historicalNodesInMaintenance; + private final int nodesInMaintenancePriority; // The pending segments of the dataSources in this list are not killed. - private final Set killPendingSegmentsSkipList; + private final Set protectedPendingSegmentDatasources; /** * The maximum number of segments that could be queued for loading to any given server. @@ -82,10 +84,12 @@ public class CoordinatorDynamicConfig // Type is Object here so that we can support both string and list as // coordinator console can not send array of strings in the update request. // See https://github.com/apache/incubator-druid/issues/3055 - @JsonProperty("killDataSourceWhitelist") Object killDataSourceWhitelist, + @JsonProperty("killDataSourceWhitelist") Object killableDataSources, @JsonProperty("killAllDataSources") boolean killAllDataSources, - @JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList, - @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue + @JsonProperty("killPendingSegmentsSkipList") Object protectedPendingSegmentDatasources, + @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue, + @JsonProperty("historicalNodesInMaintenance") Object historicalNodesInMaintenance, + @JsonProperty("nodesInMaintenancePriority") int nodesInMaintenancePriority ) { this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; @@ -97,11 +101,17 @@ public class CoordinatorDynamicConfig this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.emitBalancingStats = emitBalancingStats; this.killAllDataSources = killAllDataSources; - this.killDataSourceWhitelist = parseJsonStringOrArray(killDataSourceWhitelist); - this.killPendingSegmentsSkipList = parseJsonStringOrArray(killPendingSegmentsSkipList); + this.killableDataSources = parseJsonStringOrArray(killableDataSources); + this.protectedPendingSegmentDatasources = parseJsonStringOrArray(protectedPendingSegmentDatasources); this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; + this.historicalNodesInMaintenance = parseJsonStringOrArray(historicalNodesInMaintenance); + Preconditions.checkArgument( + nodesInMaintenancePriority >= 0 && nodesInMaintenancePriority <= 10, + "nodesInMaintenancePriority should be in range [0, 10]" + ); + this.nodesInMaintenancePriority = nodesInMaintenancePriority; - if (this.killAllDataSources && !this.killDataSourceWhitelist.isEmpty()) { + if (this.killAllDataSources && !this.killableDataSources.isEmpty()) { throw new IAE("can't have killAllDataSources and non-empty killDataSourceWhitelist"); } } @@ -188,10 +198,14 @@ public class CoordinatorDynamicConfig return balancerComputeThreads; } - @JsonProperty - public Set getKillDataSourceWhitelist() + /** + * List of dataSources for which kill tasks are sent in + * {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}. + */ + @JsonProperty("killDataSourceWhitelist") + public Set getKillableDataSources() { - return killDataSourceWhitelist; + return killableDataSources; } @JsonProperty @@ -200,10 +214,14 @@ public class CoordinatorDynamicConfig return killAllDataSources; } + /** + * List of dataSources for which pendingSegments are NOT cleaned up + * in {@link DruidCoordinatorCleanupPendingSegments}. + */ @JsonProperty - public Set getKillPendingSegmentsSkipList() + public Set getProtectedPendingSegmentDatasources() { - return killPendingSegmentsSkipList; + return protectedPendingSegmentDatasources; } @JsonProperty @@ -212,6 +230,35 @@ public class CoordinatorDynamicConfig return maxSegmentsInNodeLoadingQueue; } + /** + * Historical nodes list in maintenance mode. Coordinator doesn't assign new segments on those nodes and moves + * segments from those nodes according to a specified priority. + * + * @return list of host:port entries + */ + @JsonProperty + public Set getHistoricalNodesInMaintenance() + { + return historicalNodesInMaintenance; + } + + /** + * Priority of segments from servers in maintenance. Coordinator takes ceil(maxSegmentsToMove * (priority / 10)) + * from servers in maitenance during balancing phase, i.e.: + * 0 - no segments from servers in maintenance will be processed during balancing + * 5 - 50% segments from servers in maintenance + * 10 - 100% segments from servers in maintenance + * By leveraging the priority an operator can prevent general nodes from overload or decrease maitenance time + * instead. + * + * @return number in range [0, 10] + */ + @JsonProperty + public int getNodesInMaintenancePriority() + { + return nodesInMaintenancePriority; + } + @Override public String toString() { @@ -224,10 +271,12 @@ public class CoordinatorDynamicConfig ", replicationThrottleLimit=" + replicationThrottleLimit + ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + - ", killDataSourceWhitelist=" + killDataSourceWhitelist + ", killAllDataSources=" + killAllDataSources + - ", killPendingSegmentsSkipList=" + killPendingSegmentsSkipList + + ", killDataSourceWhitelist=" + killableDataSources + + ", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + + ", historicalNodesInMaintenance=" + historicalNodesInMaintenance + + ", nodesInMaintenancePriority=" + nodesInMaintenancePriority + '}'; } @@ -273,10 +322,16 @@ public class CoordinatorDynamicConfig if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) { return false; } - if (!Objects.equals(killDataSourceWhitelist, that.killDataSourceWhitelist)) { + if (!Objects.equals(killableDataSources, that.killableDataSources)) { return false; } - return Objects.equals(killPendingSegmentsSkipList, that.killPendingSegmentsSkipList); + if (!Objects.equals(protectedPendingSegmentDatasources, that.protectedPendingSegmentDatasources)) { + return false; + } + if (!Objects.equals(historicalNodesInMaintenance, that.historicalNodesInMaintenance)) { + return false; + } + return nodesInMaintenancePriority == that.nodesInMaintenancePriority; } @Override @@ -293,8 +348,10 @@ public class CoordinatorDynamicConfig emitBalancingStats, killAllDataSources, maxSegmentsInNodeLoadingQueue, - killDataSourceWhitelist, - killPendingSegmentsSkipList + killableDataSources, + protectedPendingSegmentDatasources, + historicalNodesInMaintenance, + nodesInMaintenancePriority ); } @@ -315,6 +372,7 @@ public class CoordinatorDynamicConfig private static final boolean DEFAULT_EMIT_BALANCING_STATS = false; private static final boolean DEFAULT_KILL_ALL_DATA_SOURCES = false; private static final int DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 0; + private static final int DEFAULT_MAINTENANCE_MODE_SEGMENTS_PRIORITY = 7; private Long millisToWaitBeforeDeleting; private Long mergeBytesLimit; @@ -324,10 +382,12 @@ public class CoordinatorDynamicConfig private Integer replicationThrottleLimit; private Boolean emitBalancingStats; private Integer balancerComputeThreads; - private Object killDataSourceWhitelist; + private Object killableDataSources; private Boolean killAllDataSources; private Object killPendingSegmentsSkipList; private Integer maxSegmentsInNodeLoadingQueue; + private Object maintenanceList; + private Integer maintenanceModeSegmentsPriority; public Builder() { @@ -343,10 +403,12 @@ public class CoordinatorDynamicConfig @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads, @JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats, - @JsonProperty("killDataSourceWhitelist") @Nullable Object killDataSourceWhitelist, + @JsonProperty("killDataSourceWhitelist") @Nullable Object killableDataSources, @JsonProperty("killAllDataSources") @Nullable Boolean killAllDataSources, @JsonProperty("killPendingSegmentsSkipList") @Nullable Object killPendingSegmentsSkipList, - @JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue + @JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue, + @JsonProperty("historicalNodesInMaintenance") @Nullable Object maintenanceList, + @JsonProperty("nodesInMaintenancePriority") @Nullable Integer maintenanceModeSegmentsPriority ) { this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; @@ -358,9 +420,11 @@ public class CoordinatorDynamicConfig this.balancerComputeThreads = balancerComputeThreads; this.emitBalancingStats = emitBalancingStats; this.killAllDataSources = killAllDataSources; - this.killDataSourceWhitelist = killDataSourceWhitelist; + this.killableDataSources = killableDataSources; this.killPendingSegmentsSkipList = killPendingSegmentsSkipList; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; + this.maintenanceList = maintenanceList; + this.maintenanceModeSegmentsPriority = maintenanceModeSegmentsPriority; } public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting) @@ -413,7 +477,7 @@ public class CoordinatorDynamicConfig public Builder withKillDataSourceWhitelist(Set killDataSourceWhitelist) { - this.killDataSourceWhitelist = killDataSourceWhitelist; + this.killableDataSources = killDataSourceWhitelist; return this; } @@ -429,6 +493,18 @@ public class CoordinatorDynamicConfig return this; } + public Builder withMaintenanceList(Set list) + { + this.maintenanceList = list; + return this; + } + + public Builder withMaintenanceModeSegmentsPriority(Integer priority) + { + this.maintenanceModeSegmentsPriority = priority; + return this; + } + public CoordinatorDynamicConfig build() { return new CoordinatorDynamicConfig( @@ -440,12 +516,16 @@ public class CoordinatorDynamicConfig replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit, balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, - killDataSourceWhitelist, + killableDataSources, killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources, killPendingSegmentsSkipList, maxSegmentsInNodeLoadingQueue == null ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE - : maxSegmentsInNodeLoadingQueue + : maxSegmentsInNodeLoadingQueue, + maintenanceList, + maintenanceModeSegmentsPriority == null + ? DEFAULT_MAINTENANCE_MODE_SEGMENTS_PRIORITY + : maintenanceModeSegmentsPriority ); } @@ -460,12 +540,18 @@ public class CoordinatorDynamicConfig replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit, balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, - killDataSourceWhitelist == null ? defaults.getKillDataSourceWhitelist() : killDataSourceWhitelist, + killableDataSources == null ? defaults.getKillableDataSources() : killableDataSources, killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources, - killPendingSegmentsSkipList == null ? defaults.getKillPendingSegmentsSkipList() : killPendingSegmentsSkipList, + killPendingSegmentsSkipList == null + ? defaults.getProtectedPendingSegmentDatasources() + : killPendingSegmentsSkipList, maxSegmentsInNodeLoadingQueue == null ? defaults.getMaxSegmentsInNodeLoadingQueue() - : maxSegmentsInNodeLoadingQueue + : maxSegmentsInNodeLoadingQueue, + maintenanceList == null ? defaults.getHistoricalNodesInMaintenance() : maintenanceList, + maintenanceModeSegmentsPriority == null + ? defaults.getNodesInMaintenancePriority() + : maintenanceModeSegmentsPriority ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java index c5ba2a53f83..586f92b2dd7 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java @@ -34,6 +34,8 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Set; import java.util.TreeSet; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** * Contains a representation of the current state of the cluster by tier. @@ -53,11 +55,19 @@ public class DruidCluster @VisibleForTesting public DruidCluster( @Nullable Set realtimes, - Map> historicals + Map> historicals ) { this.realtimes = realtimes == null ? new HashSet<>() : new HashSet<>(realtimes); - this.historicals = historicals; + this.historicals = historicals + .entrySet() + .stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + e -> StreamSupport + .stream(e.getValue().spliterator(), false) + .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + )); } public void add(ServerHolder serverHolder) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 1ac6187eba2..b92effc8651 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -694,6 +694,7 @@ public class DruidCoordinator } // Find all historical servers, group them by subType and sort by ascending usage + Set nodesInMaintenance = params.getCoordinatorDynamicConfig().getHistoricalNodesInMaintenance(); final DruidCluster cluster = new DruidCluster(); for (ImmutableDruidServer server : servers) { if (!loadManagementPeons.containsKey(server.getName())) { @@ -704,7 +705,13 @@ public class DruidCoordinator loadManagementPeons.put(server.getName(), loadQueuePeon); } - cluster.add(new ServerHolder(server, loadManagementPeons.get(server.getName()))); + cluster.add( + new ServerHolder( + server, + loadManagementPeons.get(server.getName()), + nodesInMaintenance.contains(server.getHost()) + ) + ); } segmentReplicantLookup = SegmentReplicantLookup.make(cluster); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index 6ccc9338ac5..a25ea07c523 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -89,7 +89,7 @@ public class DruidCoordinatorCleanupPendingSegments implements DruidCoordinatorH // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET). final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); for (String dataSource : params.getDataSources().keySet()) { - if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource)) { + if (!params.getCoordinatorDynamicConfig().getProtectedPendingSegmentDatasources().contains(dataSource)) { log.info( "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", indexingServiceClient.killPendingSegments(dataSource, pendingSegmentsCleanupEndTime), diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index c99438cb300..08390385045 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator; +import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -27,6 +28,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.joda.time.DateTime; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -344,9 +346,16 @@ public class DruidCoordinatorRuntimeParams return this; } - public Builder withAvailableSegments(Collection availableSegmentsCollection) + @VisibleForTesting + public Builder withAvailableSegments(DataSegment... availableSegments) { - availableSegments.addAll(Collections.unmodifiableCollection(availableSegmentsCollection)); + this.availableSegments.addAll(Arrays.asList(availableSegments)); + return this; + } + + public Builder withAvailableSegments(Collection availableSegments) + { + this.availableSegments.addAll(Collections.unmodifiableCollection(availableSegments)); return this; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java index fd1370aa38e..c7d7a86c825 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java @@ -32,11 +32,18 @@ public class ServerHolder implements Comparable private static final Logger log = new Logger(ServerHolder.class); private final ImmutableDruidServer server; private final LoadQueuePeon peon; + private final boolean inMaintenance; public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon) + { + this(server, peon, false); + } + + public ServerHolder(ImmutableDruidServer server, LoadQueuePeon peon, boolean inMaintenance) { this.server = server; this.peon = peon; + this.inMaintenance = inMaintenance; } public ImmutableDruidServer getServer() @@ -74,6 +81,17 @@ public class ServerHolder implements Comparable return (100.0 * getSizeUsed()) / getMaxSize(); } + /** + * Historical nodes can be placed in maintenance mode, which instructs Coordinator to move segments from them + * according to a specified priority. The mechanism allows to drain segments from nodes which are planned for + * replacement. + * @return true if the node is in maitenance mode + */ + public boolean isInMaintenance() + { + return inMaintenance; + } + public long getAvailableSize() { long maxSize = getMaxSize(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java index 60fc376f07f..10499ac807a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java @@ -21,6 +21,7 @@ package org.apache.druid.server.coordinator.helper; import com.google.common.collect.Lists; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordinator.BalancerSegmentHolder; @@ -34,7 +35,6 @@ import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -48,8 +48,6 @@ import java.util.stream.Collectors; */ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper { - public static final Comparator percentUsedComparator = - Comparator.comparing(ServerHolder::getPercentUsed).reversed(); protected static final EmittingLogger log = new EmittingLogger(DruidCoordinatorBalancer.class); @@ -108,27 +106,73 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper return; } - final List toMoveFrom = Lists.newArrayList(servers); - final List toMoveTo = Lists.newArrayList(servers); + /* + Take as much segments from maintenance servers as priority allows and find the best location for them on + available servers. After that, balance segments within available servers pool. + */ + Map> partitions = + servers.stream().collect(Collectors.partitioningBy(ServerHolder::isInMaintenance)); + final List maintenanceServers = partitions.get(true); + final List availableServers = partitions.get(false); + log.info( + "Found %d servers in maintenance, %d available servers servers", + maintenanceServers.size(), + availableServers.size() + ); - if (toMoveTo.size() <= 1) { - log.info("[%s]: One or fewer servers found. Cannot balance.", tier); - return; + if (maintenanceServers.isEmpty()) { + if (availableServers.size() <= 1) { + log.info("[%s]: %d available servers servers found. Cannot balance.", tier, availableServers.size()); + } + } else if (availableServers.isEmpty()) { + log.info("[%s]: no available servers servers found during maintenance. Cannot balance.", tier); } int numSegments = 0; - for (ServerHolder sourceHolder : toMoveFrom) { + for (ServerHolder sourceHolder : servers) { numSegments += sourceHolder.getServer().getSegments().size(); } - if (numSegments == 0) { log.info("No segments found. Cannot balance."); return; } - final BalancerStrategy strategy = params.getBalancerStrategy(); final int maxSegmentsToMove = Math.min(params.getCoordinatorDynamicConfig().getMaxSegmentsToMove(), numSegments); + int priority = params.getCoordinatorDynamicConfig().getNodesInMaintenancePriority(); + int maxMaintenanceSegmentsToMove = (int) Math.ceil(maxSegmentsToMove * priority / 10.0); + log.info("Processing %d segments from servers in maintenance mode", maxMaintenanceSegmentsToMove); + Pair maintenanceResult = + balanceServers(params, maintenanceServers, availableServers, maxMaintenanceSegmentsToMove); + int maxGeneralSegmentsToMove = maxSegmentsToMove - maintenanceResult.lhs; + log.info("Processing %d segments from servers in general mode", maxGeneralSegmentsToMove); + Pair generalResult = + balanceServers(params, availableServers, availableServers, maxGeneralSegmentsToMove); + + int moved = generalResult.lhs + maintenanceResult.lhs; + int unmoved = generalResult.rhs + maintenanceResult.rhs; + if (unmoved == maxSegmentsToMove) { + // Cluster should be alive and constantly adjusting + log.info("No good moves found in tier [%s]", tier); + } + stats.addToTieredStat("unmovedCount", tier, unmoved); + stats.addToTieredStat("movedCount", tier, moved); + + if (params.getCoordinatorDynamicConfig().emitBalancingStats()) { + final BalancerStrategy strategy = params.getBalancerStrategy(); + strategy.emitStats(tier, stats, Lists.newArrayList(servers)); + } + log.info("[%s]: Segments Moved: [%d] Segments Let Alone: [%d]", tier, moved, unmoved); + } + + private Pair balanceServers( + DruidCoordinatorRuntimeParams params, + List toMoveFrom, + List toMoveTo, + int maxSegmentsToMove + ) + { + final BalancerStrategy strategy = params.getBalancerStrategy(); final int maxIterations = 2 * maxSegmentsToMove; final int maxToLoad = params.getCoordinatorDynamicConfig().getMaxSegmentsInNodeLoadingQueue(); int moved = 0, unmoved = 0; @@ -136,7 +180,6 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper //noinspection ForLoopThatDoesntUseLoopVariable for (int iter = 0; (moved + unmoved) < maxSegmentsToMove; ++iter) { final BalancerSegmentHolder segmentToMoveHolder = strategy.pickSegmentToMove(toMoveFrom); - if (segmentToMoveHolder != null && params.getAvailableSegments().contains(segmentToMoveHolder.getSegment())) { final DataSegment segmentToMove = segmentToMoveHolder.getSegment(); final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer(); @@ -154,8 +197,11 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper strategy.findNewSegmentHomeBalancer(segmentToMove, toMoveToWithLoadQueueCapacityAndNotServingSegment); if (destinationHolder != null && !destinationHolder.getServer().equals(fromServer)) { - moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params); - moved++; + if (moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params)) { + moved++; + } else { + unmoved++; + } } else { log.debug("Segment [%s] is 'optimally' placed.", segmentToMove.getId()); unmoved++; @@ -174,25 +220,10 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper break; } } - - if (unmoved == maxSegmentsToMove) { - // Cluster should be alive and constantly adjusting - log.info("No good moves found in tier [%s]", tier); - } - stats.addToTieredStat("unmovedCount", tier, unmoved); - stats.addToTieredStat("movedCount", tier, moved); - if (params.getCoordinatorDynamicConfig().emitBalancingStats()) { - strategy.emitStats(tier, stats, toMoveFrom); - } - log.info( - "[%s]: Segments Moved: [%d] Segments Let Alone: [%d]", - tier, - moved, - unmoved - ); + return new Pair<>(moved, unmoved); } - protected void moveSegment( + protected boolean moveSegment( final BalancerSegmentHolder segment, final ImmutableDruidServer toServer, final DruidCoordinatorRuntimeParams params @@ -221,6 +252,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper segmentToMove, callback ); + return true; } catch (Exception e) { log.makeAlert(e, StringUtils.format("[%s] : Moving exception", segmentId)).emit(); @@ -229,5 +261,6 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper } } } + return false; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java index b907e75ed01..3f719d3f3d4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java @@ -83,7 +83,7 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources(); - Collection whitelist = params.getCoordinatorDynamicConfig().getKillDataSourceWhitelist(); + Collection whitelist = params.getCoordinatorDynamicConfig().getKillableDataSources(); if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) { log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled."); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java index 2173593b1a4..b28f569e55b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java @@ -46,8 +46,9 @@ public abstract class BroadcastDistributionRule implements Rule } else { params.getDruidCluster().getAllServers().forEach( eachHolder -> { - if (colocatedDataSources.stream() - .anyMatch(source -> eachHolder.getServer().getDataSource(source) != null)) { + if (!eachHolder.isInMaintenance() + && colocatedDataSources.stream() + .anyMatch(source -> eachHolder.getServer().getDataSource(source) != null)) { loadServerHolders.add(eachHolder); } else if (eachHolder.isServingSegment(segment)) { if (!eachHolder.getPeon().getSegmentsToDrop().contains(segment)) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java index 41f61417878..3de93cfdaf9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java @@ -153,8 +153,8 @@ public abstract class LoadRule implements Rule log.makeAlert("Tier[%s] has no servers! Check your cluster configuration!", tier).emit(); return Collections.emptyList(); } - - return queue.stream().filter(predicate).collect(Collectors.toList()); + Predicate isNotInMaintenance = s -> !s.isInMaintenance(); + return queue.stream().filter(isNotInMaintenance.and(predicate)).collect(Collectors.toList()); } /** @@ -382,21 +382,38 @@ public abstract class LoadRule implements Rule final BalancerStrategy balancerStrategy ) { - int numDropped = 0; + Map> holders = holdersInTier.stream() + .filter(s -> s.isServingSegment(segment)) + .collect(Collectors.partitioningBy( + ServerHolder::isInMaintenance, + Collectors.toCollection(TreeSet::new) + )); + TreeSet maintenanceServers = holders.get(true); + TreeSet availableServers = holders.get(false); + int left = dropSegmentFromServers(balancerStrategy, segment, maintenanceServers, numToDrop); + if (left > 0) { + left = dropSegmentFromServers(balancerStrategy, segment, availableServers, left); + } + if (left != 0) { + log.warn("Wtf, holder was null? I have no servers serving [%s]?", segment.getId()); + } + return numToDrop - left; + } - final NavigableSet isServingSubset = - holdersInTier.stream().filter(s -> s.isServingSegment(segment)).collect(Collectors.toCollection(TreeSet::new)); + private static int dropSegmentFromServers( + BalancerStrategy balancerStrategy, + DataSegment segment, + NavigableSet holders, int numToDrop + ) + { + final Iterator iterator = balancerStrategy.pickServersToDrop(segment, holders); - final Iterator iterator = balancerStrategy.pickServersToDrop(segment, isServingSubset); - - while (numDropped < numToDrop) { + while (numToDrop > 0) { if (!iterator.hasNext()) { - log.warn("Wtf, holder was null? I have no servers serving [%s]?", segment.getId()); break; } final ServerHolder holder = iterator.next(); - if (holder.isServingSegment(segment)) { log.info( "Dropping segment [%s] on server [%s] in tier [%s]", @@ -405,7 +422,7 @@ public abstract class LoadRule implements Rule holder.getServer().getTier() ); holder.getPeon().dropSegment(segment, null); - ++numDropped; + numToDrop--; } else { log.warn( "Server [%s] is no longer serving segment [%s], skipping drop.", @@ -414,8 +431,7 @@ public abstract class LoadRule implements Rule ); } } - - return numDropped; + return numToDrop; } protected static void validateTieredReplicants(final Map tieredReplicants) diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java index 85bdd3320bc..488b0e70572 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java @@ -19,12 +19,12 @@ package org.apache.druid.server.http; -import com.google.common.collect.ImmutableMap; import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; import org.apache.druid.common.config.ConfigManager.SetResult; import org.apache.druid.common.config.JacksonConfigManager; +import org.apache.druid.common.utils.ServletResourceUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.http.security.ConfigResourceFilter; @@ -80,19 +80,26 @@ public class CoordinatorDynamicConfigsResource @Context HttpServletRequest req ) { - CoordinatorDynamicConfig current = CoordinatorDynamicConfig.current(manager); + try { + CoordinatorDynamicConfig current = CoordinatorDynamicConfig.current(manager); - final SetResult setResult = manager.set( - CoordinatorDynamicConfig.CONFIG_KEY, - dynamicConfigBuilder.build(current), - new AuditInfo(author, comment, req.getRemoteAddr()) - ); + final SetResult setResult = manager.set( + CoordinatorDynamicConfig.CONFIG_KEY, + dynamicConfigBuilder.build(current), + new AuditInfo(author, comment, req.getRemoteAddr()) + ); - if (setResult.isOk()) { - return Response.ok().build(); - } else { + if (setResult.isOk()) { + return Response.ok().build(); + } else { + return Response.status(Response.Status.BAD_REQUEST) + .entity(ServletResourceUtils.sanitizeException(setResult.getException())) + .build(); + } + } + catch (IllegalArgumentException e) { return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", setResult.getException())) + .entity(ServletResourceUtils.sanitizeException(e)) .build(); } } @@ -119,7 +126,7 @@ public class CoordinatorDynamicConfigsResource } catch (IllegalArgumentException e) { return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", e.getMessage())) + .entity(ServletResourceUtils.sanitizeException(e)) .build(); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index 8e325a30294..719a2b4553f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; import org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; import org.apache.druid.server.coordinator.rules.PeriodLoadRule; import org.apache.druid.server.coordinator.rules.Rule; @@ -139,7 +138,7 @@ public class DruidCoordinatorBalancerProfiler serverHolderList.stream().collect( Collectors.toCollection( () -> new TreeSet<>( - DruidCoordinatorBalancer.percentUsedComparator + DruidCoordinatorBalancerTester.percentUsedComparator ) ) ) @@ -170,7 +169,7 @@ public class DruidCoordinatorBalancerProfiler serverHolderList.stream().collect( Collectors.toCollection( () -> new TreeSet<>( - DruidCoordinatorBalancer.percentUsedComparator + DruidCoordinatorBalancerTester.percentUsedComparator ) ) ) @@ -232,7 +231,7 @@ public class DruidCoordinatorBalancerProfiler ).collect( Collectors.toCollection( () -> new TreeSet<>( - DruidCoordinatorBalancer.percentUsedComparator + DruidCoordinatorBalancerTester.percentUsedComparator ) ) ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java index 02c35408728..67521e397e0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -21,11 +21,12 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -37,15 +38,20 @@ import org.junit.Before; import org.junit.Test; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.TreeSet; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.replay; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + /** */ public class DruidCoordinatorBalancerTest @@ -193,6 +199,187 @@ public class DruidCoordinatorBalancerTest Assert.assertEquals(2, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); } + /** + * Server 1 has 2 segments. + * Server 2 (maintenance) has 2 segments. + * Server 3 is empty. + * Maintenance has priority 7. + * Max segments to move is 3. + * 2 (of 2) segments should be moved from Server 2 and 1 (of 2) from Server 1. + */ + @Test + public void testMoveMaintenancePriority() + { + mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment2)); + mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment3, segment4)); + mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); + + EasyMock.replay(druidServer4); + + mockCoordinator(coordinator); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, false)))) + .andReturn(new BalancerSegmentHolder(druidServer2, segment3)) + .andReturn(new BalancerSegmentHolder(druidServer2, segment4)); + EasyMock.expect(strategy.pickSegmentToMove(anyObject())) + .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) + .andReturn(new BalancerSegmentHolder(druidServer1, segment2)); + + EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject())) + .andReturn(new ServerHolder(druidServer3, peon3)) + .anyTimes(); + replay(strategy); + + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( + ImmutableList.of(druidServer1, druidServer2, druidServer3), + ImmutableList.of(peon1, peon2, peon3), + ImmutableList.of(false, true, false) + ) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder() + .withMaxSegmentsToMove(3) + .withMaintenanceModeSegmentsPriority(6) + .build() // ceil(3 * 0.6) = 2 segments from servers in maintenance + ) + .withBalancerStrategy(strategy) + .build(); + + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment1, segment3, segment4)))); + } + + @Test + public void testZeroMaintenancePriority() + { + DruidCoordinatorRuntimeParams params = setupParamsForMaintenancePriority(0); + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment1)))); + } + + @Test + public void testMaxMaintenancePriority() + { + DruidCoordinatorRuntimeParams params = setupParamsForMaintenancePriority(10); + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertEquals(1L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment2)))); + } + + /** + * Should balance segments as usual (ignoring priority) with empty maintenanceList. + */ + @Test + public void testMoveMaintenancePriorityWithNoMaintenance() + { + mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment2)); + mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Arrays.asList(segment3, segment4)); + mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); + + EasyMock.replay(druidServer4); + + mockCoordinator(coordinator); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + EasyMock.expect(strategy.pickSegmentToMove(anyObject())) + .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) + .andReturn(new BalancerSegmentHolder(druidServer1, segment2)) + .andReturn(new BalancerSegmentHolder(druidServer2, segment3)) + .andReturn(new BalancerSegmentHolder(druidServer2, segment4)); + + EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject())) + .andReturn(new ServerHolder(druidServer3, peon3)) + .anyTimes(); + replay(strategy); + + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( + ImmutableList.of(druidServer1, druidServer2, druidServer3), + ImmutableList.of(peon1, peon2, peon3), + ImmutableList.of(false, false, false) + ) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(3).withMaintenanceModeSegmentsPriority(9).build() + ) + .withBalancerStrategy(strategy) + .build(); + + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertEquals(3L, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + Assert.assertThat(peon3.getSegmentsToLoad(), is(equalTo(ImmutableSet.of(segment1, segment2, segment3)))); + } + + /** + * Shouldn't move segments to a server in maintenance mode. + */ + @Test + public void testMoveToServerInMaintenance() + { + mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); + mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); + + replay(druidServer3); + replay(druidServer4); + + mockCoordinator(coordinator); + + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + EasyMock.expect(strategy.pickSegmentToMove(anyObject())) + .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) + .anyTimes(); + EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject())).andAnswer(() -> { + List holders = (List) EasyMock.getCurrentArguments()[1]; + return holders.get(0); + }).anyTimes(); + replay(strategy); + + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( + ImmutableList.of(druidServer1, druidServer2), + ImmutableList.of(peon1, peon2), + ImmutableList.of(false, true) + ) + .withBalancerStrategy(strategy) + .build(); + + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertEquals(0, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + } + + @Test + public void testMoveFromServerInMaintenance() + { + mockDruidServer(druidServer1, "1", "normal", 30L, 100L, segments); + mockDruidServer(druidServer2, "2", "normal", 0L, 100L, Collections.emptyList()); + + replay(druidServer3); + replay(druidServer4); + + mockCoordinator(coordinator); + + ServerHolder holder2 = new ServerHolder(druidServer2, peon2, false); + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + EasyMock.expect(strategy.pickSegmentToMove(anyObject())) + .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) + .once(); + EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject())).andReturn(holder2).once(); + replay(strategy); + + DruidCoordinatorRuntimeParams params = defaultRuntimeParamsBuilder( + ImmutableList.of(druidServer1, druidServer2), + ImmutableList.of(peon1, peon2), + ImmutableList.of(true, false) + ) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build()) + .withBalancerStrategy(strategy) + .build(); + + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertEquals(1, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + Assert.assertEquals(0, peon1.getNumberOfSegmentsInQueue()); + Assert.assertEquals(1, peon2.getNumberOfSegmentsInQueue()); + } + @Test public void testMoveMaxLoadQueueServerBalancer() { @@ -314,6 +501,19 @@ public class DruidCoordinatorBalancerTest List druidServers, List peons ) + { + return defaultRuntimeParamsBuilder( + druidServers, + peons, + druidServers.stream().map(s -> false).collect(Collectors.toList()) + ); + } + + private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( + List druidServers, + List peons, + List maintenance + ) { return DruidCoordinatorRuntimeParams .newBuilder() @@ -324,12 +524,8 @@ public class DruidCoordinatorBalancerTest "normal", IntStream .range(0, druidServers.size()) - .mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i))) - .collect( - Collectors.toCollection( - () -> new TreeSet<>(DruidCoordinatorBalancer.percentUsedComparator) - ) - ) + .mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), maintenance.get(i))) + .collect(Collectors.toSet()) ) ) ) @@ -349,7 +545,7 @@ public class DruidCoordinatorBalancerTest .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")); } - private void mockDruidServer( + private static void mockDruidServer( ImmutableDruidServer druidServer, String name, String tier, @@ -363,17 +559,18 @@ public class DruidCoordinatorBalancerTest EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce(); EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce(); EasyMock.expect(druidServer.getSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes(); + EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes(); if (!segments.isEmpty()) { segments.forEach( s -> EasyMock.expect(druidServer.getSegment(s.getId())).andReturn(s).anyTimes() ); - } else { - EasyMock.expect(druidServer.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); } - EasyMock.replay(druidServer); + EasyMock.expect(druidServer.getSegment(anyObject())).andReturn(null).anyTimes(); + replay(druidServer); } - private void mockCoordinator(DruidCoordinator coordinator) + private static void mockCoordinator(DruidCoordinator coordinator) { coordinator.moveSegment( EasyMock.anyObject(), @@ -424,4 +621,40 @@ public class DruidCoordinatorBalancerTest delegate.emitStats(tier, stats, serverHolderList); } } + + private DruidCoordinatorRuntimeParams setupParamsForMaintenancePriority(int priority) + { + mockDruidServer(druidServer1, "1", "normal", 30L, 100L, Arrays.asList(segment1, segment3)); + mockDruidServer(druidServer2, "2", "normal", 30L, 100L, Arrays.asList(segment2, segment3)); + mockDruidServer(druidServer3, "3", "normal", 0L, 100L, Collections.emptyList()); + + EasyMock.replay(druidServer4); + + mockCoordinator(coordinator); + + // either maintenance servers list or general ones (ie servers list is [2] or [1, 3]) + BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); + EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, true)))) + .andReturn(new BalancerSegmentHolder(druidServer2, segment2)); + EasyMock.expect(strategy.pickSegmentToMove(anyObject())) + .andReturn(new BalancerSegmentHolder(druidServer1, segment1)); + EasyMock.expect(strategy.findNewSegmentHomeBalancer(anyObject(), anyObject())) + .andReturn(new ServerHolder(druidServer3, peon3)) + .anyTimes(); + replay(strategy); + + return defaultRuntimeParamsBuilder( + ImmutableList.of(druidServer1, druidServer2, druidServer3), + ImmutableList.of(peon1, peon2, peon3), + ImmutableList.of(false, true, false) + ) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder() + .withMaxSegmentsToMove(1) + .withMaintenanceModeSegmentsPriority(priority) + .build() + ) + .withBalancerStrategy(strategy) + .build(); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java index 3a96d4b2e7f..fad85d0effc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java @@ -25,15 +25,25 @@ import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import java.util.Comparator; + public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer { + public static final Comparator percentUsedComparator = (ServerHolder a, ServerHolder b) -> { + int c = Double.compare(a.getPercentUsed(), b.getPercentUsed()); + if (c == 0) { + return a.getServer().getName().compareTo(b.getServer().getName()); + } + return c; + }; + public DruidCoordinatorBalancerTester(DruidCoordinator coordinator) { super(coordinator); } @Override - protected void moveSegment( + protected boolean moveSegment( final BalancerSegmentHolder segment, final ImmutableDruidServer toServer, final DruidCoordinatorRuntimeParams params @@ -64,10 +74,12 @@ public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer dropPeon.markSegmentToDrop(segment.getSegment()); currentlyMovingSegments.get("normal").put(segmentId, segment); + return true; } catch (Exception e) { log.info(e, StringUtils.format("[%s] : Moving exception", segmentId)); } } + return false; } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 0996058b4ad..80dfd82652a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -946,9 +946,9 @@ public class DruidCoordinatorRuleRunnerTest ImmutableMap.of( "normal", Stream.of( - new ServerHolder(server1.toImmutableDruidServer(), mockPeon), - new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon), - new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false), + new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false), + new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false) ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 045b160ebe8..359fb6895cc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -21,7 +21,6 @@ package org.apache.druid.server.coordinator.rules; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -58,6 +57,10 @@ public class BroadcastDistributionRuleTest private final List largeSegments = new ArrayList<>(); private final List largeSegments2 = new ArrayList<>(); private DataSegment smallSegment; + private DruidCluster secondCluster; + private ServerHolder generalServer; + private ServerHolder maintenanceServer2; + private ServerHolder maintenanceServer1; @Before public void setUp() @@ -197,6 +200,50 @@ public class BroadcastDistributionRuleTest ) ); + generalServer = new ServerHolder( + new DruidServer( + "general", + "host1", + null, + 100, + ServerType.HISTORICAL, + "tier1", + 0 + ).addDataSegment(largeSegments.get(0)) + .toImmutableDruidServer(), + new LoadQueuePeonTester() + ); + + maintenanceServer1 = new ServerHolder( + new DruidServer( + "maintenance1", + "host2", + null, + 100, + ServerType.HISTORICAL, + "tier1", + 0 + ).addDataSegment(smallSegment) + .toImmutableDruidServer(), + new LoadQueuePeonTester(), + true + ); + + maintenanceServer2 = new ServerHolder( + new DruidServer( + "maintenance2", + "host3", + null, + 100, + ServerType.HISTORICAL, + "tier1", + 0 + ).addDataSegment(largeSegments.get(1)) + .toImmutableDruidServer(), + new LoadQueuePeonTester(), + true + ); + druidCluster = new DruidCluster( null, ImmutableMap.of( @@ -214,6 +261,18 @@ public class BroadcastDistributionRuleTest ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) ) ); + + secondCluster = new DruidCluster( + null, + ImmutableMap.of( + "tier1", + Stream.of( + generalServer, + maintenanceServer1, + maintenanceServer2 + ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + ) + ); } @Test @@ -227,14 +286,14 @@ public class BroadcastDistributionRuleTest .withDruidCluster(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Lists.newArrayList( + .withAvailableSegments( smallSegment, largeSegments.get(0), largeSegments.get(1), largeSegments.get(2), largeSegments2.get(0), largeSegments2.get(1) - )).build(), + ).build(), smallSegment ); @@ -254,6 +313,46 @@ public class BroadcastDistributionRuleTest assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment)); } + /** + * Servers: + * name | segments + * -------------+-------------- + * general | large segment + * maintenance1 | small segment + * maintenance2 | large segment + * + * After running the rule for the small segment: + * general | large & small segments + * maintenance1 | + * maintenance2 | large segment + */ + @Test + public void testBroadcastWithMaintenance() + { + final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + + CoordinatorStats stats = rule.run( + null, + DruidCoordinatorRuntimeParams.newBuilder() + .withDruidCluster(secondCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(secondCluster)) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) + .withAvailableSegments( + smallSegment, + largeSegments.get(0), + largeSegments.get(1) + ).build(), + smallSegment + ); + + assertEquals(1L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + assertEquals(false, stats.hasPerTierStats()); + + assertEquals(1, generalServer.getPeon().getSegmentsToLoad().size()); + assertEquals(1, maintenanceServer1.getPeon().getSegmentsToDrop().size()); + assertEquals(0, maintenanceServer2.getPeon().getSegmentsToLoad().size()); + } + @Test public void testBroadcastToMultipleDataSources() { @@ -267,14 +366,14 @@ public class BroadcastDistributionRuleTest .withDruidCluster(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Lists.newArrayList( + .withAvailableSegments( smallSegment, largeSegments.get(0), largeSegments.get(1), largeSegments.get(2), largeSegments2.get(0), largeSegments2.get(1) - )).build(), + ).build(), smallSegment ); @@ -305,14 +404,14 @@ public class BroadcastDistributionRuleTest .withDruidCluster(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Lists.newArrayList( + .withAvailableSegments( smallSegment, largeSegments.get(0), largeSegments.get(1), largeSegments.get(2), largeSegments2.get(0), largeSegments2.get(1) - )).build(), + ).build(), smallSegment ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 262b635631f..9a9bcb170a7 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -65,6 +65,7 @@ import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -181,7 +182,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Collections.singletonList(segment)).build(), + .withAvailableSegments(segment).build(), segment ); @@ -252,7 +253,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Collections.singletonList(segment)).build(), + .withAvailableSegments(segment).build(), segment ); @@ -302,7 +303,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Collections.singletonList(segment)).build(), + .withAvailableSegments(segment).build(), segment ); @@ -392,7 +393,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Collections.singletonList(segment)).build(), + .withAvailableSegments(segment).build(), segment ); @@ -410,7 +411,7 @@ public class LoadRuleTest EasyMock.expectLastCall().atLeastOnce(); EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) - .times(2); + .times(4); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); LoadRule rule = createLoadRule(ImmutableMap.of( @@ -481,7 +482,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Collections.singletonList(segment)).build(), + .withAvailableSegments(segment).build(), segment ); @@ -540,7 +541,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Collections.singletonList(segment)).build(), + .withAvailableSegments(segment).build(), segment ); @@ -557,7 +558,7 @@ public class LoadRuleTest EasyMock.expectLastCall().atLeastOnce(); EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) .andDelegateTo(balancerStrategy) - .times(1); + .times(2); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); LoadRule rule = createLoadRule(ImmutableMap.of( @@ -613,7 +614,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Collections.singletonList(segment)).build(), + .withAvailableSegments(segment).build(), segment ); @@ -670,7 +671,7 @@ public class LoadRuleTest .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegments(Arrays.asList(dataSegment1, dataSegment2, dataSegment3)) + .withAvailableSegments(dataSegment1, dataSegment2, dataSegment3) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build()) .build(); @@ -685,6 +686,239 @@ public class LoadRuleTest EasyMock.verify(throttler, mockBalancerStrategy); } + /** + * 2 servers in different tiers, the first is in maitenance mode. + * Should not load a segment to the server in maintenance mode. + */ + @Test + public void testLoadDuringMaitenance() + { + final LoadQueuePeon mockPeon1 = createEmptyPeon(); + final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); + + LoadRule rule = createLoadRule(ImmutableMap.of( + "tier1", 1, + "tier2", 1 + )); + + final DataSegment segment = createDataSegment("foo"); + + EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(EasyMock.anyObject(), EasyMock.anyObject())) + .andDelegateTo(balancerStrategy) + .times(1); + + EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy); + + + DruidCluster druidCluster = new DruidCluster( + null, + ImmutableMap.of( + "tier1", + Collections.singleton(createServerHolder("tier1", mockPeon1, true)), + "tier2", + Collections.singleton(createServerHolder("tier2", mockPeon2, false)) + ) + ); + + CoordinatorStats stats = rule.run( + null, + DruidCoordinatorRuntimeParams.newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) + .withAvailableSegments(segment).build(), + segment + ); + + Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); + EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy); + } + + /** + * 2 tiers, 2 servers each, 1 server of the second tier is in maintenance. + * Should not load a segment to the server in maintenance mode. + */ + @Test + public void testLoadReplicaDuringMaitenance() + { + EasyMock.expect(throttler.canCreateReplicant(EasyMock.anyString())).andReturn(true).anyTimes(); + + final LoadQueuePeon mockPeon1 = createEmptyPeon(); + final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); + final LoadQueuePeon mockPeon3 = createOneCallPeonMock(); + final LoadQueuePeon mockPeon4 = createOneCallPeonMock(); + + LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 2, "tier2", 2)); + + final DataSegment segment = createDataSegment("foo"); + + throttler.registerReplicantCreation(EasyMock.eq("tier2"), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().times(2); + + ServerHolder holder1 = createServerHolder("tier1", mockPeon1, true); + ServerHolder holder2 = createServerHolder("tier1", mockPeon2, false); + ServerHolder holder3 = createServerHolder("tier2", mockPeon3, false); + ServerHolder holder4 = createServerHolder("tier2", mockPeon4, false); + + EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder2))) + .andReturn(holder2); + EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder4, holder3))) + .andReturn(holder3); + EasyMock.expect(mockBalancerStrategy.findNewSegmentHomeReplicator(segment, ImmutableList.of(holder4))) + .andReturn(holder4); + + EasyMock.replay(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); + + + DruidCluster druidCluster = new DruidCluster( + null, + ImmutableMap.of("tier1", Arrays.asList(holder1, holder2), "tier2", Arrays.asList(holder3, holder4)) + ); + + CoordinatorStats stats = rule.run( + null, + DruidCoordinatorRuntimeParams.newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) + .withAvailableSegments(segment).build(), + segment + ); + + Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); + Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); + + EasyMock.verify(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); + } + + /** + * 2 servers with a segment, one server in maintenance mode. + * Should drop a segment from both. + */ + @Test + public void testDropDuringMaintenance() + { + final LoadQueuePeon mockPeon = createEmptyPeon(); + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().times(2); + EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) + .andDelegateTo(balancerStrategy) + .times(4); + EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); + + LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 0)); + + final DataSegment segment1 = createDataSegment("foo1"); + final DataSegment segment2 = createDataSegment("foo2"); + + DruidServer server1 = createServer("tier1"); + server1.addDataSegment(segment1); + DruidServer server2 = createServer("tier1"); + server2.addDataSegment(segment2); + + DruidCluster druidCluster = new DruidCluster( + null, + ImmutableMap.of( + "tier1", + Arrays.asList( + new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false) + ) + ) + ); + + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) + .withAvailableSegments(segment1, segment2) + .build(); + CoordinatorStats stats = rule.run( + null, + params, + segment1 + ); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); + stats = rule.run( + null, + params, + segment2 + ); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); + + + EasyMock.verify(throttler, mockPeon); + } + + /** + * 3 servers hosting 3 replicas of the segment. + * 1 servers is in maitenance. + * 1 replica is redundant. + * Should drop from the server in maintenance. + */ + @Test + public void testRedundantReplicaDropDuringMaintenance() + { + final LoadQueuePeon mockPeon1 = new LoadQueuePeonTester(); + final LoadQueuePeon mockPeon2 = new LoadQueuePeonTester(); + final LoadQueuePeon mockPeon3 = new LoadQueuePeonTester(); + EasyMock.expect(mockBalancerStrategy.pickServersToDrop(EasyMock.anyObject(), EasyMock.anyObject())) + .andDelegateTo(balancerStrategy) + .times(4); + EasyMock.replay(throttler, mockBalancerStrategy); + + LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 2)); + + final DataSegment segment1 = createDataSegment("foo1"); + + DruidServer server1 = createServer("tier1"); + server1.addDataSegment(segment1); + DruidServer server2 = createServer("tier1"); + server2.addDataSegment(segment1); + DruidServer server3 = createServer("tier1"); + server3.addDataSegment(segment1); + + DruidCluster druidCluster = new DruidCluster( + null, + ImmutableMap.of( + "tier1", + Arrays.asList( + new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true), + new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false) + ) + ) + ); + + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) + .withAvailableSegments(segment1) + .build(); + CoordinatorStats stats = rule.run( + null, + params, + segment1 + ); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); + Assert.assertEquals(0, mockPeon1.getSegmentsToDrop().size()); + Assert.assertEquals(1, mockPeon2.getSegmentsToDrop().size()); + Assert.assertEquals(0, mockPeon3.getSegmentsToDrop().size()); + + EasyMock.verify(throttler); + } + private DataSegment createDataSegment(String dataSource) { return new DataSegment( @@ -760,4 +994,37 @@ public class LoadRuleTest return mockPeon; } + + private static final AtomicInteger serverId = new AtomicInteger(); + + private static DruidServer createServer(String tier) + { + int serverId = LoadRuleTest.serverId.incrementAndGet(); + return new DruidServer( + "server" + serverId, + "127.0.0.1:800" + serverId, + null, + 1000, + ServerType.HISTORICAL, + tier, + 0 + ); + } + + private static LoadQueuePeon createOneCallPeonMock() + { + final LoadQueuePeon mockPeon2 = createEmptyPeon(); + mockPeon2.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().once(); + return mockPeon2; + } + + private static ServerHolder createServerHolder(String tier, LoadQueuePeon mockPeon1, boolean maintenance) + { + return new ServerHolder( + createServer(tier).toImmutableDruidServer(), + mockPeon1, + maintenance + ); + } } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index 002e1977c15..e0979252012 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -38,6 +38,44 @@ public class CoordinatorDynamicConfigTest @Test public void testSerde() throws Exception + { + String jsonStr = "{\n" + + " \"millisToWaitBeforeDeleting\": 1,\n" + + " \"mergeBytesLimit\": 1,\n" + + " \"mergeSegmentsLimit\" : 1,\n" + + " \"maxSegmentsToMove\": 1,\n" + + " \"replicantLifetime\": 1,\n" + + " \"replicationThrottleLimit\": 1,\n" + + " \"balancerComputeThreads\": 2, \n" + + " \"emitBalancingStats\": true,\n" + + " \"killDataSourceWhitelist\": [\"test1\",\"test2\"],\n" + + " \"maxSegmentsInNodeLoadingQueue\": 1,\n" + + " \"historicalNodesInMaintenance\": [\"host1\", \"host2\"],\n" + + " \"nodesInMaintenancePriority\": 9\n" + + "}\n"; + + CoordinatorDynamicConfig actual = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + CoordinatorDynamicConfig.class + ) + ), + CoordinatorDynamicConfig.class + ); + ImmutableSet maintenance = ImmutableSet.of("host1", "host2"); + ImmutableSet whitelist = ImmutableSet.of("test1", "test2"); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, maintenance, 9); + + actual = CoordinatorDynamicConfig.builder().withMaintenanceList(ImmutableSet.of("host1")).build(actual); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 9); + + actual = CoordinatorDynamicConfig.builder().withMaintenanceModeSegmentsPriority(5).build(actual); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 5); + } + + @Test + public void testMaintenanceParametersBackwardCompatibility() throws Exception { String jsonStr = "{\n" + " \"millisToWaitBeforeDeleting\": 1,\n" @@ -61,7 +99,15 @@ public class CoordinatorDynamicConfigTest ), CoordinatorDynamicConfig.class ); - assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of("test1", "test2"), false, 1); + ImmutableSet maintenance = ImmutableSet.of(); + ImmutableSet whitelist = ImmutableSet.of("test1", "test2"); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, maintenance, 0); + + actual = CoordinatorDynamicConfig.builder().withMaintenanceList(ImmutableSet.of("host1")).build(actual); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 0); + + actual = CoordinatorDynamicConfig.builder().withMaintenanceModeSegmentsPriority(5).build(actual); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, whitelist, false, 1, ImmutableSet.of("host1"), 5); } @Test @@ -89,7 +135,7 @@ public class CoordinatorDynamicConfigTest ), CoordinatorDynamicConfig.class ); - assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of("test1", "test2"), false, 1); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of("test1", "test2"), false, 1, ImmutableSet.of(), 0); } @Test @@ -118,7 +164,7 @@ public class CoordinatorDynamicConfigTest CoordinatorDynamicConfig.class ); - assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 1); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 1, ImmutableSet.of(), 0); //ensure whitelist is empty when killAllDataSources is true try { @@ -163,15 +209,15 @@ public class CoordinatorDynamicConfigTest CoordinatorDynamicConfig.class ); - assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 0); + assertConfig(actual, 1, 1, 1, 1, 1, 1, 2, true, ImmutableSet.of(), true, 0, ImmutableSet.of(), 0); } @Test public void testBuilderDefaults() { - CoordinatorDynamicConfig defaultConfig = CoordinatorDynamicConfig.builder().build(); - assertConfig(defaultConfig, 900000, 524288000, 100, 5, 15, 10, 1, false, ImmutableSet.of(), false, 0); + ImmutableSet emptyList = ImmutableSet.of(); + assertConfig(defaultConfig, 900000, 524288000, 100, 5, 15, 10, 1, false, emptyList, false, 0, emptyList, 7); } @Test @@ -184,7 +230,7 @@ public class CoordinatorDynamicConfigTest Assert.assertEquals( current, new CoordinatorDynamicConfig - .Builder(null, null, null, null, null, null, null, null, null, null, null, null) + .Builder(null, null, null, null, null, null, null, null, null, null, null, null, null, null) .build(current) ); } @@ -198,18 +244,22 @@ public class CoordinatorDynamicConfigTest Assert.assertEquals(config1.hashCode(), config2.hashCode()); } - private void assertConfig(CoordinatorDynamicConfig config, - long expectedMillisToWaitBeforeDeleting, - long expectedMergeBytesLimit, - int expectedMergeSegmentsLimit, - int expectedMaxSegmentsToMove, - int expectedReplicantLifetime, - int expectedReplicationThrottleLimit, - int expectedBalancerComputeThreads, - boolean expectedEmitingBalancingStats, - Set expectedKillDataSourceWhitelist, - boolean expectedKillAllDataSources, - int expectedMaxSegmentsInNodeLoadingQueue) + private void assertConfig( + CoordinatorDynamicConfig config, + long expectedMillisToWaitBeforeDeleting, + long expectedMergeBytesLimit, + int expectedMergeSegmentsLimit, + int expectedMaxSegmentsToMove, + int expectedReplicantLifetime, + int expectedReplicationThrottleLimit, + int expectedBalancerComputeThreads, + boolean expectedEmitingBalancingStats, + Set expectedKillableDatasources, + boolean expectedKillAllDataSources, + int expectedMaxSegmentsInNodeLoadingQueue, + Set maintenanceList, + int maintenancePriority + ) { Assert.assertEquals(expectedMillisToWaitBeforeDeleting, config.getMillisToWaitBeforeDeleting()); Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit()); @@ -219,8 +269,10 @@ public class CoordinatorDynamicConfigTest Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit()); Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads()); Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats()); - Assert.assertEquals(expectedKillDataSourceWhitelist, config.getKillDataSourceWhitelist()); + Assert.assertEquals(expectedKillableDatasources, config.getKillableDataSources()); Assert.assertEquals(expectedKillAllDataSources, config.isKillAllDataSources()); Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue()); + Assert.assertEquals(maintenanceList, config.getHistoricalNodesInMaintenance()); + Assert.assertEquals(maintenancePriority, config.getNodesInMaintenancePriority()); } } From 8bc5eaa908b38af81d2148b64c6eecfca2bafef6 Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Mon, 4 Feb 2019 19:36:20 -0800 Subject: [PATCH 10/13] Set version to 0.14.0-incubating-SNAPSHOT (#7003) --- aws-common/pom.xml | 2 +- benchmarks/pom.xml | 2 +- core/pom.xml | 2 +- distribution/pom.xml | 2 +- examples/pom.xml | 2 +- extendedset/pom.xml | 2 +- extensions-contrib/ambari-metrics-emitter/pom.xml | 2 +- extensions-contrib/azure-extensions/pom.xml | 2 +- extensions-contrib/cassandra-storage/pom.xml | 2 +- extensions-contrib/cloudfiles-extensions/pom.xml | 2 +- extensions-contrib/distinctcount/pom.xml | 2 +- extensions-contrib/druid-rocketmq/pom.xml | 2 +- extensions-contrib/google-extensions/pom.xml | 2 +- extensions-contrib/graphite-emitter/pom.xml | 2 +- extensions-contrib/influx-extensions/pom.xml | 2 +- extensions-contrib/kafka-eight-simpleConsumer/pom.xml | 2 +- extensions-contrib/kafka-emitter/pom.xml | 2 +- extensions-contrib/materialized-view-maintenance/pom.xml | 2 +- extensions-contrib/materialized-view-selection/pom.xml | 2 +- extensions-contrib/opentsdb-emitter/pom.xml | 2 +- extensions-contrib/orc-extensions/pom.xml | 2 +- extensions-contrib/rabbitmq/pom.xml | 2 +- extensions-contrib/redis-cache/pom.xml | 2 +- extensions-contrib/sqlserver-metadata-storage/pom.xml | 2 +- extensions-contrib/statsd-emitter/pom.xml | 2 +- extensions-contrib/thrift-extensions/pom.xml | 2 +- extensions-contrib/time-min-max/pom.xml | 2 +- extensions-contrib/virtual-columns/pom.xml | 2 +- extensions-core/avro-extensions/pom.xml | 2 +- extensions-core/datasketches/pom.xml | 2 +- extensions-core/druid-basic-security/pom.xml | 2 +- extensions-core/druid-bloom-filter/pom.xml | 2 +- extensions-core/druid-kerberos/pom.xml | 2 +- extensions-core/hdfs-storage/pom.xml | 2 +- extensions-core/histogram/pom.xml | 2 +- extensions-core/kafka-eight/pom.xml | 2 +- extensions-core/kafka-extraction-namespace/pom.xml | 2 +- extensions-core/kafka-indexing-service/pom.xml | 2 +- extensions-core/kinesis-indexing-service/pom.xml | 2 +- extensions-core/lookups-cached-global/pom.xml | 2 +- extensions-core/lookups-cached-single/pom.xml | 2 +- extensions-core/mysql-metadata-storage/pom.xml | 2 +- extensions-core/parquet-extensions/pom.xml | 2 +- extensions-core/postgresql-metadata-storage/pom.xml | 2 +- extensions-core/protobuf-extensions/pom.xml | 2 +- extensions-core/s3-extensions/pom.xml | 2 +- extensions-core/simple-client-sslcontext/pom.xml | 2 +- extensions-core/stats/pom.xml | 2 +- hll/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- integration-tests/pom.xml | 2 +- pom.xml | 2 +- processing/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- sql/pom.xml | 2 +- web-console/pom.xml | 2 +- 58 files changed, 58 insertions(+), 58 deletions(-) diff --git a/aws-common/pom.xml b/aws-common/pom.xml index 941dc2847b0..2a65b70c42e 100644 --- a/aws-common/pom.xml +++ b/aws-common/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 9bd28e95782..685ffe05131 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -27,7 +27,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/core/pom.xml b/core/pom.xml index 48bb5f301c0..9dbe3de96ca 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -31,7 +31,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/distribution/pom.xml b/distribution/pom.xml index 96b256e9722..bf069aa38f9 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -31,7 +31,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 11ddb1b71d6..c9ba9106024 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/extendedset/pom.xml b/extendedset/pom.xml index 2528d657ee0..783528a4e75 100755 --- a/extendedset/pom.xml +++ b/extendedset/pom.xml @@ -32,7 +32,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml index ead4738e1d6..886a7630a26 100644 --- a/extensions-contrib/ambari-metrics-emitter/pom.xml +++ b/extensions-contrib/ambari-metrics-emitter/pom.xml @@ -25,7 +25,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/azure-extensions/pom.xml b/extensions-contrib/azure-extensions/pom.xml index 685bc4cb9b5..2980d5e6c1c 100644 --- a/extensions-contrib/azure-extensions/pom.xml +++ b/extensions-contrib/azure-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index 4a3347904ac..67c215f43c5 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index 307cd27ed79..f80ca1f9893 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml index 99b0aac8e98..c4aeacea7b7 100644 --- a/extensions-contrib/distinctcount/pom.xml +++ b/extensions-contrib/distinctcount/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/druid-rocketmq/pom.xml b/extensions-contrib/druid-rocketmq/pom.xml index bd22bab418e..0394abcb775 100644 --- a/extensions-contrib/druid-rocketmq/pom.xml +++ b/extensions-contrib/druid-rocketmq/pom.xml @@ -24,7 +24,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/google-extensions/pom.xml b/extensions-contrib/google-extensions/pom.xml index 508aa488095..f252f58fe1d 100644 --- a/extensions-contrib/google-extensions/pom.xml +++ b/extensions-contrib/google-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index 5c268d768ad..b48d6909aa6 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -25,7 +25,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/influx-extensions/pom.xml b/extensions-contrib/influx-extensions/pom.xml index e966b40da3d..916780dcc12 100644 --- a/extensions-contrib/influx-extensions/pom.xml +++ b/extensions-contrib/influx-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml index a92d751b3fc..4e4144632d6 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml +++ b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index fff9ab9b343..ef714b64097 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -25,7 +25,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/materialized-view-maintenance/pom.xml b/extensions-contrib/materialized-view-maintenance/pom.xml index 6df9063ae62..643d5005582 100644 --- a/extensions-contrib/materialized-view-maintenance/pom.xml +++ b/extensions-contrib/materialized-view-maintenance/pom.xml @@ -24,7 +24,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/materialized-view-selection/pom.xml b/extensions-contrib/materialized-view-selection/pom.xml index 5016b17c75b..341ce9221fa 100644 --- a/extensions-contrib/materialized-view-selection/pom.xml +++ b/extensions-contrib/materialized-view-selection/pom.xml @@ -24,7 +24,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/opentsdb-emitter/pom.xml b/extensions-contrib/opentsdb-emitter/pom.xml index ab368635f87..c89c4ceb14f 100644 --- a/extensions-contrib/opentsdb-emitter/pom.xml +++ b/extensions-contrib/opentsdb-emitter/pom.xml @@ -31,7 +31,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml index 51580372f68..18ddf7827b8 100644 --- a/extensions-contrib/orc-extensions/pom.xml +++ b/extensions-contrib/orc-extensions/pom.xml @@ -28,7 +28,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/rabbitmq/pom.xml b/extensions-contrib/rabbitmq/pom.xml index ee3aea32eeb..7b9d375fd4a 100644 --- a/extensions-contrib/rabbitmq/pom.xml +++ b/extensions-contrib/rabbitmq/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml index cf8c8c74b31..cd06e05ccaf 100644 --- a/extensions-contrib/redis-cache/pom.xml +++ b/extensions-contrib/redis-cache/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index a3d0281adf6..b886f264e70 100644 --- a/extensions-contrib/sqlserver-metadata-storage/pom.xml +++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml index e343795fa27..b55b287bec7 100644 --- a/extensions-contrib/statsd-emitter/pom.xml +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -23,7 +23,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index ec0da850cad..95137f9a5ac 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -30,7 +30,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml index 9b7aa5e273e..d34ba373f49 100644 --- a/extensions-contrib/time-min-max/pom.xml +++ b/extensions-contrib/time-min-max/pom.xml @@ -23,7 +23,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml index 06ea0b412fa..0ec600db7c4 100644 --- a/extensions-contrib/virtual-columns/pom.xml +++ b/extensions-contrib/virtual-columns/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index 2e9f69e8036..c675dad3aae 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index 47017673f75..489226224b0 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-basic-security/pom.xml b/extensions-core/druid-basic-security/pom.xml index 343decd6936..ea10f325db2 100644 --- a/extensions-core/druid-basic-security/pom.xml +++ b/extensions-core/druid-basic-security/pom.xml @@ -32,7 +32,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-bloom-filter/pom.xml b/extensions-core/druid-bloom-filter/pom.xml index c45875db8ca..f9ac9f90c07 100644 --- a/extensions-core/druid-bloom-filter/pom.xml +++ b/extensions-core/druid-bloom-filter/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index ad27a22528f..b6fad3510f5 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index e0795e1b050..c60787717db 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index bd689c79ed4..dcd706ad5b7 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-eight/pom.xml b/extensions-core/kafka-eight/pom.xml index 2ea7a80c648..0c11346668c 100644 --- a/extensions-core/kafka-eight/pom.xml +++ b/extensions-core/kafka-eight/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 24a06cafecb..60fa27fd46f 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index 37de81ffa5b..5004bd5bad4 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 8469334641a..a7e576ec435 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index 5a2b4be1680..8866e3b062c 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml index 706a899010c..a3a1294c4be 100644 --- a/extensions-core/lookups-cached-single/pom.xml +++ b/extensions-core/lookups-cached-single/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml index bff092d0dd0..37b0617e01d 100644 --- a/extensions-core/mysql-metadata-storage/pom.xml +++ b/extensions-core/mysql-metadata-storage/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/parquet-extensions/pom.xml b/extensions-core/parquet-extensions/pom.xml index bfdcf267898..586b66d5dd4 100644 --- a/extensions-core/parquet-extensions/pom.xml +++ b/extensions-core/parquet-extensions/pom.xml @@ -29,7 +29,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml index e828cfb3686..a6bed8c96a4 100644 --- a/extensions-core/postgresql-metadata-storage/pom.xml +++ b/extensions-core/postgresql-metadata-storage/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index fcb28207438..c004982b715 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -31,7 +31,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index c61874adaf2..8d05b309dd3 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/extensions-core/simple-client-sslcontext/pom.xml b/extensions-core/simple-client-sslcontext/pom.xml index 97d9c5b1d0a..b6286c237c5 100644 --- a/extensions-core/simple-client-sslcontext/pom.xml +++ b/extensions-core/simple-client-sslcontext/pom.xml @@ -24,7 +24,7 @@ druid org.apache.druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index ea5c591cd51..704f2543462 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT ../../pom.xml diff --git a/hll/pom.xml b/hll/pom.xml index 1fe71b1604d..ebdfe8de677 100644 --- a/hll/pom.xml +++ b/hll/pom.xml @@ -24,7 +24,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT druid-hll diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 92cd1e778fd..1fd923c99bf 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4aec189a636..f664dd47123 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 49a61583d15..9c89d38b50a 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/pom.xml b/pom.xml index 114eb59bb0e..4e2f85e173d 100644 --- a/pom.xml +++ b/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT pom Druid diff --git a/processing/pom.xml b/processing/pom.xml index 714a1f2a025..ccaf9c53429 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ab2ca24ec32..9bdafbb7580 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 8a18dcb311d..1220492fdd1 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/sql/pom.xml b/sql/pom.xml index 162eb23f241..57c0d5f0487 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT diff --git a/web-console/pom.xml b/web-console/pom.xml index d49c1112a5b..936830d98be 100644 --- a/web-console/pom.xml +++ b/web-console/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.14.0-incubating-SNAPSHOT From 75c70c2ccc1feb064be63708557316bc4e1585ae Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 5 Feb 2019 11:52:09 -0800 Subject: [PATCH 11/13] Add doc for S3 permissions settings (#7011) * Add doc for S3 permissions settings * add a comment about additional settings --- docs/content/configuration/index.md | 3 ++- docs/content/development/extensions-core/s3.md | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index c63eb41994e..639bb45cce2 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -536,6 +536,7 @@ This deep storage doesn't do anything. There are no configs. #### S3 Deep Storage This deep storage is used to interface with Amazon's S3. Note that the `druid-s3-extensions` extension must be loaded. +The below table shows some important configurations for S3. See [S3 Deep Storage](../development/extensions-core/s3.html) for full configurations. |Property|Description|Default| |--------|-----------|-------| @@ -543,7 +544,7 @@ This deep storage is used to interface with Amazon's S3. Note that the `druid-s3 |`druid.s3.secretKey`|The secret key to use to access S3.|none| |`druid.storage.bucket`|S3 bucket name.|none| |`druid.storage.baseKey`|S3 object key prefix for storage.|none| -|`druid.storage.disableAcl`|Boolean flag for ACL.|false| +|`druid.storage.disableAcl`|Boolean flag for ACL. If this is set to `false`, the full control would be granted to the bucket owner. This may require to set additional permissions. See [S3 permissions settings](../development/extensions-core/s3.html#s3-permissions-settings).|false| |`druid.storage.archiveBucket`|S3 bucket name for archiving when running the *archive task*.|none| |`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none| |`druid.storage.useS3aSchema`|If true, use the "s3a" filesystem when using Hadoop-based ingestion. If false, the "s3n" filesystem will be used. Only affects Hadoop-based ingestion.|false| diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index cead6cdc687..53e5df92cd1 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -45,6 +45,7 @@ As an example, to set the region to 'us-east-1' through system properties: |`druid.s3.secretKey`|S3 secret key.|Must be set.| |`druid.storage.bucket`|Bucket to store in.|Must be set.| |`druid.storage.baseKey`|Base key prefix to use, i.e. what directory.|Must be set.| +|`druid.storage.disableAcl`|Boolean flag to disable ACL. If this is set to `false`, the full control would be granted to the bucket owner. This may require to set additional permissions. See [S3 permissions settings](#s3-permissions-settings).|false| |`druid.storage.sse.type`|Server-side encryption type. Should be one of `s3`, `kms`, and `custom`. See the below [Server-side encryption section](#server-side-encryption) for more details.|None| |`druid.storage.sse.kms.keyId`|AWS KMS key ID. Can be empty if `druid.storage.sse.type` is `kms`.|None| |`druid.storage.sse.custom.base64EncodedKey`|Base64-encoded key. Should be specified if `druid.storage.sse.type` is `custom`.|None| @@ -59,6 +60,11 @@ As an example, to set the region to 'us-east-1' through system properties: |`druid.s3.proxy.username`|User name to use when connecting through a proxy.|None| |`druid.s3.proxy.password`|Password to use when connecting through a proxy.|None| +### S3 permissions settings + +`s3:GetObject` and `s3:PutObject` are basically required for pushing/loading segments to/from S3. +If `druid.storage.disableAcl` is set to `false`, then `s3:GetBucketAcl` and `s3:PutObjectAcl` are additionally required to set ACL for objects. + ## Server-side encryption You can enable [server-side encryption](https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html) by setting From ef451d3603af7bdca8f77609a621c160f55b2b30 Mon Sep 17 00:00:00 2001 From: Surekha Date: Tue, 5 Feb 2019 13:42:20 -0800 Subject: [PATCH 12/13] Add null checks in DruidSchema (#6830) * Add null checks in DruidSchema * Add unit tests * Add VisibleForTesting annotation * PR comments * unused import --- .../druid/sql/calcite/schema/DruidSchema.java | 38 +++++++++++++------ .../sql/calcite/schema/DruidSchemaTest.java | 35 +++++++++++++++++ 2 files changed, 61 insertions(+), 12 deletions(-) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index 3efbd8d0074..2929229b4c5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -19,6 +19,7 @@ package org.apache.druid.sql.calcite.schema; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import com.google.common.collect.FluentIterable; @@ -402,7 +403,8 @@ public class DruidSchema extends AbstractSchema } } - private void removeSegment(final DataSegment segment) + @VisibleForTesting + protected void removeSegment(final DataSegment segment) { synchronized (lock) { log.debug("Segment[%s] is gone.", segment.getId()); @@ -450,7 +452,8 @@ public class DruidSchema extends AbstractSchema * Attempt to refresh "segmentSignatures" for a set of segments. Returns the set of segments actually refreshed, * which may be a subset of the asked-for set. */ - private Set refreshSegments(final Set segments) throws IOException + @VisibleForTesting + protected Set refreshSegments(final Set segments) throws IOException { final Set retVal = new HashSet<>(); @@ -506,15 +509,26 @@ public class DruidSchema extends AbstractSchema log.debug("Segment[%s] has signature[%s].", segment.getId(), rowSignature); final Map dataSourceSegments = segmentMetadataInfo.get(segment.getDataSource()); - SegmentMetadataHolder holder = dataSourceSegments.get(segment); - SegmentMetadataHolder updatedHolder = SegmentMetadataHolder - .from(holder) - .withRowSignature(rowSignature) - .withNumRows(analysis.getNumRows()) - .build(); - dataSourceSegments.put(segment, updatedHolder); - setSegmentSignature(segment, updatedHolder); - retVal.add(segment); + if (dataSourceSegments == null) { + log.warn("No segment map found with datasource[%s], skipping refresh", segment.getDataSource()); + } else { + SegmentMetadataHolder holder = dataSourceSegments.get(segment); + if (holder == null) { + log.warn( + "No segment[%s] found, skipping refresh", + segment.getId() + ); + } else { + SegmentMetadataHolder updatedHolder = SegmentMetadataHolder + .from(holder) + .withRowSignature(rowSignature) + .withNumRows(analysis.getNumRows()) + .build(); + dataSourceSegments.put(segment, updatedHolder); + setSegmentSignature(segment, updatedHolder); + retVal.add(segment); + } + } } } @@ -628,7 +642,7 @@ public class DruidSchema extends AbstractSchema return rowSignatureBuilder.build(); } - public Map getSegmentMetadata() + Map getSegmentMetadata() { final Map segmentMetadata = new HashMap<>(); synchronized (lock) { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java index 1b4e00819bb..30b99e59ab4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java @@ -63,6 +63,7 @@ import java.io.File; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Set; public class DruidSchemaTest extends CalciteTestBase { @@ -237,4 +238,38 @@ public class DruidSchemaTest extends CalciteTestBase Assert.assertEquals("m1", fields.get(2).getName()); Assert.assertEquals(SqlTypeName.BIGINT, fields.get(2).getType().getSqlTypeName()); } + + @Test + public void testNullDatasource() throws IOException + { + Map segmentMetadatas = schema.getSegmentMetadata(); + Set segments = segmentMetadatas.keySet(); + Assert.assertEquals(segments.size(), 3); + // segments contains two segments with datasource "foo" and one with datasource "foo2" + // let's remove the only segment with datasource "foo2" + final DataSegment segmentToRemove = segments.stream().filter(segment -> segment.getDataSource().equals("foo2")).findFirst().orElse(null); + Assert.assertFalse(segmentToRemove == null); + schema.removeSegment(segmentToRemove); + schema.refreshSegments(segments); // can cause NPE without dataSourceSegments null check in DruidSchema#refreshSegmentsForDataSource + segmentMetadatas = schema.getSegmentMetadata(); + segments = segmentMetadatas.keySet(); + Assert.assertEquals(segments.size(), 2); + } + + @Test + public void testNullSegmentMetadataHolder() throws IOException + { + Map segmentMetadatas = schema.getSegmentMetadata(); + Set segments = segmentMetadatas.keySet(); + Assert.assertEquals(segments.size(), 3); + //remove one of the segments with datasource "foo" + final DataSegment segmentToRemove = segments.stream().filter(segment -> segment.getDataSource().equals("foo")).findFirst().orElse(null); + Assert.assertFalse(segmentToRemove == null); + schema.removeSegment(segmentToRemove); + schema.refreshSegments(segments); // can cause NPE without holder null check in SegmentMetadataHolder#from + segmentMetadatas = schema.getSegmentMetadata(); + segments = segmentMetadatas.keySet(); + Assert.assertEquals(segments.size(), 2); + } + } From 315ccb76b819790a4861bf16e493dba47605291b Mon Sep 17 00:00:00 2001 From: anantmf Date: Tue, 5 Feb 2019 15:45:45 -0800 Subject: [PATCH 13/13] Fix for getSingleObjectSummary, replacing keyCount with objectSummaries().size (#7000) * Instead of using keyCount, changing it to check the size of objectSummaries. For issue: https://github.com/apache/incubator-druid/issues/6980 https://github.com/apache/incubator-druid/issues/6980#issuecomment-460006580 * Changing another usage of keyCount with size of objectSummaries. * Adding some comments to explain why using keyCount is not working as expected. --- .../java/org/apache/druid/storage/s3/S3DataSegmentMover.java | 5 ++++- .../src/main/java/org/apache/druid/storage/s3/S3Utils.java | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java index 4fd54b49956..84a76d15e41 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3DataSegmentMover.java @@ -174,7 +174,10 @@ public class S3DataSegmentMover implements DataSegmentMover .withPrefix(s3Path) .withMaxKeys(1) ); - if (listResult.getKeyCount() == 0) { + // Using getObjectSummaries().size() instead of getKeyCount as, in some cases + // it is observed that even though the getObjectSummaries returns some data + // keyCount is still zero. + if (listResult.getObjectSummaries().size() == 0) { // should never happen throw new ISE("Unable to list object [s3://%s/%s]", s3Bucket, s3Path); } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java index e0a3dac8cae..97858864b14 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java @@ -251,7 +251,10 @@ public class S3Utils .withMaxKeys(1); final ListObjectsV2Result result = s3Client.listObjectsV2(request); - if (result.getKeyCount() == 0) { + // Using getObjectSummaries().size() instead of getKeyCount as, in some cases + // it is observed that even though the getObjectSummaries returns some data + // keyCount is still zero. + if (result.getObjectSummaries().size() == 0) { throw new ISE("Cannot find object for bucket[%s] and key[%s]", bucket, key); } final S3ObjectSummary objectSummary = result.getObjectSummaries().get(0);