mirror of https://github.com/apache/druid.git
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:
parent
05346a9e0c
commit
3bee6adcf7
|
@ -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$)) { $m$.put($k$, $v$); }" 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">
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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]++;
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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<>(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
|
|
Loading…
Reference in New Issue