Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Enable spotbugs and fix high priority bugs #38

Merged
merged 21 commits into from
Jul 13, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
21 commits
Select commit Hold shift + click to select a range
14c73ee
Set spotbugs threshold to high
kaijchen Jul 8, 2022
bf65108
BufferSegment defines equals and uses Object.hashCode()
kaijchen Jul 8, 2022
a0594c3
ShufflePartitionedBlock defines equals and uses Object.hashCode()
kaijchen Jul 8, 2022
a994cff
Compare array content instead of reference in ShufflePartitionedBlock
kaijchen Jul 8, 2022
d294467
Invocation of toString on ShufflePartitionedData.blockList
kaijchen Jul 8, 2022
8271252
ConfigOption defines equals and uses Object.hashCode()
kaijchen Jul 8, 2022
08a88bf
Public static field in ConfigUtils should be final
kaijchen Jul 8, 2022
190cdde
RssConf defines clone() but doesn't implement Cloneable
kaijchen Jul 8, 2022
d52bd21
Invocation of toString on ipPort
kaijchen Jul 8, 2022
ffa7dae
Reliance on default encoding in SimpleClusterManager
kaijchen Jul 8, 2022
c6abc7c
Public static field in FileBasedShuffleSegment should be final
kaijchen Jul 8, 2022
161280b
Reliance on default encoding in ShuffleStorageUtils
kaijchen Jul 8, 2022
ac20a92
Random object created and used only once in LocalStorageChecker
kaijchen Jul 8, 2022
76c3cd8
public static field should be final in ShuffleFlushManager
kaijchen Jul 8, 2022
ab77201
Fix checkstyle
kaijchen Jul 8, 2022
8f0e85c
public static field should be final in RssMRConfig
kaijchen Jul 8, 2022
0979fda
Fixup RssConf#clone()
kaijchen Jul 8, 2022
6899a3c
Merge branch 'master' into spotbugs
kaijchen Jul 8, 2022
4676e54
Merge branch 'apache:master' into spotbugs
kaijchen Jul 11, 2022
eea4158
Merge branch 'apache:master' into spotbugs
kaijchen Jul 11, 2022
4288a2f
Merge branch 'master' into spotbugs
kaijchen Jul 13, 2022
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 0 additions & 1 deletion .github/workflows/build.yml
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ jobs:
uses: ./.github/workflows/sequential.yml
with:
maven-args: test-compile spotbugs:check
experimental: true

