Skip to content

Commit

Permalink
Instantiate collection builder with factory method
Browse files Browse the repository at this point in the history
The immutable collection classes provide `Builder` which can be
instantiated with constructor or static factory method. The latter is
preferred, and now also required.
  • Loading branch information
findepi committed Feb 25, 2022
1 parent 14f563a commit 03a4d53
Show file tree
Hide file tree
Showing 168 changed files with 233 additions and 187 deletions.
46 changes: 46 additions & 0 deletions .mvn/modernizer/violations.xml
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,52 @@
<comment>Use buildOrThrow() instead, as it makes it clear that it will throw on duplicated values</comment>
</violation>

<violation>
<name>com/google/common/collect/ImmutableBiMap$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use builder() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableList$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use builder() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableMap$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use builder() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableMultimap$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use builder() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableMultiset$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use builder() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableSet$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use builder() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableSortedMap$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use orderedBy() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableSortedSet$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use orderedBy() static factory method instead</comment>
</violation>
<violation>
<name>com/google/common/collect/ImmutableTable$Builder."&lt;init&gt;":()V</name>
<version>1.8</version>
<comment>Use builder() static factory method instead</comment>
</violation>

<violation>
<name>com/google/common/cache/CacheBuilder.build:()Lcom/google/common/cache/Cache;</name>
<version>1.8</version>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ private synchronized void updateAllNodes(AllNodes allNodes)
currentCount = Sets.difference(allNodes.getActiveNodes(), allNodes.getActiveCoordinators()).size();
}

