Skip to content

Commit 79b0548

Browse files
kiszksrowen
authored andcommitted
[SPARK-26508][CORE][SQL] Address warning messages in Java reported at lgtm.com
## What changes were proposed in this pull request? This PR addresses warning messages in Java files reported at [lgtm.com](https://lgtm.com). [lgtm.com](https://lgtm.com) provides automated code review of Java/Python/JavaScript files for OSS projects. [Here](https://lgtm.com/projects/g/apache/spark/alerts/?mode=list&severity=warning) are warning messages regarding Apache Spark project. This PR addresses the following warnings: - Result of multiplication cast to wider type - Implicit narrowing conversion in compound assignment - Boxed variable is never null - Useless null check NOTE: `Potential input resource leak` looks false positive for now. ## How was this patch tested? Existing UTs Closes apache#23420 from kiszk/SPARK-26508. Authored-by: Kazuaki Ishizaki <[email protected]> Signed-off-by: Sean Owen <[email protected]>
1 parent d371180 commit 79b0548

File tree

11 files changed

+24
-25
lines changed

11 files changed

+24
-25
lines changed

common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java

+6-6
Original file line numberDiff line numberDiff line change
@@ -18,11 +18,11 @@
1818

1919
public enum ByteUnit {
2020
BYTE(1),
21-
KiB(1024L),
22-
MiB((long) Math.pow(1024L, 2L)),
23-
GiB((long) Math.pow(1024L, 3L)),
24-
TiB((long) Math.pow(1024L, 4L)),
25-
PiB((long) Math.pow(1024L, 5L));
21+
KiB(1L << 10),
22+
MiB(1L << 20),
23+
GiB(1L << 30),
24+
TiB(1L << 40),
25+
PiB(1L << 50);
2626

2727
ByteUnit(long multiplier) {
2828
this.multiplier = multiplier;
@@ -50,7 +50,7 @@ public long convertTo(long d, ByteUnit u) {
5050
}
5151
}
5252

53-
public double toBytes(long d) {
53+
public long toBytes(long d) {
5454
if (d < 0) {
5555
throw new IllegalArgumentException("Negative size value. Size must be positive: " + d);
5656
}

common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -309,8 +309,8 @@ public int chunkFetchHandlerThreads() {
309309
}
310310
int chunkFetchHandlerThreadsPercent =
311311
conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100);
312-
return (int)Math.ceil(
313-
(this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors()) *
314-
chunkFetchHandlerThreadsPercent/(double)100);
312+
int threads =
313+
this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors();
314+
return (int) Math.ceil(threads * (chunkFetchHandlerThreadsPercent / 100.0));
315315
}
316316
}

core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -712,7 +712,7 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff
712712
final long recordOffset = offset;
713713
UnsafeAlignedOffset.putSize(base, offset, klen + vlen + uaoSize);
714714
UnsafeAlignedOffset.putSize(base, offset + uaoSize, klen);
715-
offset += (2 * uaoSize);
715+
offset += (2L * uaoSize);
716716
Platform.copyMemory(kbase, koff, base, offset, klen);
717717
offset += klen;
718718
Platform.copyMemory(vbase, voff, base, offset, vlen);
@@ -780,7 +780,7 @@ private void allocate(int capacity) {
780780
assert (capacity >= 0);
781781
capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64);
782782
assert (capacity <= MAX_CAPACITY);
783-
longArray = allocateArray(capacity * 2);
783+
longArray = allocateArray(capacity * 2L);
784784
longArray.zeroOut();
785785

786786
this.growthThreshold = (int) (capacity * loadFactor);

examples/src/main/java/org/apache/spark/examples/JavaTC.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ public static void main(String[] args) {
7171

7272
JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext());
7373

74-
Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2;
74+
int slices = (args.length > 0) ? Integer.parseInt(args[0]): 2;
7575
JavaPairRDD<Integer, Integer> tc = jsc.parallelizePairs(generateGraph(), slices).cache();
7676