build:
uses: ./.github/workflows/parallel.yml
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ public class RssMRConfig {
RssClientConfig.RSS_CLIENT_DEFAULT_SEND_NUM;
public static final String RSS_CLIENT_SEND_THRESHOLD = MR_RSS_CONFIG_PREFIX + "rss.client.send.threshold";
public static final double RSS_CLIENT_DEFAULT_SEND_THRESHOLD = 0.2f;
public static boolean RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE =
public static final boolean RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE =
RssClientConfig.RSS_DATA_REPLICA_SKIP_ENABLED_DEFAULT_VALUE;
public static final String RSS_HEARTBEAT_INTERVAL =
MR_RSS_CONFIG_PREFIX + RssClientConfig.RSS_HEARTBEAT_INTERVAL;
Expand Down Expand Up @@ -109,26 +109,26 @@ public class RssMRConfig {
MR_RSS_CONFIG_PREFIX + RssClientConfig.RSS_INDEX_READ_LIMIT;
public static final int RSS_INDEX_READ_LIMIT_DEFAULT_VALUE =
RssClientConfig.RSS_INDEX_READ_LIMIT_DEFAULT_VALUE;
public static String RSS_CLIENT_READ_BUFFER_SIZE =
public static final String RSS_CLIENT_READ_BUFFER_SIZE =
MR_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE;

// When the size of read buffer reaches the half of JVM region (i.e., 32m),
// it will incur humongous allocation, so we set it to 14m.
public static String RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE =
public static final String RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE =
RssClientConfig.RSS_CLIENT_READ_BUFFER_SIZE_DEFAULT_VALUE;

public static String RSS_DYNAMIC_CLIENT_CONF_ENABLED =
public static final String RSS_DYNAMIC_CLIENT_CONF_ENABLED =
MR_RSS_CONFIG_PREFIX + RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED;
public static boolean RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE =
public static final boolean RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE =
RssClientConfig.RSS_DYNAMIC_CLIENT_CONF_ENABLED_DEFAULT_VALUE;
public static String RSS_ACCESS_TIMEOUT_MS = MR_RSS_CONFIG_PREFIX + RssClientConfig.RSS_ACCESS_TIMEOUT_MS;
public static int RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE = RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE;
public static final String RSS_ACCESS_TIMEOUT_MS = MR_RSS_CONFIG_PREFIX + RssClientConfig.RSS_ACCESS_TIMEOUT_MS;
public static final int RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE = RssClientConfig.RSS_ACCESS_TIMEOUT_MS_DEFAULT_VALUE;

public static final String RSS_CLIENT_ASSIGNMENT_TAGS =
MR_RSS_CONFIG_PREFIX + RssClientConfig.RSS_CLIENT_ASSIGNMENT_TAGS;

public static String RSS_CONF_FILE = "rss_conf.xml";
public static final String RSS_CONF_FILE = "rss_conf.xml";

public static Set<String> RSS_MANDATORY_CLUSTER_CONF = Sets.newHashSet(
public static final Set<String> RSS_MANDATORY_CLUSTER_CONF = Sets.newHashSet(
RSS_STORAGE_TYPE, RSS_REMOTE_STORAGE_PATH);
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.uniffle.common;

import java.util.Objects;

public class BufferSegment {

private long blockId;
Expand Down Expand Up @@ -48,6 +50,11 @@ public boolean equals(Object obj) {
return false;
}

@Override
public int hashCode() {
return Objects.hash(blockId, offset, length, uncompressLength, crc, taskAttemptId);
}

@Override
public String toString() {
return "BufferSegment{blockId[" + blockId + "], taskAttemptId[" + taskAttemptId
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,9 @@

package org.apache.uniffle.common;

import java.util.Arrays;
import java.util.Objects;

public class ShufflePartitionedBlock {

private int length;
Expand Down Expand Up @@ -59,7 +62,12 @@ public boolean equals(Object o) {
return length == that.length
&& crc == that.crc
&& blockId == that.blockId
&& data.equals(that.data);
&& Arrays.equals(data, that.data);
}

@Override
public int hashCode() {
return Objects.hash(length, crc, blockId, Arrays.hashCode(data));
}

public int getLength() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.uniffle.common;

import java.util.Arrays;

public class ShufflePartitionedData {

private int partitionId;
Expand All @@ -29,7 +31,7 @@ public ShufflePartitionedData(int partitionId, ShufflePartitionedBlock[] blockLi

@Override
public String toString() {
return "ShufflePartitionedData{partitionId=" + partitionId + ", blockList=" + blockList + '}';
return "ShufflePartitionedData{partitionId=" + partitionId + ", blockList=" + Arrays.toString(blockList) + '}';
}

public int getPartitionId() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,11 @@ public boolean equals(Object o) {
}
}

@Override
public int hashCode() {
return Objects.hash(key, defaultValue);
}

@Override
public String toString() {
return String.format("Key: '%s' , default: %s", key, defaultValue);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,17 +177,17 @@ && isFinal(modifiers) && field.getType().isAssignableFrom(ConfigOption.class)) {
return configOptionList;
}

public static Function<Long, Boolean> positiveLongValidator = value -> value > 0;
public static final Function<Long, Boolean> POSITIVE_LONG_VALIDATOR = value -> value > 0;

public static Function<Long, Boolean> non_negativeLongValidator = value -> value >= 0;
public static final Function<Long, Boolean> NON_NEGATIVE_LONG_VALIDATOR = value -> value >= 0;

public static Function<Long, Boolean> positiveIntegerValidator =
public static final Function<Long, Boolean> POSITIVE_INTEGER_VALIDATOR =
value -> value > 0L && value <= Integer.MAX_VALUE;

public static Function<Integer, Boolean> positiveIntegerValidator2 =
public static final Function<Integer, Boolean> POSITIVE_INTEGER_VALIDATOR_2 =
value -> value > 0;

public static Function<Double, Boolean> percentageDoubleValidator =
public static final Function<Double, Boolean> PERCENTAGE_DOUBLE_VALIDATOR =
(Function<Double, Boolean>) value -> Double.compare(value, 100.0) <= 0 && Double.compare(value, 0.0) >= 0;

}
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ public class RssBaseConf extends RssConf {
public static final ConfigOption<Long> RPC_MESSAGE_MAX_SIZE = ConfigOptions
.key("rss.rpc.message.max.size")
.longType()
.checkValue(ConfigUtils.positiveIntegerValidator,
.checkValue(ConfigUtils.POSITIVE_INTEGER_VALIDATOR,
"The value must be positive integer")
.defaultValue(1024L * 1024L * 1024L)
.withDescription("Max size of rpc message (byte)");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,12 @@

import org.apache.uniffle.common.util.UnitConverter;

public class RssConf {
public class RssConf implements Cloneable {
frankliee marked this conversation as resolved.
Show resolved Hide resolved

/**
* Stores the concrete key/value pairs of this configuration object.
*/
private final ConcurrentHashMap<String, Object> settings;
private ConcurrentHashMap<String, Object> settings;

/**
* Creates a new empty configuration.
Expand Down Expand Up @@ -532,9 +532,9 @@ public String getValue(ConfigOption<?> configOption) {
// --------------------------------------------------------------------------------------------

@Override
public RssConf clone() {
RssConf config = new RssConf();
config.addAll(this);
public RssConf clone() throws CloneNotSupportedException {
RssConf config = (RssConf) super.clone();
jerqi marked this conversation as resolved.
Show resolved Hide resolved
config.settings = new ConcurrentHashMap<>(settings);
return config;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public class CoordinatorConf extends RssBaseConf {
public static final ConfigOption<Integer> COORDINATOR_ACCESS_CANDIDATES_UPDATE_INTERVAL_SEC = ConfigOptions
.key("rss.coordinator.access.candidates.updateIntervalSec")
.intType()
.checkValue(ConfigUtils.positiveIntegerValidator2, "access candidates update interval must be positive")
.checkValue(ConfigUtils.POSITIVE_INTEGER_VALIDATOR_2, "access candidates update interval must be positive")
.defaultValue(120)
.withDescription("Accessed candidates update interval in seconds");
public static final ConfigOption<String> COORDINATOR_ACCESS_CANDIDATES_PATH = ConfigOptions
Expand All @@ -82,14 +82,14 @@ public class CoordinatorConf extends RssBaseConf {
public static final ConfigOption<Double> COORDINATOR_ACCESS_LOADCHECKER_MEMORY_PERCENTAGE = ConfigOptions
.key("rss.coordinator.access.loadChecker.memory.percentage")
.doubleType()
.checkValue(ConfigUtils.percentageDoubleValidator,
.checkValue(ConfigUtils.PERCENTAGE_DOUBLE_VALIDATOR,
"The recovery usage percentage must be between 0.0 and 100.0")
.defaultValue(15.0)
.withDescription("The minimal percentage of available memory percentage of a server");
public static final ConfigOption<Integer> COORDINATOR_ACCESS_LOADCHECKER_SERVER_NUM_THRESHOLD = ConfigOptions
.key("rss.coordinator.access.loadChecker.serverNum.threshold")
.intType()
.checkValue(ConfigUtils.positiveIntegerValidator2, "load checker serverNum threshold must be positive")
.checkValue(ConfigUtils.POSITIVE_INTEGER_VALIDATOR_2, "load checker serverNum threshold must be positive")
.noDefaultValue()
.withDescription("The minimal required number of healthy shuffle servers when being accessed by client");
public static final ConfigOption<Boolean> COORDINATOR_DYNAMIC_CLIENT_CONF_ENABLED = ConfigOptions
Expand All @@ -110,7 +110,7 @@ public class CoordinatorConf extends RssBaseConf {
public static final ConfigOption<Integer> COORDINATOR_DYNAMIC_CLIENT_CONF_UPDATE_INTERVAL_SEC = ConfigOptions
.key("rss.coordinator.dynamicClientConf.updateIntervalSec")
.intType()
.checkValue(ConfigUtils.positiveIntegerValidator2, "dynamic client conf update interval in seconds")
.checkValue(ConfigUtils.POSITIVE_INTEGER_VALIDATOR_2, "dynamic client conf update interval in seconds")
.defaultValue(120)
.withDescription("Accessed candidates update interval in seconds");
public static final ConfigOption<String> COORDINATOR_REMOTE_STORAGE_CLUSTER_CONF = ConfigOptions
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import java.util.Set;
Expand Down Expand Up @@ -127,7 +128,7 @@ private void updateExcludeNodes(String path) {

private void parseExcludeNodesFile(DataInputStream fsDataInputStream) throws IOException {
Set<String> nodes = Sets.newConcurrentHashSet();
try (BufferedReader br = new BufferedReader(new InputStreamReader(fsDataInputStream))) {
try (BufferedReader br = new BufferedReader(new InputStreamReader(fsDataInputStream, StandardCharsets.UTF_8))) {
String line;
while ((line = br.readLine()) != null) {
if (!StringUtils.isEmpty(line)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.uniffle.client.factory;

import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -58,7 +59,7 @@ public List<CoordinatorClient> createCoordinatorClient(String coordinators) {
for (String coordinator: coordinatorList) {
String[] ipPort = coordinator.trim().split(":");
if (ipPort.length != 2) {
String msg = "Invalid coordinator format " + ipPort;
String msg = "Invalid coordinator format " + Arrays.toString(ipPort);
LOG.error(msg);
throw new RuntimeException(msg);
}
Expand Down
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -820,6 +820,7 @@
</dependency>
</dependencies>
<configuration>
<threshold>high</threshold>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about other threshold?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It will report too many errors, and some of them you might not want to fix.

For example, the default threshold is medium, see "Summary of failures" in
https://github.com/apache/incubator-uniffle/runs/7299882996?check_suite_focus=true

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If there is a need, we can change it step by step, instead of trying to fix all of them in one huge PR.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think high is OK, LGTM.

<excludeFilterFile>spotbugs-exclude.xml</excludeFilterFile>
</configuration>
</plugin>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,11 @@
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.List;
import java.util.Random;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.RandomUtils;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -156,8 +156,7 @@ boolean checkStorageReadAndWrite() {
if (!writeFile.createNewFile()) {
return false;
}
byte[] data = new byte[1024];
new Random().nextBytes(data);
byte[] data = RandomUtils.nextBytes(1024);
frankliee marked this conversation as resolved.
Show resolved Hide resolved
try (FileOutputStream fos = new FileOutputStream(writeFile)) {
fos.write(data);
fos.flush();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@
public class ShuffleFlushManager {

private static final Logger LOG = LoggerFactory.getLogger(ShuffleFlushManager.class);
public static AtomicLong ATOMIC_EVENT_ID = new AtomicLong(0);
public static final AtomicLong ATOMIC_EVENT_ID = new AtomicLong(0);
private final ShuffleServer shuffleServer;
private final BlockingQueue<ShuffleDataFlushEvent> flushQueue = Queues.newLinkedBlockingQueue();
private final ThreadPoolExecutor threadPoolExecutor;
Expand Down
Loading