Use map.putIfAbsent() or map.computeIfAbsent() as appropriate instead of containsKey() + put() (#7764)

* https://github.com/apache/incubator-druid/issues/7316 Use Map.putIfAbsent() instead of containsKey() + put()

* fixing indentation

* Using map.computeIfAbsent() instead of map.putIfAbsent() where appropriate

* fixing checkstyle

* Changing the recommendation text

* Reverting auto changes made by IDE

* Implementing recommendation: A ConcurrentHashMap on which computeIfAbsent() is called should be assigned into variables of ConcurrentHashMap type, not ConcurrentMap

* Removing unused import
This commit is contained in:
Sashidhar Thallam 2019-06-14 21:29:36 +05:30 committed by Roman Leventov
parent 05346a9e0c
commit 3bee6adcf7
13 changed files with 374 additions and 393 deletions

View File

@ -306,6 +306,11 @@
<constraint name="x" maxCount="2147483647" within="" contains="" />
<constraint name="ImmutableMap" regexp="Immutable.*" within="" contains="" />
</searchConfiguration>
<searchConfiguration name="Use map.putIfAbsent(k,v) or map.computeIfAbsent(k,v) where appropriate instead of containsKey() + put(). If computing v is expensive or has side effects use map.computeIfAbsent() instead" created="1558868694225" text="if (!$m$.containsKey($k$)) {&#10; $m$.put($k$, $v$);&#10;}" recursive="false" caseInsensitive="true" type="JAVA">
<constraint name="m" within="" contains="" />
<constraint name="k" within="" contains="" />
<constraint name="v" within="" contains="" />
</searchConfiguration>
</inspection_tool>
<inspection_tool class="SimplifyStreamApiCallChains" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false">

View File

@ -261,9 +261,7 @@ public class S3DataSegmentMoverTest
@Override
public PutObjectResult putObject(String bucketName, String key, File file)
{
if (!storage.containsKey(bucketName)) {
storage.put(bucketName, new HashSet<>());
}
storage.putIfAbsent(bucketName, new HashSet<>());
storage.get(bucketName).add(key);
return new PutObjectResult();
}

View File

@ -307,9 +307,7 @@ public class DetermineHashedPartitionsJob implements Jobby
.getSegmentGranularity()
.bucket(DateTimes.utc(inputRow.getTimestampFromEpoch()));
if (!hyperLogLogs.containsKey(interval)) {
hyperLogLogs.put(interval, HyperLogLogCollector.makeLatestCollector());
}
hyperLogLogs.computeIfAbsent(interval, intv -> HyperLogLogCollector.makeLatestCollector());
} else {
final Optional<Interval> maybeInterval = config.getGranularitySpec()
.bucketInterval(DateTimes.utc(inputRow.getTimestampFromEpoch()));

View File

@ -50,9 +50,8 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
// Group by dataSource
final Map<String, Set<SegmentIdWithShardSpec>> identifiersByDataSource = new TreeMap<>();
for (SegmentIdWithShardSpec identifier : identifiers) {
if (!identifiersByDataSource.containsKey(identifier.getDataSource())) {
identifiersByDataSource.put(identifier.getDataSource(), new HashSet<>());
}
identifiersByDataSource.computeIfAbsent(identifier.getDataSource(), k -> new HashSet<>());
identifiersByDataSource.get(identifier.getDataSource()).add(identifier);
}

View File

@ -768,9 +768,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
}
if (determineNumPartitions) {
if (!hllCollectors.containsKey(interval)) {
hllCollectors.put(interval, Optional.of(HyperLogLogCollector.makeLatestCollector()));
}
hllCollectors.computeIfAbsent(interval, intv -> Optional.of(HyperLogLogCollector.makeLatestCollector()));
List<Object> groupKey = Rows.toGroupKey(
queryGranularity.bucketStart(inputRow.getTimestamp()).getMillis(),
@ -781,9 +779,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
} else {
// we don't need to determine partitions but we still need to determine intervals, so add an Optional.absent()
// for the interval and don't instantiate a HLL collector
if (!hllCollectors.containsKey(interval)) {
hllCollectors.put(interval, Optional.absent());
}
hllCollectors.putIfAbsent(interval, Optional.absent());
}
determinePartitionsMeters.incrementProcessed();
}

View File

