mirror of https://github.com/apache/druid.git
Forbid MapMaker (#4845)
* Forbid MapMaker * Shorter syntax * Forbid Maps.newConcurrentMap()
This commit is contained in:
parent
c702ac771f
commit
9c126e2aa9
|
@ -0,0 +1,2 @@
|
|||
com.google.common.collect.MapMaker @ Create java.util.concurrent.ConcurrentHashMap directly
|
||||
com.google.common.collect.Maps#newConcurrentMap() @ Create java.util.concurrent.ConcurrentHashMap directly
|
|
@ -21,21 +21,19 @@ package io.druid.common.config;
|
|||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
|
||||
import io.druid.java.util.common.concurrent.ScheduledExecutors;
|
||||
import io.druid.java.util.common.lifecycle.LifecycleStart;
|
||||
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
|
||||
import org.joda.time.Duration;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
@ -66,7 +64,7 @@ public class ConfigManager
|
|||
this.config = config;
|
||||
|
||||
this.exec = ScheduledExecutors.fixed(1, "config-manager-%s");
|
||||
this.watchedConfigs = Maps.newConcurrentMap();
|
||||
this.watchedConfigs = new ConcurrentHashMap<>();
|
||||
|
||||
this.configTable = dbTables.get().getConfigTable();
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.ByteSink;
|
||||
import com.google.common.io.ByteSource;
|
||||
import com.google.common.io.ByteStreams;
|
||||
|
@ -55,10 +54,10 @@ import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
|
|||
import io.druid.indexing.worker.config.WorkerConfig;
|
||||
import io.druid.java.util.common.DateTimes;
|
||||
import io.druid.java.util.common.IOE;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.io.Closer;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.io.Closer;
|
||||
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.DruidMetrics;
|
||||
|
@ -84,6 +83,7 @@ import java.util.Properties;
|
|||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -109,7 +109,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
private final CopyOnWriteArrayList<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
// Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting.
|
||||
private final Map<String, ForkingTaskRunnerWorkItem> tasks = Maps.newConcurrentMap();
|
||||
private final Map<String, ForkingTaskRunnerWorkItem> tasks = new ConcurrentHashMap<>();
|
||||
|
||||
private volatile boolean stopping = false;
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
|
@ -134,6 +133,7 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
|
@ -989,7 +989,7 @@ public class RealtimeIndexTaskTest
|
|||
)
|
||||
)
|
||||
);
|
||||
handOffCallbacks = Maps.newConcurrentMap();
|
||||
handOffCallbacks = new ConcurrentHashMap<>();
|
||||
final SegmentHandoffNotifierFactory handoffNotifierFactory = new SegmentHandoffNotifierFactory()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -32,7 +32,6 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
@ -140,6 +139,7 @@ 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.CountDownLatch;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
|
@ -354,7 +354,7 @@ public class TaskLifecycleTest
|
|||
emitter = newMockEmitter();
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
mapper = TEST_UTILS.getTestObjectMapper();
|
||||
handOffCallbacks = Maps.newConcurrentMap();
|
||||
handOffCallbacks = new ConcurrentHashMap<>();
|
||||
|
||||
// Set up things, the order does matter as if it is messed up then the setUp
|
||||
// should fail because of the Precondition checks in the respective setUp methods
|
||||
|
|
|
@ -20,8 +20,6 @@
|
|||
package io.druid.indexing.test;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.client.FilteredServerInventoryView;
|
||||
import io.druid.client.ServerView;
|
||||
|
@ -30,12 +28,13 @@ import io.druid.server.coordination.DruidServerMetadata;
|
|||
import io.druid.timeline.DataSegment;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
public class TestServerView implements FilteredServerInventoryView, ServerView.SegmentCallback
|
||||
{
|
||||
final ConcurrentMap<ServerView.SegmentCallback, Pair<Predicate<Pair<DruidServerMetadata, DataSegment>>, Executor>> callbacks = Maps.newConcurrentMap();
|
||||
final ConcurrentMap<ServerView.SegmentCallback, Pair<Predicate<Pair<DruidServerMetadata, DataSegment>>, Executor>> callbacks = new ConcurrentHashMap<>();
|
||||
|
||||
@Override
|
||||
public void registerSegmentCallback(
|
||||
|
|
|
@ -218,6 +218,7 @@
|
|||
<signaturesFiles>
|
||||
<!-- Needed because of https://github.com/policeman-tools/forbidden-apis/issues/126 -->
|
||||
<signaturesFile>../codestyle/joda-time-forbidden-apis.txt</signaturesFile>
|
||||
<signaturesFile>../codestyle/druid-forbidden-apis.txt</signaturesFile>
|
||||
</signaturesFiles>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
@ -261,6 +262,7 @@
|
|||
<signaturesFiles>
|
||||
<!-- Needed because of https://github.com/policeman-tools/forbidden-apis/issues/126 -->
|
||||
<signaturesFile>../codestyle/joda-time-forbidden-apis.txt</signaturesFile>
|
||||
<signaturesFile>../codestyle/druid-forbidden-apis.txt</signaturesFile>
|
||||
</signaturesFiles>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
|
1
pom.xml
1
pom.xml
|
@ -938,6 +938,7 @@
|
|||
</bundledSignatures>
|
||||
<signaturesFiles>
|
||||
<signaturesFile>${session.executionRootDirectory}/codestyle/joda-time-forbidden-apis.txt</signaturesFile>
|
||||
<signaturesFile>${session.executionRootDirectory}/codestyle/druid-forbidden-apis.txt</signaturesFile>
|
||||
</signaturesFiles>
|
||||
</configuration>
|
||||
<executions>
|
||||
|
|
|
@ -19,11 +19,11 @@
|
|||
|
||||
package io.druid.query;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
|
@ -38,7 +38,7 @@ public class ReflectionLoaderThingy<T>
|
|||
return new ReflectionLoaderThingy<K>(interfaceClass);
|
||||
}
|
||||
|
||||
Map<Class<?>, AtomicReference<T>> toolChestMap = Maps.newConcurrentMap();
|
||||
Map<Class<?>, AtomicReference<T>> toolChestMap = new ConcurrentHashMap<>();
|
||||
|
||||
private final Class<T> interfaceClass;
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.java.util.common.DateTimes;
|
||||
import io.druid.java.util.common.Intervals;
|
||||
|
@ -40,6 +39,7 @@ import org.junit.Test;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
public class RetryQueryRunnerTest
|
||||
{
|
||||
|
@ -89,7 +89,7 @@ public class RetryQueryRunnerTest
|
|||
@Test
|
||||
public void testRunWithMissingSegments() throws Exception
|
||||
{
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
|
||||
new QueryRunner<Result<TimeseriesResultValue>>()
|
||||
|
@ -139,7 +139,7 @@ public class RetryQueryRunnerTest
|
|||
@Test
|
||||
public void testRetry() throws Exception
|
||||
{
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put("count", 0);
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
|
||||
|
@ -193,7 +193,7 @@ public class RetryQueryRunnerTest
|
|||
@Test
|
||||
public void testRetryMultiple() throws Exception
|
||||
{
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put("count", 0);
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
|
||||
|
@ -247,7 +247,7 @@ public class RetryQueryRunnerTest
|
|||
@Test(expected = SegmentMissingException.class)
|
||||
public void testException() throws Exception
|
||||
{
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
|
||||
new QueryRunner<Result<TimeseriesResultValue>>()
|
||||
|
@ -285,7 +285,7 @@ public class RetryQueryRunnerTest
|
|||
@Test
|
||||
public void testNoDuplicateRetry() throws Exception
|
||||
{
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put("count", 0);
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.DateTimes;
|
||||
|
@ -53,6 +52,7 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
public class DataSourceMetadataQueryTest
|
||||
{
|
||||
|
@ -137,7 +137,7 @@ public class DataSourceMetadataQueryTest
|
|||
DataSourceMetadataQuery dataSourceMetadataQuery = Druids.newDataSourceMetadataQueryBuilder()
|
||||
.dataSource("testing")
|
||||
.build();
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
Iterable<Result<DataSourceMetadataResultValue>> results = Sequences.toList(
|
||||
runner.run(QueryPlus.wrap(dataSourceMetadataQuery), context),
|
||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.query.timeboundary;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.io.CharSource;
|
||||
import io.druid.java.util.common.DateTimes;
|
||||
import io.druid.java.util.common.Intervals;
|
||||
|
@ -59,6 +58,7 @@ import java.util.Arrays;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -240,7 +240,7 @@ public class TimeBoundaryQueryRunnerTest
|
|||
.dataSource("testing")
|
||||
.bound(TimeBoundaryQuery.MAX_TIME)
|
||||
.build();
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||
runner.run(QueryPlus.wrap(timeBoundaryQuery), context),
|
||||
|
@ -262,7 +262,7 @@ public class TimeBoundaryQueryRunnerTest
|
|||
.dataSource("testing")
|
||||
.bound(TimeBoundaryQuery.MIN_TIME)
|
||||
.build();
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
Map<String, Object> context = new ConcurrentHashMap<>();
|
||||
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
|
||||
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||
runner.run(QueryPlus.wrap(timeBoundaryQuery), context),
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.concurrent.Execs;
|
||||
import io.druid.curator.inventory.CuratorInventoryManager;
|
||||
|
@ -38,6 +37,7 @@ import org.apache.curator.utils.ZKPaths;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -52,8 +52,8 @@ public abstract class AbstractCuratorServerInventoryView<InventoryType> implemen
|
|||
private final CuratorInventoryManager<DruidServer, InventoryType> inventoryManager;
|
||||
private final AtomicBoolean started = new AtomicBoolean(false);
|
||||
|
||||
private final ConcurrentMap<ServerRemovedCallback, Executor> serverRemovedCallbacks = new MapMaker().makeMap();
|
||||
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks = new MapMaker().makeMap();
|
||||
private final ConcurrentMap<ServerRemovedCallback, Executor> serverRemovedCallbacks = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks = new ConcurrentHashMap<>();
|
||||
|
||||
public AbstractCuratorServerInventoryView(
|
||||
final EmittingLogger log,
|
||||
|
|
|
@ -26,7 +26,6 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
@ -39,6 +38,7 @@ import io.druid.timeline.DataSegment;
|
|||
import org.apache.curator.framework.CuratorFramework;
|
||||
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
|
@ -50,9 +50,9 @@ public class BatchServerInventoryView extends AbstractCuratorServerInventoryView
|
|||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(BatchServerInventoryView.class);
|
||||
|
||||
final private ConcurrentMap<String, Set<DataSegment>> zNodes = new MapMaker().makeMap();
|
||||
final private ConcurrentMap<SegmentCallback, Predicate<Pair<DruidServerMetadata, DataSegment>>> segmentPredicates = new MapMaker()
|
||||
.makeMap();
|
||||
final private ConcurrentMap<String, Set<DataSegment>> zNodes = new ConcurrentHashMap<>();
|
||||
final private ConcurrentMap<SegmentCallback, Predicate<Pair<DruidServerMetadata, DataSegment>>> segmentPredicates =
|
||||
new ConcurrentHashMap<>();
|
||||
final private Predicate<Pair<DruidServerMetadata, DataSegment>> defaultFilter;
|
||||
|
||||
@Inject
|
||||
|
|
|
@ -96,7 +96,7 @@ public class BrokerServerView implements TimelineServerView
|
|||
this.baseView = baseView;
|
||||
this.tierSelectorStrategy = tierSelectorStrategy;
|
||||
this.emitter = emitter;
|
||||
this.clients = Maps.newConcurrentMap();
|
||||
this.clients = new ConcurrentHashMap<>();
|
||||
this.selectors = Maps.newHashMap();
|
||||
this.timelines = Maps.newHashMap();
|
||||
|
||||
|
|
|
@ -29,8 +29,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
|||
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.ByteSource;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
|
@ -42,10 +40,10 @@ import com.metamx.http.client.response.ClientResponse;
|
|||
import com.metamx.http.client.response.HttpResponseHandler;
|
||||
import com.metamx.http.client.response.StatusResponseHandler;
|
||||
import com.metamx.http.client.response.StatusResponseHolder;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.java.util.common.RE;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.guava.BaseSequence;
|
||||
import io.druid.java.util.common.guava.CloseQuietly;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
|
@ -85,6 +83,7 @@ import java.util.Iterator;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
@ -104,7 +103,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
|||
|
||||
private static final Logger log = new Logger(DirectDruidClient.class);
|
||||
|
||||
private static final Map<Class<? extends Query>, Pair<JavaType, JavaType>> typesMap = Maps.newConcurrentMap();
|
||||
private static final Map<Class<? extends Query>, Pair<JavaType, JavaType>> typesMap = new ConcurrentHashMap<>();
|
||||
|
||||
private final QueryToolChestWarehouse warehouse;
|
||||
private final QueryWatcher queryWatcher;
|
||||
|
@ -142,7 +141,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
|||
|
||||
public static Map<String, Object> makeResponseContextForQuery(Query query, long startTimeMillis)
|
||||
{
|
||||
final Map<String, Object> responseContext = new MapMaker().makeMap();
|
||||
final Map<String, Object> responseContext = new ConcurrentHashMap<>();
|
||||
responseContext.put(
|
||||
DirectDruidClient.QUERY_FAIL_TIME,
|
||||
startTimeMillis + QueryContexts.getTimeout(query)
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.net.HostAndPort;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
|
@ -91,11 +90,11 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
|||
|
||||
private final LifecycleLock lifecycleLock = new LifecycleLock();
|
||||
|
||||
private final ConcurrentMap<ServerRemovedCallback, Executor> serverCallbacks = new MapMaker().makeMap();
|
||||
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks = new MapMaker().makeMap();
|
||||
private final ConcurrentMap<ServerRemovedCallback, Executor> serverCallbacks = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks = new ConcurrentHashMap<>();
|
||||
|
||||
private final ConcurrentMap<SegmentCallback, Predicate<Pair<DruidServerMetadata, DataSegment>>> segmentPredicates = new MapMaker()
|
||||
.makeMap();
|
||||
private final ConcurrentMap<SegmentCallback, Predicate<Pair<DruidServerMetadata, DataSegment>>> segmentPredicates =
|
||||
new ConcurrentHashMap<>();
|
||||
private final Predicate<Pair<DruidServerMetadata, DataSegment>> defaultFilter;
|
||||
private volatile Predicate<Pair<DruidServerMetadata, DataSegment>> finalPredicate;
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.guice.ManageLifecycle;
|
||||
|
@ -34,6 +33,7 @@ import io.druid.server.initialization.ZkPathsConfig;
|
|||
import io.druid.timeline.DataSegment;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
|
@ -44,8 +44,8 @@ public class SingleServerInventoryView extends AbstractCuratorServerInventoryVie
|
|||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(SingleServerInventoryView.class);
|
||||
|
||||
final private ConcurrentMap<SegmentCallback, Predicate<Pair<DruidServerMetadata, DataSegment>>> segmentPredicates = new MapMaker()
|
||||
.makeMap();
|
||||
final private ConcurrentMap<SegmentCallback, Predicate<Pair<DruidServerMetadata, DataSegment>>> segmentPredicates =
|
||||
new ConcurrentHashMap<>();
|
||||
private final Predicate<Pair<DruidServerMetadata, DataSegment>> defaultFilter;
|
||||
|
||||
@Inject
|
||||
|
|
|
@ -21,13 +21,12 @@ package io.druid.curator.announcement;
|
|||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.curator.cache.PathChildrenCacheFactory;
|
||||
import io.druid.java.util.common.io.Closer;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.guava.CloseQuietly;
|
||||
import io.druid.java.util.common.io.Closer;
|
||||
import io.druid.java.util.common.lifecycle.LifecycleStart;
|
||||
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -47,6 +46,7 @@ import java.util.Arrays;
|
|||
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.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -65,8 +65,8 @@ public class Announcer
|
|||
|
||||
private final List<Announceable> toAnnounce = Lists.newArrayList();
|
||||
private final List<Announceable> toUpdate = Lists.newArrayList();
|
||||
private final ConcurrentMap<String, PathChildrenCache> listeners = new MapMaker().makeMap();
|
||||
private final ConcurrentMap<String, ConcurrentMap<String, byte[]>> announcements = new MapMaker().makeMap();
|
||||
private final ConcurrentMap<String, PathChildrenCache> listeners = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, ConcurrentMap<String, byte[]>> announcements = new ConcurrentHashMap<>();
|
||||
private final List<String> parentsIBuilt = new CopyOnWriteArrayList<String>();
|
||||
|
||||
private boolean started = false;
|
||||
|
@ -200,7 +200,7 @@ public class Announcer
|
|||
}
|
||||
|
||||
// I don't have a watcher on this path yet, create a Map and start watching.
|
||||
announcements.putIfAbsent(parentPath, new MapMaker().<String, byte[]>makeMap());
|
||||
announcements.putIfAbsent(parentPath, new ConcurrentHashMap<>());
|
||||
|
||||
// Guaranteed to be non-null, but might be a map put in there by another thread.
|
||||
final ConcurrentMap<String, byte[]> finalSubPaths = announcements.get(parentPath);
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.curator.inventory;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.curator.cache.PathChildrenCacheFactory;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
|
@ -38,6 +37,7 @@ import org.apache.curator.utils.ZKPaths;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
@ -80,7 +80,7 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
|
|||
this.config = config;
|
||||
this.strategy = strategy;
|
||||
|
||||
this.containers = new MapMaker().makeMap();
|
||||
this.containers = new ConcurrentHashMap<>();
|
||||
this.uninitializedInventory = Sets.newConcurrentHashSet();
|
||||
|
||||
this.pathChildrenCacheExecutor = exec;
|
||||
|
|
|
@ -90,6 +90,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -112,7 +113,7 @@ public class AppenderatorImpl implements Appenderator
|
|||
private final IndexIO indexIO;
|
||||
private final IndexMerger indexMerger;
|
||||
private final Cache cache;
|
||||
private final Map<SegmentIdentifier, Sink> sinks = Maps.newConcurrentMap();
|
||||
private final Map<SegmentIdentifier, Sink> sinks = new ConcurrentHashMap<>();
|
||||
private final Set<SegmentIdentifier> droppingSinks = Sets.newConcurrentHashSet();
|
||||
private final VersionedIntervalTimeline<String, Sink> sinkTimeline = new VersionedIntervalTimeline<>(
|
||||
String.CASE_INSENSITIVE_ORDER
|
||||
|
|
|
@ -27,7 +27,6 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
@ -64,6 +63,7 @@ import org.joda.time.Period;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -80,7 +80,7 @@ public class AppenderatorPlumber implements Plumber
|
|||
private final SegmentPublisher segmentPublisher;
|
||||
private final SegmentHandoffNotifier handoffNotifier;
|
||||
private final Object handoffCondition = new Object();
|
||||
private final Map<Long, SegmentIdentifier> segments = Maps.newConcurrentMap();
|
||||
private final Map<Long, SegmentIdentifier> segments = new ConcurrentHashMap<>();
|
||||
private final Appenderator appenderator;
|
||||
|
||||
private volatile boolean shuttingDown = false;
|
||||
|
|
|
@ -20,16 +20,15 @@
|
|||
package io.druid.segment.realtime.firehose;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
|
||||
import io.druid.curator.discovery.ServiceAnnouncer;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.guice.annotations.RemoteChatHandler;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.server.initialization.ServerConfig;
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
|
@ -58,7 +57,7 @@ public class ServiceAnnouncingChatHandlerProvider implements ChatHandlerProvider
|
|||
this.node = node;
|
||||
this.serviceAnnouncer = serviceAnnouncer;
|
||||
this.serverConfig = serverConfig;
|
||||
this.handlers = Maps.newConcurrentMap();
|
||||
this.handlers = new ConcurrentHashMap<>();
|
||||
this.announcements = new ConcurrentSkipListSet<>();
|
||||
}
|
||||
|
||||
|
|
|
@ -21,8 +21,6 @@ package io.druid.segment.realtime.plumber;
|
|||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
import io.druid.client.ImmutableSegmentLoadInfo;
|
||||
import io.druid.client.coordinator.CoordinatorClient;
|
||||
import io.druid.concurrent.Execs;
|
||||
|
@ -34,6 +32,7 @@ import io.druid.server.coordination.DruidServerMetadata;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
@ -43,7 +42,7 @@ public class CoordinatorBasedSegmentHandoffNotifier implements SegmentHandoffNot
|
|||
{
|
||||
private static final Logger log = new Logger(CoordinatorBasedSegmentHandoffNotifier.class);
|
||||
|
||||
private final ConcurrentMap<SegmentDescriptor, Pair<Executor, Runnable>> handOffCallbacks = Maps.newConcurrentMap();
|
||||
private final ConcurrentMap<SegmentDescriptor, Pair<Executor, Runnable>> handOffCallbacks = new ConcurrentHashMap<>();
|
||||
private final CoordinatorClient coordinatorClient;
|
||||
private volatile ScheduledExecutorService scheduledExecutor;
|
||||
private final long pollDurationMillis;
|
||||
|
|
|
@ -29,7 +29,6 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
|
@ -88,6 +87,7 @@ import java.util.Arrays;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -108,7 +108,7 @@ public class RealtimePlumber implements Plumber
|
|||
private final SegmentPublisher segmentPublisher;
|
||||
private final SegmentHandoffNotifier handoffNotifier;
|
||||
private final Object handoffCondition = new Object();
|
||||
private final Map<Long, Sink> sinks = Maps.newConcurrentMap();
|
||||
private final Map<Long, Sink> sinks = new ConcurrentHashMap<>();
|
||||
private final VersionedIntervalTimeline<String, Sink> sinkTimeline = new VersionedIntervalTimeline<String, Sink>(
|
||||
String.CASE_INSENSITIVE_ORDER
|
||||
);
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.SettableFuture;
|
||||
|
@ -48,6 +47,7 @@ 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.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<SegmentZNode> availableZNodes = new ConcurrentSkipListSet<SegmentZNode>();
|
||||
private final Map<DataSegment, SegmentZNode> segmentLookup = Maps.newConcurrentMap();
|
||||
private final Map<DataSegment, SegmentZNode> segmentLookup = new ConcurrentHashMap<>();
|
||||
private final Function<DataSegment, DataSegment> segmentTransformer;
|
||||
|
||||
private final SegmentChangeRequestHistory changes = new SegmentChangeRequestHistory();
|
||||
|
|
|
@ -85,6 +85,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
|
@ -164,7 +165,7 @@ public class DruidCoordinator
|
|||
taskMaster,
|
||||
serviceAnnouncer,
|
||||
self,
|
||||
Maps.<String, LoadQueuePeon>newConcurrentMap(),
|
||||
new ConcurrentHashMap<>(),
|
||||
indexingServiceHelpers,
|
||||
factory,
|
||||
lookupCoordinatorManager,
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.junit.Test;
|
|||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
public class CostBalancerStrategyTest
|
||||
|
@ -110,7 +111,7 @@ public class CostBalancerStrategyTest
|
|||
{
|
||||
// Not using EasyMock as it hampers the performance of multithreads.
|
||||
DataSegment segment = new DataSegment(
|
||||
dataSource, interval, String.valueOf(index), Maps.<String, Object>newConcurrentMap(),
|
||||
dataSource, interval, String.valueOf(index), new ConcurrentHashMap<>(),
|
||||
Lists.<String>newArrayList(), Lists.<String>newArrayList(), null, 0, index * 100L
|
||||
);
|
||||
return segment;
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.junit.Test;
|
|||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
public class DiskNormalizedCostBalancerStrategyTest
|
||||
|
@ -108,7 +109,7 @@ public class DiskNormalizedCostBalancerStrategyTest
|
|||
{
|
||||
// Not using EasyMock as it hampers the performance of multithreads.
|
||||
DataSegment segment = new DataSegment(
|
||||
dataSource, interval, String.valueOf(index), Maps.<String, Object>newConcurrentMap(),
|
||||
dataSource, interval, String.valueOf(index), new ConcurrentHashMap<>(),
|
||||
Lists.<String>newArrayList(), Lists.<String>newArrayList(), null, 0, index * 100L
|
||||
);
|
||||
return segment;
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.client.DruidDataSource;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.client.ImmutableDruidDataSource;
|
||||
|
@ -71,6 +69,7 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Executors;
|
||||
|
@ -153,7 +152,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
|||
);
|
||||
loadQueuePeon.start();
|
||||
druidNode = new DruidNode("hey", "what", 1234, null, new ServerConfig());
|
||||
loadManagementPeons = new MapMaker().makeMap();
|
||||
loadManagementPeons = new ConcurrentHashMap<>();
|
||||
scheduledExecutorFactory = new ScheduledExecutorFactory()
|
||||
{
|
||||
@Override
|
||||
|
@ -457,7 +456,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
|||
{
|
||||
// Not using EasyMock as it hampers the performance of multithreads.
|
||||
DataSegment segment = new DataSegment(
|
||||
dataSource, interval, "dummy_version", Maps.<String, Object>newConcurrentMap(),
|
||||
dataSource, interval, "dummy_version", new ConcurrentHashMap<>(),
|
||||
Lists.<String>newArrayList(), Lists.<String>newArrayList(), null, 0, 0L
|
||||
);
|
||||
return segment;
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.server.coordinator.cost;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.java.util.common.DateTimes;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -31,6 +30,7 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
@ -184,7 +184,7 @@ public class SegmentsCostCacheTest
|
|||
dataSource,
|
||||
interval,
|
||||
UUID.randomUUID().toString(),
|
||||
Maps.<String, Object>newConcurrentMap(),
|
||||
new ConcurrentHashMap<>(),
|
||||
Lists.<String>newArrayList(),
|
||||
Lists.<String>newArrayList(),
|
||||
null,
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package io.druid.sql.avatica;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import com.metamx.metrics.AbstractMonitor;
|
||||
|
@ -32,6 +31,7 @@ import org.apache.calcite.avatica.metrics.MetricsSystem;
|
|||
import org.apache.calcite.avatica.metrics.Timer;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -40,8 +40,8 @@ public class AvaticaMonitor extends AbstractMonitor implements MetricsSystem
|
|||
{
|
||||
private static final Logger log = new Logger(AvaticaMonitor.class);
|
||||
|
||||
private final ConcurrentMap<String, AtomicLong> counters = Maps.newConcurrentMap();
|
||||
private final ConcurrentMap<String, Gauge<?>> gauges = Maps.newConcurrentMap();
|
||||
private final ConcurrentMap<String, AtomicLong> counters = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, Gauge<?>> gauges = new ConcurrentHashMap<>();
|
||||
|
||||
@Override
|
||||
public boolean doMonitor(final ServiceEmitter emitter)
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableMultimap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
|
@ -71,6 +70,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -138,7 +138,7 @@ public class DruidSchema extends AbstractSchema
|
|||
this.config = Preconditions.checkNotNull(config, "config");
|
||||
this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager");
|
||||
this.cacheExec = ScheduledExecutors.fixed(1, "DruidSchema-Cache-%d");
|
||||
this.tables = Maps.newConcurrentMap();
|
||||
this.tables = new ConcurrentHashMap<>();
|
||||
this.escalatingAuthenticator = authenticatorMapper.getEscalatingAuthenticator();
|
||||
|
||||
serverView.registerTimelineCallback(
|
||||
|
|
Loading…
Reference in New Issue