ImmutableList.Builder<SettableFuture<Void>> listenersBuilder = new ImmutableList.Builder<>();
ImmutableList.Builder<SettableFuture<Void>> listenersBuilder = ImmutableList.builder();
while (!futuresQueue.isEmpty()) {
MinNodesFuture minNodesFuture = futuresQueue.peek();
if (minNodesFuture == null || minNodesFuture.getExecutionMinCount() > currentCount) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -361,7 +361,7 @@ public void finish()
return;
}
finished = true;
ImmutableMap.Builder<DynamicFilterId, Domain> domainsBuilder = new ImmutableMap.Builder<>();
ImmutableMap.Builder<DynamicFilterId, Domain> domainsBuilder = ImmutableMap.builder();
if (valueSets == null) {
if (minValues == null) {
// there were too many rows to collect min/max range
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ private class ResultIterator

ResultIterator()
{
ImmutableList.Builder<Type> sourceTypesBuilders = new ImmutableList.Builder<Type>().addAll(sourceTypes);
ImmutableList.Builder<Type> sourceTypesBuilders = ImmutableList.<Type>builder().addAll(sourceTypes);
if (produceRanking) {
sourceTypesBuilders.add(BIGINT);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ private class ResultIterator

ResultIterator()
{
ImmutableList.Builder<Type> sourceTypesBuilders = new ImmutableList.Builder<Type>().addAll(sourceTypes);
ImmutableList.Builder<Type> sourceTypesBuilders = ImmutableList.<Type>builder().addAll(sourceTypes);
if (produceRowNumber) {
sourceTypesBuilders.add(BIGINT);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ public void forEachBucket(BucketConsumer<K> consumer)
@VisibleForTesting
public Map<K, Long> getBuckets()
{
ImmutableMap.Builder<K, Long> buckets = new ImmutableMap.Builder<>();
ImmutableMap.Builder<K, Long> buckets = ImmutableMap.builder();
forEachBucket(buckets::put);

return buckets.buildOrThrow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ public List<BasicQueryInfo> getAllQueryInfo(@QueryParam("state") String stateFil
List<BasicQueryInfo> queries = dispatchManager.getQueries();
queries = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queries, accessControl);

ImmutableList.Builder<BasicQueryInfo> builder = new ImmutableList.Builder<>();
ImmutableList.Builder<BasicQueryInfo> builder = ImmutableList.builder();
for (BasicQueryInfo queryInfo : queries) {
if (stateFilter == null || queryInfo.getState() == expectedState) {
builder.add(queryInfo);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ public List<TrimmedBasicQueryInfo> getAllQueryInfo(@QueryParam("state") String s
List<BasicQueryInfo> queries = dispatchManager.getQueries();
queries = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queries, accessControl);

ImmutableList.Builder<TrimmedBasicQueryInfo> builder = new ImmutableList.Builder<>();
ImmutableList.Builder<TrimmedBasicQueryInfo> builder = ImmutableList.builder();
for (BasicQueryInfo queryInfo : queries) {
if (stateFilter == null || queryInfo.getState() == expectedState) {
builder.add(new TrimmedBasicQueryInfo(queryInfo));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,10 +162,10 @@ public static Optional<PlanNode> pushAggregationIntoTableScan(
AggregationApplicationResult<TableHandle> result = aggregationPushdownResult.get();

// The new scan outputs should be the symbols associated with grouping columns plus the symbols associated with aggregations.
ImmutableList.Builder<Symbol> newScanOutputs = new ImmutableList.Builder<>();
ImmutableList.Builder<Symbol> newScanOutputs = ImmutableList.builder();
newScanOutputs.addAll(tableScan.getOutputSymbols());

ImmutableBiMap.Builder<Symbol, ColumnHandle> newScanAssignments = new ImmutableBiMap.Builder<>();
ImmutableBiMap.Builder<Symbol, ColumnHandle> newScanAssignments = ImmutableBiMap.builder();
newScanAssignments.putAll(tableScan.getAssignments());

Map<String, Symbol> variableMappings = new HashMap<>();
Expand Down Expand Up @@ -221,7 +221,7 @@ private static AggregateFunction toAggregateFunction(Metadata metadata, Context
String canonicalName = metadata.getFunctionMetadata(aggregation.getResolvedFunction()).getCanonicalName();
BoundSignature signature = aggregation.getResolvedFunction().getSignature();

ImmutableList.Builder<ConnectorExpression> arguments = new ImmutableList.Builder<>();
ImmutableList.Builder<ConnectorExpression> arguments = ImmutableList.builder();
for (int i = 0; i < aggregation.getArguments().size(); i++) {
SymbolReference argument = (SymbolReference) aggregation.getArguments().get(i);
arguments.add(new Variable(argument.getName(), signature.getArgumentTypes().get(i)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("event.max-output-stage-size", "512kB")
.buildOrThrow();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ public void testExplicitPropertyMappings()
Path config1 = Files.createTempFile(null, null);
Path config2 = Files.createTempFile(null, null);

Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("event-listener.config-files", config1.toString() + "," + config2.toString())
.buildOrThrow();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("enable-dynamic-filtering", "false")
.put("enable-coordinator-dynamic-filters-distribution", "false")
.put("enable-large-dynamic-filters", "true")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("failure-injection.request-timeout", "12m")
.put("failure-injection.expiration-period", "7m")
.buildOrThrow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("node-scheduler.policy", "topology")
.put("node-scheduler.min-candidates", "11")
.put("node-scheduler.include-coordinator", "false")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("query.client.timeout", "10s")
.put("query.min-expire-age", "30s")
.put("query.max-history", "10")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ public void testDefaults()
public void testExplicitPropertyMappings()
{
int processorCount = DEFAULT_PROCESSOR_COUNT == 32 ? 16 : 32;
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("task.initial-splits-per-node", "1")
.put("task.split-concurrency-adjustment-interval", "1s")
.put("task.status-refresh-max-wait", "2s")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public void testDefaults()
public void testExplicitPropertyMappings()
throws IOException
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("node-scheduler.network-topology.subnet.cidr-prefix-lengths", "24,26")
.put("node-scheduler.network-topology.subnet.ip-address-protocol", "IPv6")
.buildOrThrow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("node-scheduler.network-topology.type", "FILE")
.put("node-scheduler.network-topology.segments", "rack,machine")
.buildOrThrow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public void testExplicitPropertyMappings()
{
Path networkTopologyFile = Files.createTempFile(null, null);

Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("node-scheduler.network-topology.file", networkTopologyFile.toString())
.put("node-scheduler.network-topology.refresh-period", "27m")
.buildOrThrow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("query.low-memory-killer.policy", "none")
.put("query.low-memory-killer.delay", "20s")
.put("query.max-memory", "2GB")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("query.max-memory-per-node", "1GB")
.put("query.max-memory-per-task", "200MB")
.put("memory.heap-headroom-per-node", "1GB")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ public void testInformationSchemaPredicatePushdown()
{
TransactionId transactionId = transactionManager.beginTransaction(false);

ImmutableMap.Builder<ColumnHandle, Domain> domains = new ImmutableMap.Builder<>();
ImmutableMap.Builder<ColumnHandle, Domain> domains = ImmutableMap.builder();
domains.put(new InformationSchemaColumnHandle("table_schema"), Domain.singleValue(VARCHAR, Slices.utf8Slice("test_schema")));
domains.put(new InformationSchemaColumnHandle("table_name"), Domain.singleValue(VARCHAR, Slices.utf8Slice("test_view")));
Constraint constraint = new Constraint(TupleDomain.withColumnDomains(domains.buildOrThrow()));
Expand Down Expand Up @@ -133,7 +133,7 @@ public void testInformationSchemaPredicatePushdownWithoutSchemaPredicate()
TransactionId transactionId = transactionManager.beginTransaction(false);

// predicate without schema predicates should cause schemas to be enumerated when table predicates are present
ImmutableMap.Builder<ColumnHandle, Domain> domains = new ImmutableMap.Builder<>();
ImmutableMap.Builder<ColumnHandle, Domain> domains = ImmutableMap.builder();
domains.put(new InformationSchemaColumnHandle("table_name"), Domain.singleValue(VARCHAR, Slices.utf8Slice("test_view")));
Constraint constraint = new Constraint(TupleDomain.withColumnDomains(domains.buildOrThrow()));

Expand All @@ -157,7 +157,7 @@ public void testInformationSchemaPredicatePushdownWithoutTablePredicate()
TransactionId transactionId = transactionManager.beginTransaction(false);

// predicate without table name predicates should not cause table level prefixes to be evaluated
ImmutableMap.Builder<ColumnHandle, Domain> domains = new ImmutableMap.Builder<>();
ImmutableMap.Builder<ColumnHandle, Domain> domains = ImmutableMap.builder();
domains.put(new InformationSchemaColumnHandle("table_schema"), Domain.singleValue(VARCHAR, Slices.utf8Slice("test_schema")));
Constraint constraint = new Constraint(TupleDomain.withColumnDomains(domains.buildOrThrow()));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("catalog.config-dir", "/foo")
.put("catalog.disabled-catalogs", "abc,xyz")
.buildOrThrow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("exchange.max-buffer-size", "1GB")
.put("exchange.concurrent-request-multiplier", "13")
.put("exchange.min-error-duration", "13s")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ private void testCustomAggregation(Long[] values, int n)
{
PriorityQueue<Long> heap = new PriorityQueue<>(n);
Arrays.stream(values).filter(Objects::nonNull).forEach(heap::add);
ImmutableList.Builder<List<Long>> expected = new ImmutableList.Builder<>();
ImmutableList.Builder<List<Long>> expected = ImmutableList.builder();
for (int i = heap.size() - 1; i >= 0; i--) {
expected.add(ImmutableList.of(heap.remove()));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public void testAllRegistered()

private static List<Type> getTypes()
{
return new ImmutableList.Builder<Type>()
return ImmutableList.<Type>builder()
.addAll(STANDARD_TYPES)
.add(VARCHAR)
.add(createDecimalType(1))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ public void testExplicitPropertyMappings()
Path config1 = Files.createTempFile(null, null);
Path config2 = Files.createTempFile(null, null);

Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("access-control.config-files", config1.toString() + "," + config2.toString())
.buildOrThrow();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("discovery-server.enabled", "false")
.buildOrThrow();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("failure-detector.expiration-grace-interval", "5m")
.put("failure-detector.warmup-interval", "60s")
.put("failure-detector.heartbeat-interval", "10s")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public void testExplicitPropertyMappings()
Path keystoreFile = Files.createTempFile(null, null);
Path truststoreFile = Files.createTempFile(null, null);

Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("internal-communication.shared-secret", "secret")
.put("internal-communication.http2.enabled", "true")
.put("internal-communication.https.required", "true")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("jmx.base-name", "my.stuff")
.buildOrThrow();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("protocol.v1.alternate-header-name", "taco")
.put("protocol.v1.prepared-statement-compression.length-threshold", "412")
.put("protocol.v1.prepared-statement-compression.min-gain", "0")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public void testDefaults()
@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
Map<String, String> properties = ImmutableMap.<String, String>builder()
.put("coordinator", "false")
.put("experimental.concurrent-startup", "true")
.put("http.include-exception-in-response", "false")
Expand Down
Loading

0 comments on commit 03a4d53

Please sign in to comment.