@ -204,7 +204,6 @@ public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory<Input
segmentIds
);
try {
final List<TimelineObjectHolder<String, DataSegment>> timeLineSegments = getTimeline();
// Download all segments locally.
@ -216,9 +215,16 @@ public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory<Input
for (TimelineObjectHolder<String, DataSegment> holder : timeLineSegments) {
for (PartitionChunk<DataSegment> chunk : holder.getObject()) {
final DataSegment segment = chunk.getObject();
if (!segmentFileMap.containsKey(segment)) {
segmentFileMap.put(segment, segmentLoader.getSegmentFiles(segment));
segmentFileMap.computeIfAbsent(segment, k -> {
try {
return segmentLoader.getSegmentFiles(segment);
}
catch (SegmentLoadingException e) {
throw new RuntimeException(e);
}
});
}
}
@ -240,16 +246,14 @@ public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory<Input
Iterables.concat(
Iterables.transform(
timeLineSegments,
new Function<TimelineObjectHolder<String, DataSegment>, Iterable<WindowedStorageAdapter>>()
{
new Function<TimelineObjectHolder<String, DataSegment>, Iterable<WindowedStorageAdapter>>() {
@Override
public Iterable<WindowedStorageAdapter> apply(final TimelineObjectHolder<String, DataSegment> holder)
{
return
Iterables.transform(
holder.getObject(),
new Function<PartitionChunk<DataSegment>, WindowedStorageAdapter>()
{
new Function<PartitionChunk<DataSegment>, WindowedStorageAdapter>() {
@Override
public WindowedStorageAdapter apply(final PartitionChunk<DataSegment> input)
{
@ -282,10 +286,6 @@ public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory<Input
final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser);
return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter);
}
catch (SegmentLoadingException e) {
throw new RuntimeException(e);
}
}
private long jitter(long input)
{
@ -508,13 +508,14 @@ public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory<Input
// segments to olders.
// timelineSegments are sorted in order of interval
int index = 0;
int[] index = {0};
for (TimelineObjectHolder<String, DataSegment> timelineHolder : Lists.reverse(timelineSegments)) {
for (PartitionChunk<DataSegment> chunk : timelineHolder.getObject()) {
for (String metric : chunk.getObject().getMetrics()) {
if (!uniqueMetrics.containsKey(metric)) {
uniqueMetrics.put(metric, index++);
uniqueMetrics.computeIfAbsent(metric, k -> {
return index[0]++;
}
);
}
}
}

View File

@ -84,7 +84,6 @@ import java.util.Set;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
@ -108,7 +107,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 ConcurrentMap<String, ForkingTaskRunnerWorkItem> tasks = new ConcurrentHashMap<>();
private final ConcurrentHashMap<String, ForkingTaskRunnerWorkItem> tasks = new ConcurrentHashMap<>();
private volatile boolean stopping = false;
@ -214,14 +213,12 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
public ListenableFuture<TaskStatus> run(final Task task)
{
synchronized (tasks) {
if (!tasks.containsKey(task.getId())) {
tasks.put(
task.getId(),
tasks.computeIfAbsent(
task.getId(), k ->
new ForkingTaskRunnerWorkItem(
task,
exec.submit(
new Callable<TaskStatus>()
{
new Callable<TaskStatus>() {
@Override
public TaskStatus call()
{
@ -516,7 +513,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
)
)
);
}
saveRunningTasks();
return tasks.get(task.getId()).getResult();
}

View File

@ -166,9 +166,7 @@ public class SQLMetadataSupervisorManager implements MetadataSupervisorManager
{
try {
String specId = pair.lhs;
if (!retVal.containsKey(specId)) {
retVal.put(specId, new ArrayList<>());
}
retVal.putIfAbsent(specId, new ArrayList<>());
retVal.get(specId).add(pair.rhs);
return retVal;

View File

@ -549,9 +549,7 @@ public class DataSourcesResource
continue;
}
if (!tierDistinctSegments.containsKey(tier)) {
tierDistinctSegments.put(tier, new HashSet<>());
}
tierDistinctSegments.computeIfAbsent(tier, k -> new HashSet<>());
long dataSourceSegmentSize = 0;
for (DataSegment dataSegment : druidDataSource.getSegments()) {

View File

@ -2198,9 +2198,7 @@ public class CachingClusteredClientTest
serverExpectationList.add(serverExpectations);
for (int j = 0; j < numChunks; ++j) {
DruidServer lastServer = servers[random.nextInt(servers.length)];
if (!serverExpectations.containsKey(lastServer)) {
serverExpectations.put(lastServer, new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class)));
}
serverExpectations.computeIfAbsent(lastServer, server -> new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class)));
DataSegment mockSegment = makeMock(mocks, DataSegment.class);
ServerExpectation<Object> expectation = new ServerExpectation<>(

View File

@ -428,9 +428,7 @@ public class StreamAppenderatorDriverTest extends EasyMockSupport
synchronized (counters) {
DateTime dateTimeTruncated = granularity.bucketStart(row.getTimestamp());
final long timestampTruncated = dateTimeTruncated.getMillis();
if (!counters.containsKey(timestampTruncated)) {
counters.put(timestampTruncated, new AtomicInteger());
}
counters.putIfAbsent(timestampTruncated, new AtomicInteger());
final int partitionNum = counters.get(timestampTruncated).getAndIncrement();
return new SegmentIdWithShardSpec(
dataSource,

View File

@ -122,9 +122,7 @@ public class SqlLifecycle
if (queryContext != null) {
newContext.putAll(queryContext);
}
if (!newContext.containsKey(PlannerContext.CTX_SQL_QUERY_ID)) {
newContext.put(PlannerContext.CTX_SQL_QUERY_ID, UUID.randomUUID().toString());
}
newContext.computeIfAbsent(PlannerContext.CTX_SQL_QUERY_ID, k -> UUID.randomUUID().toString());
return newContext;
}

View File

@ -78,9 +78,7 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C
)
{
final Segment segment = new QueryableIndexSegment(index, descriptor.getId());
if (!timelines.containsKey(descriptor.getDataSource())) {
timelines.put(descriptor.getDataSource(), new VersionedIntervalTimeline<>(Ordering.natural()));
}
timelines.computeIfAbsent(descriptor.getDataSource(), datasource -> new VersionedIntervalTimeline<>(Ordering.natural()));
final VersionedIntervalTimeline<String, Segment> timeline = timelines.get(descriptor.getDataSource());
timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment));