Skip to content

Commit

Permalink
Add a reference to the comment about guarded computeIfAbsent() optimi…
Browse files Browse the repository at this point in the history
…zation; IdentityHashMap optimization
  • Loading branch information
leventov committed Jan 29, 2019
1 parent 9d48d18 commit 14307c3
Show file tree
Hide file tree
Showing 58 changed files with 184 additions and 239 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,7 @@ public void setup() throws IOException

factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -407,7 +407,7 @@ public String getFormatString()
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
)
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -495,7 +495,7 @@ public String getFormatString()
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
)
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ public static <T, QueryType extends Query<T>> QueryRunner<T> makeQueryRunner(
);
}

public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator()
public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator()
{
return new IntervalChunkingQueryRunnerDecorator(null, null, null) {
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -373,7 +373,7 @@ public void setup() throws IOException
new SearchStrategySelector(Suppliers.ofInstance(config)),
new SearchQueryQueryToolChest(
config,
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,7 @@ public void setup() throws IOException
factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
JSON_MAPPER,
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator(),
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator(),
selectConfigSupplier
),
new SelectQueryEngine(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ public void setup() throws IOException

factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()
),
new TimeseriesQueryEngine(),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
);
}
Expand Down Expand Up @@ -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
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public void testMakePostComputeManipulatorFn()
.put(
TimeseriesQuery.class,
new TimeseriesQueryQueryToolChest(
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
)
.build()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public int getNumThreads()
strategySelector,
new GroupByQueryQueryToolChest(
strategySelector,
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
)
);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ public static Iterable<Object[]> constructorFeeder() throws IOException
SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory(
new SelectQueryQueryToolChest(
new DefaultObjectMapper(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(),
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator(),
selectConfigSupplier
),
new SelectQueryEngine(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public static Iterable<Object[]> constructorFeeder()
defaultPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
Expand All @@ -91,7 +91,7 @@ public static Iterable<Object[]> constructorFeeder()
customPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public static Iterable<Object[]> constructorFeeder()
defaultPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
Expand All @@ -91,7 +91,7 @@ public static Iterable<Object[]> constructorFeeder()
customPool,
new TopNQueryQueryToolChest(
new TopNQueryConfig(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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()
Expand Down Expand Up @@ -506,6 +509,7 @@ public Entry scheduleAndWait(ExtractionNamespace namespace, long waitForFirstRun

public <T extends ExtractionNamespace> Entry schedule(final T namespace)
{
@SuppressWarnings("unchecked")
final CacheGenerator<T> generator = (CacheGenerator<T>) namespaceGeneratorMap.get(namespace.getClass());
if (generator == null) {
throw new ISE("Cannot find generator for namespace [%s]", namespace);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ private Sequence<Result<TopNResultValue>> assertExpectedResults(
{
final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(
new TopNQueryConfig(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()
);
final QueryRunner<Result<TopNResultValue>> mergeRunner = chest.mergeResults(runner);
final Sequence<Result<TopNResultValue>> retval = mergeRunner.run(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ public final class Counters
public static <K> int incrementAndGetInt(ConcurrentHashMap<K, AtomicInteger> 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());
Expand All @@ -38,6 +39,7 @@ public static <K> int incrementAndGetInt(ConcurrentHashMap<K, AtomicInteger> cou
public static <K> long incrementAndGetLong(ConcurrentHashMap<K, AtomicLong> 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());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import com.google.inject.Inject;

import java.util.IdentityHashMap;
import java.util.Map;

/**
Expand All @@ -30,11 +31,11 @@ public class DefaultQueryRunnerFactoryConglomerate implements QueryRunnerFactory
private final Map<Class<? extends Query>, QueryRunnerFactory> factories;

@Inject
public DefaultQueryRunnerFactoryConglomerate(
Map<Class<? extends Query>, QueryRunnerFactory> factories
)
public DefaultQueryRunnerFactoryConglomerate(Map<Class<? extends Query>, 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import com.google.inject.Inject;

import java.util.IdentityHashMap;
import java.util.Map;

/**
Expand All @@ -30,11 +31,11 @@ public class MapQueryToolChestWarehouse implements QueryToolChestWarehouse
private final Map<Class<? extends Query>, QueryToolChest> toolchests;

@Inject
public MapQueryToolChestWarehouse(
Map<Class<? extends Query>, QueryToolChest> toolchests
)
public MapQueryToolChestWarehouse(Map<Class<? extends Query>, 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -34,6 +36,34 @@
@ExtensionPoint
public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultType>>
{
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
Expand Down
Loading

0 comments on commit 14307c3

Please sign in to comment.