7777
// Linear transitive closure: each round grows paths by one edge,

examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -111,7 +111,7 @@ public static void main(String[] args) {
111111
.setMetricName("rmse")
112112
.setLabelCol("rating")
113113
.setPredictionCol("prediction");
114-
Double rmse = evaluator.evaluate(predictions);
114+
double rmse = evaluator.evaluate(predictions);
115115
System.out.println("Root-mean-square error = " + rmse);
116116

117117
// Generate top 10 movie recommendations for each user

examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@ public static void main(String[] args) {
4646

4747
// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method.
4848
// If a method is not specified, Pearson's method will be used by default.
49-
Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson");
49+
double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson");
5050
System.out.println("Correlation is: " + correlation);
5151

5252
// note that each Vector is a row and not a column

examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java

+4-4
Original file line numberDiff line numberDiff line change
@@ -48,14 +48,14 @@ public static void main(String[] args) {
4848

4949
// Train a RandomForest model.
5050
// Empty categoricalFeaturesInfo indicates all features are continuous.
51-
Integer numClasses = 2;
51+
int numClasses = 2;
5252
Map<Integer, Integer> categoricalFeaturesInfo = new HashMap<>();
5353
Integer numTrees = 3; // Use more in practice.
5454
String featureSubsetStrategy = "auto"; // Let the algorithm choose.
5555
String impurity = "gini";
56-
Integer maxDepth = 5;
57-
Integer maxBins = 32;
58-
Integer seed = 12345;
56+
int maxDepth = 5;
57+
int maxBins = 32;
58+
int seed = 12345;
5959

6060
RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses,
6161
categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins,

launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java

+3-4
Original file line numberDiff line numberDiff line change
@@ -318,18 +318,17 @@ protected void handle(Message msg) throws IOException {
318318
throw new IllegalArgumentException("Received Hello for unknown client.");
319319
}
320320
} else {
321+
String msgClassName = msg != null ? msg.getClass().getName() : "no message";
321322
if (handle == null) {
322-
throw new IllegalArgumentException("Expected hello, got: " +
323-
msg != null ? msg.getClass().getName() : null);
323+
throw new IllegalArgumentException("Expected hello, got: " + msgClassName);
324324
}
325325
if (msg instanceof SetAppId) {
326326
SetAppId set = (SetAppId) msg;
327327
handle.setAppId(set.appId);
328328
} else if (msg instanceof SetState) {
329329
handle.setState(((SetState)msg).state);
330330
} else {
331-
throw new IllegalArgumentException("Invalid message: " +
332-
msg != null ? msg.getClass().getName() : null);
331+
throw new IllegalArgumentException("Invalid message: " + msgClassName);
333332
}
334333
}
335334
} catch (Exception e) {

sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ public void initialize(int numElements) {
7474
}
7575

7676
private long getElementOffset(int ordinal) {
77-
return startingOffset + headerInBytes + ordinal * elementSize;
77+
return startingOffset + headerInBytes + ordinal * (long) elementSize;
7878
}
7979

8080
private void setNullBit(int ordinal) {

sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -132,7 +132,7 @@ public void setNull8Bytes(int ordinal) {
132132
}
133133

134134
public long getFieldOffset(int ordinal) {
135-
return startingOffset + nullBitsSize + 8 * ordinal;
135+
return startingOffset + nullBitsSize + 8L * ordinal;
136136
}
137137

138138
public void write(int ordinal, boolean value) {

sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -180,7 +180,7 @@ public long skip(long ns) throws IOException {
180180
return 0;
181181
}
182182
// Bound skip by beginning and end of the source
183-
long n = Math.min(length - next, ns);
183+
int n = (int) Math.min(length - next, ns);
184184
n = Math.max(-next, n);
185185
next += n;
186186
return n;

0 commit comments

Comments
 (0)