From 101ca9c00f5fd3721abc01add0aa2b81afcf381d Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 22 Jan 2019 12:47:56 +0700 Subject: [PATCH 1/9] Prohibit assigning concurrent maps into Map-types variables and fields; Fix a race condition in CoordinatorRuleManager; improve logic in DirectDruidClient and ResourcePool --- .idea/inspectionProfiles/Druid.xml | 9 +++++- .../util/http/client/pool/ResourcePool.java | 26 ++++++++++++---- .../lookup/KafkaLookupExtractorFactory.java | 3 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 5 ++-- .../kinesis/KinesisRecordSupplier.java | 4 ++- ...OnHeapNamespaceExtractionCacheManager.java | 2 +- .../indexing/overlord/ForkingTaskRunner.java | 5 ++-- .../druid/indexing/overlord/TaskQueue.java | 5 ++-- .../SeekableStreamIndexTaskRunner.java | 5 ++-- .../indexing/worker/WorkerTaskManager.java | 4 +-- .../hrtr/HttpRemoteTaskRunnerTest.java | 9 +++--- .../druid/query/ReflectionLoaderThingy.java | 4 +-- .../segment/incremental/IncrementalIndex.java | 4 +-- .../druid/query/RetryQueryRunnerTest.java | 11 +++---- .../DataSourceMetadataQueryTest.java | 3 +- .../TimeBoundaryQueryRunnerTest.java | 6 ++-- .../druid/client/DirectDruidClient.java | 30 +++++++++---------- .../CuratorDruidNodeDiscoveryProvider.java | 8 ++--- .../discovery/DruidNodeDiscoveryProvider.java | 3 +- .../appenderator/AppenderatorImpl.java | 2 +- .../appenderator/AppenderatorPlumber.java | 3 +- .../realtime/plumber/RealtimePlumber.java | 3 +- .../apache/druid/server/QueryLifecycle.java | 3 +- .../BatchDataSegmentAnnouncer.java | 4 +-- .../coordinator/ReplicationThrottler.java | 14 ++++----- .../helper/DruidCoordinatorBalancer.java | 4 ++- .../server/router/CoordinatorRuleManager.java | 17 ++++++----- .../druid/client/CachingQueryRunnerTest.java | 5 ++-- .../apache/druid/sql/avatica/DruidMeta.java | 11 ++++--- 29 files changed, 128 insertions(+), 84 deletions(-) diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml index b3c94b37f57e..35eb3c4dc43a 100644 --- a/.idea/inspectionProfiles/Druid.xml +++ b/.idea/inspectionProfiles/Druid.xml @@ -147,7 +147,7 @@ - + @@ -236,6 +236,13 @@ + + + + + + + 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 bd8d5e8e5e1f..40c491e1e1e1 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.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 @@ protected void finalize() throws Throwable 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 @@ private boolean holderListContains(V object) return resourceHolderList.stream().anyMatch(a -> a.getResource().equals(object)); } - void close() + @Override + public void close() { synchronized (this) { closed = true; 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 e65c7a382184..01b3dd2aff49 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.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 boolean start() 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 dedb5221746a..d081a0e0d796 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.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/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 622c1f37bbc3..a05eade5f4cd 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.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 @@ private void rescheduleRunnable(long delayMillis) 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/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 3612ce54565d..9c6871efc9c9 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 @@ void monitor(ServiceEmitter serviceEmitter) 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/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 c3a0bc94ded3..d3199834c635 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.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 ef30f15f2259..1f6612e7a542 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 @@ -54,6 +54,7 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -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 ConcurrentMap totalSuccessfulTaskCount = new ConcurrentHashMap<>(); + private final ConcurrentMap totalFailedTaskCount = new ConcurrentHashMap<>(); private Map prevTotalSuccessfulTaskCount = new HashMap<>(); private Map prevTotalFailedTaskCount = new HashMap<>(); 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 8e502f0387eb..857264965f50 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.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 enum Status private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; private final Map 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 0ec7b41089ff..613c69b3a2e1 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/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index 814a20f5e8cd..0adcf266014f 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.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 void testTaskRunnerRestart() throws Exception .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 void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); - Map workerHolders = new ConcurrentHashMap<>(); + ConcurrentMap workerHolders = new ConcurrentHashMap<>(); HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), @@ -600,7 +601,7 @@ public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); - Map workerHolders = new ConcurrentHashMap<>(); + ConcurrentMap workerHolders = new ConcurrentHashMap<>(); HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( TestHelper.makeJsonMapper(), @@ -777,7 +778,7 @@ public void testMarkWorkersLazy() throws Exception 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/ReflectionLoaderThingy.java b/processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java index a976b07e9992..6fb01cd0102a 100644 --- a/processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java +++ b/processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java @@ -22,8 +22,8 @@ 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.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; /** @@ -38,7 +38,7 @@ public static ReflectionLoaderThingy create(Class interfaceClass) return new ReflectionLoaderThingy(interfaceClass); } - Map, AtomicReference> toolChestMap = new ConcurrentHashMap<>(); + ConcurrentMap, AtomicReference> toolChestMap = new ConcurrentHashMap<>(); private final Class interfaceClass; 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 f20484ef3b34..865cbf45e7b2 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 Iterable timeRangeIterable(boolean descending, long 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 Iterable timeRangeIterable(boolean descending, long { 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/RetryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/RetryQueryRunnerTest.java index 6e921527aa76..378c024bbf00 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.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; public class RetryQueryRunnerTest { @@ -89,7 +90,7 @@ public boolean isReturnPartialResults() @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 boolean isReturnPartialResults() @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 Sequence> run( @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 Sequence> run( @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 Sequence> run( @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/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java index b2d6b87f242d..2a23b547f660 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.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; public class DataSourceMetadataQueryTest { @@ -137,7 +138,7 @@ public void testMaxIngestedEventTime() throws Exception 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/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 094b732dfce0..769d5cf0e3e8 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.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 void testTimeBoundaryMax() .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 void testTimeBoundaryMin() .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/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java index 3f86efd18c67..a6bbd81d02cc 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java @@ -74,6 +74,7 @@ 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,7 +93,8 @@ public class DirectDruidClient implements QueryRunner private static final Logger log = new Logger(DirectDruidClient.class); - private static final Map, Pair> typesMap = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap, Pair> typesMap = + new ConcurrentHashMap<>(); private final QueryToolChestWarehouse warehouse; private final QueryWatcher queryWatcher; @@ -113,13 +115,10 @@ public static void removeMagicResponseContextFields(Map response 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; } @@ -159,14 +158,15 @@ public Sequence run(final QueryPlus queryPlus, final Map c 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); + types = typesMap.computeIfAbsent(query.getClass(), queryClass -> { + final TypeFactory typeFactory = objectMapper.getTypeFactory(); + JavaType baseType = typeFactory.constructType(toolChest.getResultTypeReference()); + JavaType bySegmentType = typeFactory.constructParametricType( + Result.class, + typeFactory.constructParametricType(BySegmentResultValueClass.class, baseType) + ); + return Pair.of(baseType, bySegmentType); + }); } final JavaType typeRef; 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 7025319d2de9..86fe0c32a6f4 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.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 @@ private static class NodeTypeWatcher implements DruidNodeDiscovery 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 79e73c158fc9..4dcb77759a86 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.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; /** * Provider of {@link DruidNodeDiscovery} instances. @@ -81,7 +82,7 @@ private static class ServiceDruidNodeDiscovery implements DruidNodeDiscovery 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/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 83be7dc24310..69b6696565eb 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 @@ -121,7 +121,7 @@ public class AppenderatorImpl implements Appenderator private final IndexIO indexIO; private final IndexMerger indexMerger; private final Cache cache; - private final Map sinks = new ConcurrentHashMap<>(); + private final ConcurrentHashMap sinks = new ConcurrentHashMap<>(); private final Set droppingSinks = Sets.newConcurrentHashSet(); private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( String.CASE_INSENSITIVE_ORDER 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 ed0b8e43e7dc..fb20b1a4f97c 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.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; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index 645ceabef5d5..fe7a5a229942 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -90,6 +90,7 @@ import java.util.List; import java.util.Map; 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; @@ -110,7 +111,7 @@ public class RealtimePlumber implements Plumber private final SegmentPublisher segmentPublisher; private final SegmentHandoffNotifier handoffNotifier; private final Object handoffCondition = new Object(); - private final Map 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 7db7cf2d7d93..fe42ad05c05b 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 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 QueryResponse execute() { 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 07ad0efa0050..d45c51cced6f 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.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 0129197482ad..2c41cd1ddd9f 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.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 void addSegment(String tier, SegmentId segmentId, String serverId) 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 void removeSegment(String tier, SegmentId segmentId) 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 void resetLifetime(String tier) 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 7e978e879de5..60fc376f07fc 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.NavigableSet; import java.util.SortedSet; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; /** @@ -210,7 +211,8 @@ protected void moveSegment( 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/router/CoordinatorRuleManager.java b/server/src/main/java/org/apache/druid/server/router/CoordinatorRuleManager.java index c8c3b56a1041..49472be54374 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.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 void poll() 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/CachingQueryRunnerTest.java b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java index 530fd05f983c..444db9c2e1ec 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.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; @@ -227,7 +228,7 @@ public void after(boolean isDone, Throwable thrown) 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 void close(String namespace) } @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 ba5a87f006ec..814dcdd63f4e 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.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 From 0dc165996e924b5a1c554613f1a892004db3cae5 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 23 Jan 2019 11:07:11 +0700 Subject: [PATCH 2/9] 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 --- .idea/inspectionProfiles/Druid.xml | 25 ++++++++ .../indexing/kafka/KafkaIndexTaskTest.java | 4 +- .../kinesis/KinesisIndexTaskTest.java | 4 +- .../druid/indexing/common/Counters.java | 59 ++++++------------- .../common/actions/TaskActionToolbox.java | 11 +--- .../parallel/ParallelIndexSupervisorTask.java | 7 ++- .../SinglePhaseParallelIndexTaskRunner.java | 5 +- .../druid/indexing/overlord/TaskQueue.java | 12 ++-- .../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 +- .../SpecializationService.java | 6 +- .../druid/client/DirectDruidClient.java | 1 + .../druid/initialization/Initialization.java | 22 ++++--- .../EventReceiverFirehoseFactory.java | 3 +- .../server/http/DataSourcesResource.java | 6 +- .../druid/server/http/TiersResource.java | 6 +- 22 files changed, 94 insertions(+), 116 deletions(-) diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml index 35eb3c4dc43a..dcb28b16d6b6 100644 --- a/.idea/inspectionProfiles/Druid.xml +++ b/.idea/inspectionProfiles/Druid.xml @@ -243,6 +243,31 @@ + + + + + + + + + + + + + + + + + + + + + + + + + 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 0b6e7c9e0002..d7be7afecb7d 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.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 boolean checkPointDataSourceMetadata( ); return true; } - }, - new Counters() + } ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, 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 31bedd9ae892..0429e4e4dde6 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.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 boolean checkPointDataSourceMetadata( ); return true; } - }, - new Counters() + } ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, 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 3cf4ac8ba788..18c1cd0fd9f3 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,29 @@ 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) - { - return intCounters.computeIfAbsent(key, k -> new AtomicInteger()).addAndGet(val); + public static int incrementAndGetInt(ConcurrentHashMap counters, K key) { + // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. + 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) - { - return doubleCounters.computeIfAbsent(key, k -> new AtomicDouble()).addAndGet(val); + public static long incrementAndGetLong(ConcurrentHashMap counters, K key) { + // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. + 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 bb04d8dffcdb..e81a81403690 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 TaskActionToolbox( TaskStorage taskStorage, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, ServiceEmitter emitter, - SupervisorManager supervisorManager, - Counters counters + SupervisorManager supervisorManager ) { this.taskLockbox = taskLockbox; @@ -51,7 +48,6 @@ public TaskActionToolbox( this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; this.emitter = emitter; this.supervisorManager = supervisorManager; - this.counters = counters; } public TaskLockbox getTaskLockbox() @@ -78,9 +74,4 @@ public SupervisorManager getSupervisorManager() { 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 2591987d1f3f..7ff35e0ccbc2 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.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,8 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan private final AuthorizerMapper authorizerMapper; private final RowIngestionMetersFactory rowIngestionMetersFactory; - private final Counters counters = new Counters(); + /** interval -> parititionNum */ + private final ConcurrentHashMap partitionNumCountersPerInterval = new ConcurrentHashMap<>(); private volatile ParallelIndexTaskRunner runner; @@ -377,7 +380,7 @@ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException 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 d04315668e66..e1919ec56099 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.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/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 1f6612e7a542..afcbdb1f5ada 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.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; @@ -54,7 +55,6 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -104,8 +104,8 @@ public class TaskQueue private static final EmittingLogger log = new EmittingLogger(TaskQueue.class); - private final ConcurrentMap totalSuccessfulTaskCount = new ConcurrentHashMap<>(); - private final ConcurrentMap 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<>(); @@ -521,11 +521,9 @@ private void handleStatus(final TaskStatus status) ); 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/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 98a3e65f7138..ceb65d1f31e7 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 @@ 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 void before() 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 59899feb8151..4ce39a0cd562 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.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 SegmentPublishResult announceHistoricalSegments( 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 a9d68b5e1be3..8406adb7e948 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 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 TaskActionToolbox createTaskActionToolbox() 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 5317b66e4a46..9cea9576b896 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.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 @@ private TaskToolbox makeToolbox( 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 cc3ca979dcbb..668c8ec19856 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 9db06583a800..3329419cf337 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.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 void deleteSegments(Set segments) 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 31cbf2c580f4..265150c60564 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.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 @@ private TaskToolboxFactory setUpTaskToolboxFactory( 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 cb6cffee0d99..26af4b15fb01 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.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/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java b/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java index 43f1a1a5afdc..b5de1e27597c 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.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 static SpecializationState getSpecializationState( 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,7 @@ static class PerPrototypeClassState 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. SpecializationState alreadyExistingState = specializationStates.get(specializationId); if (alreadyExistingState != null) { return alreadyExistingState; @@ -278,7 +278,7 @@ static class WindowedLoopIterationCounter extends SpecializationState impl 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/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java index a6bbd81d02cc..d5005ebb4bbf 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java @@ -156,6 +156,7 @@ public Sequence run(final QueryPlus queryPlus, final Map c QueryToolChest> toolChest = warehouse.getToolChest(query); boolean isBySegment = QueryContexts.isBySegment(query); + // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. Pair types = typesMap.get(query.getClass()); if (types == null) { types = typesMap.computeIfAbsent(query.getClass(), queryClass -> { 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 fcf288ac6a30..81866d3b6989 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.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.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,17 @@ static Map getLoadersMap() * 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) -> { + Collection modulesToLoad = new ServiceLoadingFromExtensions<>(config, serviceC).implsToLoad; + extensionsMap.put(serviceClass, modulesToLoad); + return modulesToLoad; + }); + //noinspection unchecked + return (Collection) modules; } private static class ServiceLoadingFromExtensions 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 3abf0c4dcbf6..d6f843c578b1 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.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 EventReceiverFirehose implements ChatHandler, Firehose, EventReceiv private volatile boolean closed = false; private final AtomicLong bytesReceived = new AtomicLong(0); private final AtomicLong lastBufferAddFailMsgTime = new AtomicLong(0); - private final ConcurrentMap producerSequences = new ConcurrentHashMap<>(); + private final ConcurrentHashMap producerSequences = new ConcurrentHashMap<>(); private final Stopwatch idleWatch = Stopwatch.createUnstarted(); public EventReceiverFirehose(InputRowParser> parser) 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 93bda0ba6325..05535053f83a 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 @@ private Response getSegmentDataSourceIntervals( 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 7698b19e37d9..4debc83d17e4 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 @@ private enum IntervalProperties 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 Response getTierDataSources(@PathParam("tierName") String tierName, @Quer } } - return Response.ok(retVal).build(); + return Response.ok(tierToStatsPerInterval).build(); } Set retVal = serverInventoryView From 19056c1fdd6c65a4a776ccdea375279ba15f8c2f Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 23 Jan 2019 11:23:43 +0700 Subject: [PATCH 3/9] Remove unnecessary comment --- .../common/task/batch/parallel/ParallelIndexSupervisorTask.java | 1 - 1 file changed, 1 deletion(-) 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 7ff35e0ccbc2..b8a23097688b 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 @@ -100,7 +100,6 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan private final AuthorizerMapper authorizerMapper; private final RowIngestionMetersFactory rowIngestionMetersFactory; - /** interval -> parititionNum */ private final ConcurrentHashMap partitionNumCountersPerInterval = new ConcurrentHashMap<>(); private volatile ParallelIndexTaskRunner runner; From 2791d9f9bfc51ace72f81770a1df95f885a1da04 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 23 Jan 2019 18:43:43 +0700 Subject: [PATCH 4/9] Checkstyle --- .../java/org/apache/druid/indexing/common/Counters.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 18c1cd0fd9f3..4ea064abc983 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 @@ -25,7 +25,8 @@ public final class Counters { - public static int incrementAndGetInt(ConcurrentHashMap counters, K key) { + public static int incrementAndGetInt(ConcurrentHashMap counters, K key) + { // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. AtomicInteger counter = counters.get(key); if (counter == null) { @@ -34,7 +35,8 @@ public static int incrementAndGetInt(ConcurrentHashMap cou return counter.incrementAndGet(); } - public static long incrementAndGetLong(ConcurrentHashMap counters, K key) { + public static long incrementAndGetLong(ConcurrentHashMap counters, K key) + { // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. AtomicLong counter = counters.get(key); if (counter == null) { From 9d48d1889845f290ec9e7669be98e1669e378f22 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 25 Jan 2019 20:27:02 +0700 Subject: [PATCH 5/9] Fix getFromExtensions() --- .../org/apache/druid/initialization/Initialization.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) 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 81866d3b6989..bee01d16fffa 100644 --- a/server/src/main/java/org/apache/druid/initialization/Initialization.java +++ b/server/src/main/java/org/apache/druid/initialization/Initialization.java @@ -140,11 +140,10 @@ public static Collection getFromExtensions(ExtensionsConfig config, Class { // 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) -> { - Collection modulesToLoad = new ServiceLoadingFromExtensions<>(config, serviceC).implsToLoad; - extensionsMap.put(serviceClass, modulesToLoad); - return modulesToLoad; - }); + Collection modules = extensionsMap.compute( + serviceClass, + (serviceC, ignored) -> new ServiceLoadingFromExtensions<>(config, serviceC).implsToLoad + ); //noinspection unchecked return (Collection) modules; } From 14307c31c6ebba81fcbc44bfc44d522afcea8c91 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 29 Jan 2019 10:46:53 +0700 Subject: [PATCH 6/9] Add a reference to the comment about guarded computeIfAbsent() optimization; IdentityHashMap optimization --- .../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 +-- ...terializedViewQueryQueryToolChestTest.java | 2 +- .../segment/MapVirtualColumnGroupByTest.java | 2 +- .../segment/MapVirtualColumnSelectTest.java | 2 +- .../segment/MapVirtualColumnTopNTest.java | 2 +- .../ApproximateHistogramTopNQueryTest.java | 4 +- .../FixedBucketsHistogramTopNQueryTest.java | 4 +- .../namespace/cache/CacheScheduler.java | 6 +- .../variance/VarianceTopNQueryTest.java | 2 +- .../druid/indexing/common/Counters.java | 2 + ...DefaultQueryRunnerFactoryConglomerate.java | 9 ++- .../query/MapQueryToolChestWarehouse.java | 9 ++- .../apache/druid/query/QueryToolChest.java | 30 ++++++++ .../druid/query/ReflectionLoaderThingy.java | 77 ------------------- .../ReflectionQueryToolChestWarehouse.java | 22 +++++- .../DataSourceQueryQueryToolChest.java | 5 +- .../SpecializationService.java | 1 + .../druid/query/MultiValuedDimensionTest.java | 2 +- .../druid/query/QueryRunnerTestHelper.java | 4 +- .../apache/druid/query/TestQueryRunners.java | 6 +- .../aggregation/AggregationTestHelper.java | 8 +- .../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 +- .../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 | 31 +------- .../EventReceiverFirehoseFactory.java | 10 +-- .../client/CachingClusteredClientTest.java | 30 ++++---- .../CachingClusteredClientTestUtils.java | 8 +- .../druid/client/CachingQueryRunnerTest.java | 4 +- .../druid/sql/calcite/util/CalciteTests.java | 8 +- 58 files changed, 184 insertions(+), 239 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/query/ReflectionLoaderThingy.java 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 2116a14c86bf..e156d24b38c4 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 void setup() throws IOException 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 43a353cd2db2..9a13f7c68458 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 String getFormatString() 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 2974441ae2a7..3d01e1a3eec7 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 void setup() throws IOException 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 243564c0e29e..f96aef616c10 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 String getFormatString() 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 411ca226e00c..6e6ae2a82b72 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 static > QueryRunner makeQueryRunner( ); } - 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 5c84a6799bce..122914ba00e1 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 void setup() throws IOException 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 bd2126465664..36fd251ba312 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 void setup() throws IOException 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 94efa930e065..5c58c94ff87f 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 void setup() throws IOException 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 717cd36968bb..566ebe326b2f 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 void setup() throws IOException 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 179bbd7cb7a1..85ef6007af8f 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 @@ private void setupQueries() 0, Integer.MAX_VALUE ), - new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() + ), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); } @@ -270,7 +273,7 @@ private void setupQueries() 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 2588b32614be..8e4db9edf8ba 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 void emit(Event event) { 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/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 0c750c3f5179..e3974eecb635 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 void testMakePostComputeManipulatorFn() .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 29f25fa83fa4..5a28fbf98d75 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 int getNumThreads() 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 ee15c7ec3689..f248bec0d3bb 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 static Iterable constructorFeeder() throws IOException 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 be1c10a43d77..761230d91cbf 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 void setup() throws IOException 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 5bac2ac8a0a8..4a5585da717f 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 static Iterable constructorFeeder() defaultPool, new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -91,7 +91,7 @@ public static Iterable constructorFeeder() 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 42fc6487f3e9..04bc90469b1c 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 static Iterable constructorFeeder() defaultPool, new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -91,7 +91,7 @@ public static Iterable constructorFeeder() customPool, new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) 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 658cd2ce1058..40a05992114f 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 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 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 Entry scheduleAndWait(ExtractionNamespace namespace, long waitForFirstRun 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/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 e634b67984b6..7b5406bc4b3e 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 @@ private Sequence> assertExpectedResults( { 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 4ea064abc983..e46389083fb6 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 @@ -28,6 +28,7 @@ public final class Counters public static int incrementAndGetInt(ConcurrentHashMap counters, K key) { // 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()); @@ -38,6 +39,7 @@ public static int incrementAndGetInt(ConcurrentHashMap cou public static long incrementAndGetLong(ConcurrentHashMap counters, K key) { // 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()); 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 ecd77b70021b..7f004ffa6343 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 @@ 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 db88db14dbe3..7199c957c4a4 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 @@ 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 0b890c061331..22194598350f 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,34 @@ @ExtensionPoint public abstract class QueryToolChest> { + private final JavaType baseResultType; + private final JavaType bySegmentResultType; + + protected QueryToolChest() + { + final TypeFactory typeFactory = TypeFactory.defaultInstance(); + baseResultType = typeFactory.constructType(getResultTypeReference()); + bySegmentResultType = typeFactory.constructParametrizedType( + Result.class, + Result.class, + typeFactory.constructParametrizedType( + BySegmentResultValueClass.class, + BySegmentResultValueClass.class, + baseResultType + ) + ); + } + + 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 6fb01cd0102a..000000000000 --- 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.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -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); - } - - ConcurrentMap, 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 4c7aab72d439..e2b047c1b2d1 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 13d265186b40..e1befab71ad3 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 @@ 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 b5de1e27597c..7f0089d73a05 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 @@ -164,6 +164,7 @@ SpecializationState getSpecializationState(String runtimeShape, ImmutableMap< { 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; 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 877043bcf729..c4c4a0b99af6 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 void testTopNWithDimFilterAndWithFilteredDimSpec() 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 1b7b8900a6fc..094e8fc2c321 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 Sequence run(QueryPlus queryPlus, Map responseConte .applyPostMergeDecoration(); } - public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator() + public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() { return new IntervalChunkingQueryRunnerDecorator(null, null, null) { @@ -518,7 +518,7 @@ public static Map of(Object... keyvalues) 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/TestQueryRunners.java b/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java index 24b104da87c6..ecba0055ef2e 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 static QueryRunner makeTopNQueryRunner(Segment adapter, NonBlockin pool, new TopNQueryQueryToolChest( topNConfig, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -71,7 +71,7 @@ public static QueryRunner makeTimeSeriesQueryRunner(Segment adapter) { QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -89,7 +89,7 @@ public static QueryRunner makeSearchQueryRunner(Segment adapter) 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 f1503cb1c312..cff1b4f689ac 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 static AggregationTestHelper createSelectQueryAggregationTestHelper( 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 static AggregationTestHelper createTimeseriesQueryAggregationTestHelper( ObjectMapper mapper = TestHelper.makeJsonMapper(); TimeseriesQueryQueryToolChest toolchest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( @@ -284,7 +284,7 @@ public static AggregationTestHelper createTopNQueryAggregationTestHelper( 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/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index 19f3a5862677..36e55051e5f8 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 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( ); 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 9222ba5c7ace..e804de9cf49d 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 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( ); 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 3a2d5324e53b..96e369966dbb 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 ByteBuffer get() 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 0eb5e9afd922..e730abf1d410 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 String getFormatString() strategySelector, new GroupByQueryQueryToolChest( strategySelector, - NoopIntervalChunkingQueryRunnerDecorator() + noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -362,7 +362,7 @@ public String getFormatString() strategySelector2, new GroupByQueryQueryToolChest( strategySelector2, - NoopIntervalChunkingQueryRunnerDecorator() + noopIntervalChunkingQueryRunnerDecorator() ) ); } @@ -919,7 +919,7 @@ public void registerQuery(Query query, ListenableFuture future) } }; - 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 4d3d1e059057..af2bdf8b28f6 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 98d3568b3fbd..b80253890ea2 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.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 @@ static SearchQueryRunnerFactory makeRunnerFactory(final SearchQueryConfig config 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 2edc2b6c8e5c..1b7eacc449f0 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 5d9da392a89c..46f12daad0ff 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 a5ce234c68be..3034315c5984 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/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 4efcddb990e0..48a322b08c87 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 static Iterable constructorFeeder() 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 void testUnionResultMerging() ) .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 824552203d9f..22134a10e863 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 void testOneRowAtATime() throws Exception 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 d59ddf12e2fc..c24fc72c0da2 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 static Iterable constructorFeeder() QueryRunnerTestHelper.makeQueryRunners( new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER @@ -508,7 +508,7 @@ public void testTimeseriesGrandTotal() // 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 void testTimeseriesIntervalOutOfRanges() // 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 void testTimeSeriesWithSelectionFilterLookupExtractionFn() 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 void testTimeseriesWithLimit() // 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 8212dc7f7a7c..e263c1eef2b8 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 void testMinTopNThreshold() 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 a43957e755cf..839ee6d5aba7 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 ByteBuffer get() } } ), - 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 da23ca673a73..0244372dc309 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 @@ private Sequence> runWithMerge(TopNQuery query, Map> mergeRunner = new FinalizeResultsQueryRunner( chest.mergeResults(runner), @@ -4281,7 +4281,7 @@ private Sequence> runWithPreMergeAndMerge(TopNQuery quer { 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 0aa1fcde8827..076bbda48fe3 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 static Iterable constructorFeeder() defaultPool, new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -83,7 +83,7 @@ public static Iterable constructorFeeder() 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 1f94d71b5b77..e2064e0d89e7 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 void testSpatialQuery() try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER @@ -628,7 +628,7 @@ public void testSpatialQueryWithOtherSpatialDim() try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER @@ -717,7 +717,7 @@ public void testSpatialQueryMorePoints() 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 a9c60350f4c9..a3c3478ebedd 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 void testSingleThreadedIndexingAndQuery() throws Exception 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 void testConcurrentAddRead() throws InterruptedException, ExecutionExcept 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 ff54321b5d07..69857f5c5ecc 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 void testSpatialQuery() try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -596,7 +596,7 @@ public void testSpatialQueryMorePoints() try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -690,7 +690,7 @@ public void testSpatialQueryFilteredAggregator() 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 ee745a50f1cb..22e3e6838297 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 void testSpatialQuery() try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -617,7 +617,7 @@ public void testSpatialQueryWithOtherSpatialDim() try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -705,7 +705,7 @@ public void testSpatialQueryMorePoints() 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 eb4609cb6881..096e4d3de432 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 void testConcurrentAddRead() 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 d5005ebb4bbf..7f73abf97c8f 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 @@ 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.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.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.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; @@ -93,9 +89,6 @@ public class DirectDruidClient implements QueryRunner private static final Logger log = new Logger(DirectDruidClient.class); - private static final ConcurrentHashMap, Pair> typesMap = - new ConcurrentHashMap<>(); - private final QueryToolChestWarehouse warehouse; private final QueryWatcher queryWatcher; private final ObjectMapper objectMapper; @@ -155,27 +148,7 @@ public Sequence run(final QueryPlus queryPlus, final Map c final Query query = queryPlus.getQuery(); QueryToolChest> toolChest = warehouse.getToolChest(query); boolean isBySegment = QueryContexts.isBySegment(query); - - // get() before computeIfAbsent() is an optimization to avoid locking in computeIfAbsent() if not needed. - Pair types = typesMap.get(query.getClass()); - if (types == null) { - types = typesMap.computeIfAbsent(query.getClass(), queryClass -> { - final TypeFactory typeFactory = objectMapper.getTypeFactory(); - JavaType baseType = typeFactory.constructType(toolChest.getResultTypeReference()); - JavaType bySegmentType = typeFactory.constructParametricType( - Result.class, - typeFactory.constructParametricType(BySegmentResultValueClass.class, baseType) - ); - return Pair.of(baseType, bySegmentType); - }); - } - - 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); @@ -544,7 +517,7 @@ public void onFailure(Throwable t) @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/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index d6f843c578b1..09751f9d0fb4 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 @@ -486,11 +486,11 @@ private Optional checkProducerSequence( if (newSequence <= producerSequence) { return Optional.of( Response.ok( - responseMapper.writeValueAsString( - ImmutableMap.of("eventCount", 0, "skipped", true) - ), - responseContentType - ).build() + responseMapper.writeValueAsString( + ImmutableMap.of("eventCount", 0, "skipped", true) + ), + responseContentType + ).build() ); } 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 2eff321d78f0..ac57d5f62e5b 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 void run() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -483,7 +483,7 @@ public void testTimeseriesCaching() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -610,7 +610,7 @@ public void testTimeseriesMergingOutOfOrderPartitions() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()) + new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()) ); testQueryCaching( @@ -671,7 +671,7 @@ public void testTimeseriesCachingTimeZone() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -716,7 +716,7 @@ public void testDisableUseCache() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()) + new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()) ); testQueryCaching( runner, @@ -792,7 +792,7 @@ public void testTopNCaching() getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -869,7 +869,7 @@ public void testTopNCachingTimeZone() getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -974,7 +974,7 @@ public void testTopNCachingEmptyResults() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); testQueryCaching( @@ -1047,7 +1047,7 @@ public void testTopNOnPostAggMetricCaching() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); testQueryCaching( @@ -1148,7 +1148,7 @@ public void testSearchCaching() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new SearchQueryQueryToolChest( new SearchQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); HashMap context = new HashMap(); @@ -1218,7 +1218,7 @@ public void testSearchCachingRenamedOutput() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new SearchQueryQueryToolChest( new SearchQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); HashMap context = new HashMap(); @@ -1316,7 +1316,7 @@ public void testSelectCaching() getDefaultQueryRunner(), new SelectQueryQueryToolChest( JSON_MAPPER, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), SELECT_CONFIG_SUPPLIER ) ); @@ -1394,7 +1394,7 @@ public void testSelectCachingRenamedOutputName() getDefaultQueryRunner(), new SelectQueryQueryToolChest( JSON_MAPPER, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), SELECT_CONFIG_SUPPLIER ) ); @@ -1638,7 +1638,7 @@ public void testTimeSeriesWithFilter() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ) ); @@ -1712,7 +1712,7 @@ public void testSingleDimensionPruning() 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 5a283f893978..8de4223c3e69 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 static Pair createWarehouse( .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 444db9c2e1ec..630321a0247f 100644 --- a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java @@ -142,7 +142,7 @@ public void testCloseAndPopulate() throws Exception QueryToolChest toolchest = new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); testCloseAndPopulate(expectedRes, expectedCacheRes, builder.build(), toolchest); @@ -190,7 +190,7 @@ public void testTimeseries() throws Exception } QueryToolChest toolChest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ); testCloseAndPopulate(expectedResults, expectedResults, query, toolChest); 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 359ca25bd066..a4bbb67b7960 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 @@ -514,7 +514,7 @@ public int getNumMergeBuffers() new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( TestHelper.makeJsonMapper(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(), SELECT_CONFIG_SUPPLIER ), new SelectQueryEngine(), @@ -524,9 +524,7 @@ public int getNumMergeBuffers() .put( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -537,7 +535,7 @@ public int getNumMergeBuffers() stupidPool, new TopNQueryQueryToolChest( new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) From e363b0197ea5d6ad88e059859dd722068e801be4 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 1 Feb 2019 10:03:44 +0700 Subject: [PATCH 7/9] Fix UriCacheGeneratorTest --- .../lookup/namespace/UriCacheGeneratorTest.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) 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 87cae6af656c..8f8b507b3156 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.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.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 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 From cd472a106b34a7bc879ec7c68ee4afaf7beb8e25 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 4 Feb 2019 08:20:31 +0700 Subject: [PATCH 8/9] Workaround issue with MaterializedViewQueryQueryToolChest --- .../apache/druid/query/QueryToolChest.java | 28 ++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) 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 22194598350f..81f59e43f2e5 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -42,16 +42,24 @@ public abstract class QueryToolChest 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() From 4bb8a7a16f67e0d6dc4cd026b4dec391b180498a Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 4 Feb 2019 19:46:32 +0700 Subject: [PATCH 9/9] Strengthen Appenderator's contract regarding concurrency --- .../realtime/appenderator/Appenderator.java | 52 +++++++++++-------- .../appenderator/AppenderatorImpl.java | 23 ++++---- .../appenderator/AppenderatorPlumber.java | 3 +- .../appenderator/BaseAppenderatorDriver.java | 1 + 4 files changed, 47 insertions(+), 32 deletions(-) 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 7f07ea74030b..7ad67ecc8afe 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 @@ * 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 @@ default AppenderatorAddResult add(SegmentIdWithShardSpec identifier, InputRow ro * 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 @@ AppenderatorAddResult add( * * @throws IllegalStateException if the segment is unknown */ + @VisibleForTesting int getRowCount(SegmentIdWithShardSpec identifier); /** @@ -129,20 +137,23 @@ AppenderatorAddResult add( * 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 @@ AppenderatorAddResult add( * 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 @@ AppenderatorAddResult add( *

* 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 @@ ListenableFuture push( ); /** - * 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 69b6696565eb..351013474c1e 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.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 ConcurrentHashMap 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 @@ private ListenableFuture abandonSegment( // 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 Object apply(@Nullable Object input) .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 fb20b1a4f97c..8eb27db0dd38 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 @@ -148,7 +148,8 @@ public Object startJob() } @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 a644e607fd3b..cb6ba9085a7f 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 @@ ListenableFuture publishInBackground( /** * Clears out all our state and also calls {@link Appenderator#clear()} on the underlying Appenderator. */ + @VisibleForTesting public void clear() throws InterruptedException { synchronized (segments) {