Skip to content

Commit

Permalink
[cleanup][broker][Modernizer] fix violations in pulsar-broker (#17172)
Browse files Browse the repository at this point in the history
Master Issue: #12271 #16991 

### Motivation

Apply Maven Modernizer plugin to enforce we move away from legacy APIs.

### Modifications

- set `failOnViolations`=true
- fix violations except `broker` and `client` packages
  • Loading branch information
youzipi authored Aug 24, 2022
1 parent e4723a7 commit 05a2ea8
Show file tree
Hide file tree
Showing 11 changed files with 45 additions and 47 deletions.
7 changes: 5 additions & 2 deletions pulsar-broker/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -429,9 +429,12 @@
<groupId>org.gaul</groupId>
<artifactId>modernizer-maven-plugin</artifactId>
<configuration>
<failOnViolations>false</failOnViolations>
<failOnViolations>true</failOnViolations>
<javaVersion>17</javaVersion>
<violationLogLevel>warn</violationLogLevel>
<ignorePackages>
<ignorePackage>org.apache.pulsar.broker</ignorePackage>
<ignorePackage>org.apache.pulsar.client</ignorePackage>
</ignorePackages>
</configuration>
<executions>
<execution>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
*/
package org.apache.pulsar;

import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static org.apache.commons.lang3.StringUtils.isBlank;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
Expand All @@ -37,6 +36,7 @@
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Date;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.common.component.ComponentStarter;
Expand Down Expand Up @@ -240,8 +240,8 @@ && isBlank(starterArguments.bookieConfigFile)) {

// init bookie server
if (starterArguments.runBookie) {
checkNotNull(bookieConfig, "No ServerConfiguration for Bookie");
checkNotNull(bookieStatsProvider, "No Stats Provider for Bookie");
Objects.requireNonNull(bookieConfig, "No ServerConfiguration for Bookie");
Objects.requireNonNull(bookieStatsProvider, "No Stats Provider for Bookie");
bookieServer = org.apache.bookkeeper.server.Main
.buildBookieServer(new BookieConfiguration(bookieConfig));
} else {
Expand All @@ -250,7 +250,7 @@ && isBlank(starterArguments.bookieConfigFile)) {

// init bookie AutorecoveryMain
if (starterArguments.runBookieAutoRecovery) {
checkNotNull(bookieConfig, "No ServerConfiguration for Bookie Autorecovery");
Objects.requireNonNull(bookieConfig, "No ServerConfiguration for Bookie Autorecovery");
autoRecoveryMain = new AutoRecoveryMain(bookieConfig);
} else {
autoRecoveryMain = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,10 @@
package org.apache.pulsar.common.naming;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import com.google.common.base.Objects;
import com.google.common.collect.BoundType;
import com.google.common.collect.Range;
import java.util.Objects;

public class NamespaceBundle implements ServiceUnitId, Comparable<NamespaceBundle> {
private final NamespaceName nsname;
Expand All @@ -36,8 +35,8 @@ public class NamespaceBundle implements ServiceUnitId, Comparable<NamespaceBundl
private final String bundleRange;

public NamespaceBundle(NamespaceName nsname, Range<Long> keyRange, NamespaceBundleFactory factory) {
this.nsname = checkNotNull(nsname);
this.keyRange = checkNotNull(keyRange);
this.nsname = Objects.requireNonNull(nsname);
this.keyRange = Objects.requireNonNull(keyRange);
checkArgument(this.keyRange.lowerBoundType().equals(BoundType.CLOSED),
"Invalid hash range. Lower Endpoint has to be inclusive");
checkArgument(
Expand All @@ -47,7 +46,7 @@ public NamespaceBundle(NamespaceName nsname, Range<Long> keyRange, NamespaceBund
&& this.keyRange.upperBoundType().equals(BoundType.OPEN)),
"Invalid hash range. Upper Endpoint should be exclusive unless it is 0xffffffff");
checkArgument(!this.keyRange.isEmpty(), "Cannot create bundle object for an empty key range");
this.factory = checkNotNull(factory);
this.factory = Objects.requireNonNull(factory);
this.key = getKey(this.nsname, this.keyRange);
this.bundleRange = String.format("0x%08x_0x%08x", keyRange.lowerEndpoint(), keyRange.upperEndpoint());
}
Expand Down Expand Up @@ -94,7 +93,7 @@ public int compareTo(NamespaceBundle other) {

@Override
public int hashCode() {
return Objects.hashCode(nsname,
return Objects.hash(nsname,
keyRange.lowerEndpoint(), keyRange.lowerBoundType(),
keyRange.upperEndpoint(), keyRange.upperBoundType());
}
Expand All @@ -103,11 +102,11 @@ public int hashCode() {
public boolean equals(Object other) {
if (other instanceof NamespaceBundle) {
NamespaceBundle obj = (NamespaceBundle) other;
return Objects.equal(this.nsname, obj.nsname)
&& (Objects.equal(this.keyRange.lowerEndpoint(), obj.keyRange.lowerEndpoint())
&& Objects.equal(this.keyRange.lowerBoundType(), obj.keyRange.lowerBoundType())
&& Objects.equal(this.keyRange.upperEndpoint(), obj.keyRange.upperEndpoint())
&& Objects.equal(this.keyRange.upperBoundType(), obj.keyRange.upperBoundType()));
return Objects.equals(this.nsname, obj.nsname)
&& (Objects.equals(this.keyRange.lowerEndpoint(), obj.keyRange.lowerEndpoint())
&& Objects.equals(this.keyRange.lowerBoundType(), obj.keyRange.lowerBoundType())
&& Objects.equals(this.keyRange.upperEndpoint(), obj.keyRange.upperEndpoint())
&& Objects.equals(this.keyRange.upperBoundType(), obj.keyRange.upperBoundType()));
}
return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,24 +19,24 @@
package org.apache.pulsar.common.naming;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.pulsar.common.policies.data.Policies.FIRST_BOUNDARY;
import static org.apache.pulsar.common.policies.data.Policies.LAST_BOUNDARY;
import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
import com.github.benmanes.caffeine.cache.Caffeine;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
import com.google.common.base.Splitter;
import com.google.common.collect.BoundType;
import com.google.common.collect.Range;
import com.google.common.hash.HashFunction;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.SortedSet;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -288,7 +288,7 @@ public CompletableFuture<NamespaceBundle> getFullBundleAsync(NamespaceName fqnn)
}

public long getLongHashCode(String name) {
return this.hashFunc.hashString(name, Charsets.UTF_8).padToLong();
return this.hashFunc.hashString(name, StandardCharsets.UTF_8).padToLong();
}

public NamespaceBundles getBundles(NamespaceName nsname, BundlesData bundleData) {
Expand Down Expand Up @@ -324,8 +324,8 @@ public CompletableFuture<Pair<NamespaceBundles, List<NamespaceBundle>>> splitBun
"The given fixed keys must between the key range of the %s bundle", targetBundle);
argNumBundles = splitBoundaries.size() + 1;
}
checkNotNull(targetBundle, "can't split null bundle");
checkNotNull(targetBundle.getNamespaceObject(), "namespace must be present");
Objects.requireNonNull(targetBundle, "can't split null bundle");
Objects.requireNonNull(targetBundle.getNamespaceObject(), "namespace must be present");
NamespaceName nsname = targetBundle.getNamespaceObject();

final int numBundles = argNumBundles;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,18 +19,16 @@
package org.apache.pulsar.common.naming;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static java.lang.String.format;
import static org.apache.pulsar.common.policies.data.Policies.FIRST_BOUNDARY;
import static org.apache.pulsar.common.policies.data.Policies.LAST_BOUNDARY;
import com.google.common.base.Objects;
import com.google.common.collect.BoundType;
import com.google.common.collect.Lists;
import com.google.common.collect.Range;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.SortedSet;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -64,13 +62,13 @@ public NamespaceBundles(NamespaceName nsname, NamespaceBundleFactory factory,
NamespaceBundles(NamespaceName nsname, NamespaceBundleFactory factory,
Optional<Pair<LocalPolicies, Long>> localPolicies, long[] partitions) {
// check input arguments
this.nsname = checkNotNull(nsname);
this.factory = checkNotNull(factory);
this.nsname = Objects.requireNonNull(nsname);
this.factory = Objects.requireNonNull(factory);
this.localPolicies = localPolicies;
checkArgument(partitions.length > 0, "Can't create bundles w/o partition boundaries");

// calculate bundles based on partition boundaries
this.bundles = Lists.newArrayList();
this.bundles = new ArrayList<>();
fullBundle = new NamespaceBundle(nsname,
Range.range(FULL_LOWER_BOUND, BoundType.CLOSED, FULL_UPPER_BOUND, BoundType.CLOSED), factory);

Expand Down Expand Up @@ -131,7 +129,7 @@ protected NamespaceBundle getBundle(long hash) {
}

private static long[] convertPartitions(SortedSet<Long> partitionsSet) {
checkNotNull(partitionsSet);
Objects.requireNonNull(partitionsSet);
long[] partitions = new long[partitionsSet.size()];
int idx = 0;
for (long p : partitionsSet) {
Expand All @@ -143,14 +141,14 @@ private static long[] convertPartitions(SortedSet<Long> partitionsSet) {

@Override
public int hashCode() {
return Objects.hashCode(nsname, bundles);
return Objects.hash(nsname, bundles);
}

@Override
public boolean equals(Object obj) {
if (obj != null && obj instanceof NamespaceBundles) {
NamespaceBundles other = (NamespaceBundles) obj;
return (Objects.equal(this.nsname, other.nsname) && Objects.equal(this.bundles, other.bundles));
return (Objects.equals(this.nsname, other.nsname) && Objects.equals(this.bundles, other.bundles));
}
return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,15 @@
*/
package org.apache.pulsar.common.naming;

import com.google.common.collect.Maps;
import java.util.Map;
import java.util.TreeMap;
import org.apache.bookkeeper.mledger.ManagedLedgerInfo;

public class PartitionedManagedLedgerInfo extends ManagedLedgerInfo {

public Map<String, ManagedLedgerInfo> partitions;

public PartitionedManagedLedgerInfo() {
partitions = Maps.newTreeMap();
partitions = new TreeMap<>();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
*/
package org.apache.pulsar.compaction;

import com.google.common.collect.ImmutableMap;
import io.netty.buffer.ByteBuf;
import java.io.IOException;
import java.time.Duration;
Expand Down Expand Up @@ -201,12 +200,12 @@ private CompletableFuture<Long> phaseTwoSeekThenLoop(RawReader reader, MessageId

reader.seekAsync(from).thenCompose((v) -> {
Semaphore outstanding = new Semaphore(MAX_OUTSTANDING);
CompletableFuture<Void> loopPromise = new CompletableFuture<Void>();
CompletableFuture<Void> loopPromise = new CompletableFuture<>();
phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, loopPromise);
return loopPromise;
}).thenCompose((v) -> closeLedger(ledger))
.thenCompose((v) -> reader.acknowledgeCumulativeAsync(lastReadId,
ImmutableMap.of(COMPACTED_TOPIC_LEDGER_PROPERTY, ledger.getId())))
Map.of(COMPACTED_TOPIC_LEDGER_PROPERTY, ledger.getId())))
.whenComplete((res, exception) -> {
if (exception != null) {
deleteLedger(bk, ledger).whenComplete((res2, exception2) -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.ParameterException;
import com.beust.jcommander.Parameters;
import com.google.common.base.Charsets;
import io.jsonwebtoken.Claims;
import io.jsonwebtoken.Jwt;
import io.jsonwebtoken.Jwts;
Expand All @@ -35,6 +34,7 @@
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.security.Key;
Expand Down Expand Up @@ -194,7 +194,7 @@ public void run() throws Exception {
BufferedReader r = new BufferedReader(new InputStreamReader(System.in));
token = r.readLine();
} else if (tokenFile != null) {
token = new String(Files.readAllBytes(Paths.get(tokenFile)), Charsets.UTF_8);
token = new String(Files.readAllBytes(Paths.get(tokenFile)), StandardCharsets.UTF_8);
} else if (System.getenv("TOKEN") != null) {
token = System.getenv("TOKEN");
} else {
Expand Down Expand Up @@ -259,7 +259,7 @@ public void run() throws Exception {
BufferedReader r = new BufferedReader(new InputStreamReader(System.in));
token = r.readLine();
} else if (tokenFile != null) {
token = new String(Files.readAllBytes(Paths.get(tokenFile)), Charsets.UTF_8);
token = new String(Files.readAllBytes(Paths.get(tokenFile)), StandardCharsets.UTF_8);
} else if (System.getenv("TOKEN") != null) {
token = System.getenv("TOKEN");
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
import com.google.common.collect.BoundType;
import com.google.common.collect.Range;
import com.google.common.collect.Sets;
import com.google.common.hash.Hashing;
import java.lang.reflect.Field;
import java.util.ArrayList;
Expand All @@ -37,6 +36,7 @@
import java.util.List;
import java.util.Optional;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.PulsarService;
Expand Down Expand Up @@ -108,7 +108,7 @@ private NamespaceBundleFactory getNamespaceBundleFactory() {

@Test
public void testFindBundle() throws Exception {
SortedSet<Long> partitions = Sets.newTreeSet();
SortedSet<Long> partitions = new TreeSet<>();
partitions.add(0L);
partitions.add(0x40000000L);
partitions.add(0xa0000000L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
*/
package org.apache.pulsar.common.naming;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
Expand Down Expand Up @@ -114,7 +113,7 @@ public void testInitFailure() throws Exception {

private InputStream updateProp(String zookeeperServer, String brokerServicePort, String namespace, double backlogQuotaGB)
throws IOException {
checkNotNull(fileName);
Objects.requireNonNull(fileName);
Properties properties = new Properties();
InputStream stream = this.getClass().getClassLoader().getResourceAsStream(fileName);
properties.load(stream);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,14 +24,14 @@
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertThrows;
import com.google.common.base.Charsets;
import com.google.common.collect.Lists;
import com.google.common.hash.Hashing;
import org.apache.pulsar.broker.namespace.NamespaceService;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.testng.annotations.Test;


Expand Down Expand Up @@ -67,7 +67,7 @@ public void testAlgorithmReturnCorrectResult() {
List<Long> hashList = new ArrayList<>();
NamespaceBundleFactory namespaceBundleFactoryForMockResult = mock(NamespaceBundleFactory.class);
mockTopics.forEach((topic) -> {
long hashValue = Hashing.crc32().hashString(topic, Charsets.UTF_8).padToLong();
long hashValue = Hashing.crc32().hashString(topic, StandardCharsets.UTF_8).padToLong();
doReturn(namespaceBundleFactoryForMockResult)
.when(namespaceBundleForMockResult).getNamespaceBundleFactory();
doReturn(hashValue)
Expand All @@ -87,7 +87,7 @@ public void testAlgorithmReturnCorrectResult() {
mockTopics.forEach((topic) -> {
doReturn(mockNamespaceBundleFactory)
.when(mockNamespaceBundle).getNamespaceBundleFactory();
long hashValue = Hashing.crc32().hashString(topic, Charsets.UTF_8).padToLong();
long hashValue = Hashing.crc32().hashString(topic, StandardCharsets.UTF_8).padToLong();
doReturn(hashValue)
.when(mockNamespaceBundleFactory).getLongHashCode(topic);
});
Expand Down

0 comments on commit 05a2ea8

Please sign in to comment.