[SPARK-18420][BUILD] Fix the errors caused by lint check in Java
## What changes were proposed in this pull request? Small fix, fix the errors caused by lint check in Java - Clear unused objects and `UnusedImports`. - Add comments around the method `finalize` of `NioBufferedFileInputStream`to turn off checkstyle. - Cut the line which is longer than 100 characters into two lines. ## How was this patch tested? Travis CI. ``` $ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install $ dev/lint-java ``` Before: ``` Checkstyle checks failed at following occurrences: [ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[21,8] (imports) UnusedImports: Unused import - org.apache.commons.crypto.cipher.CryptoCipherFactory. [ERROR] src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java:[516,5] (modifier) RedundantModifier: Redundant 'public' modifier. [ERROR] src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java:[133] (coding) NoFinalizer: Avoid using finalizer method. [ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java:[71] (sizes) LineLength: Line is longer than 100 characters (found 113). [ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java:[112] (sizes) LineLength: Line is longer than 100 characters (found 110). [ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[31,17] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions. [ERROR]src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103). [ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.ml.linalg.Vectors. [ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed. ``` After: ``` $ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install $ dev/lint-java Using `mvn` from path: /home/travis/build/ConeyLiu/spark/build/apache-maven-3.3.9/bin/mvn Checkstyle checks passed. ``` Author: Xianyang Liu <xyliu0530@icloud.com> Closes #15865 from ConeyLiu/master.
This commit is contained in:
parent
a75e3fe923
commit
7569cf6cb8
|
@ -18,7 +18,6 @@
|
|||
package org.apache.spark.network.util;
|
||||
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.commons.crypto.cipher.CryptoCipherFactory;
|
||||
|
||||
/**
|
||||
* A central location that tracks all the settings we expose to users.
|
||||
|
|
|
@ -513,7 +513,7 @@ public class SparkSaslSuite {
|
|||
boolean foundEncryptionHandler;
|
||||
String encryptHandlerName;
|
||||
|
||||
public EncryptionCheckerBootstrap(String encryptHandlerName) {
|
||||
EncryptionCheckerBootstrap(String encryptHandlerName) {
|
||||
this.encryptHandlerName = encryptHandlerName;
|
||||
}
|
||||
|
||||
|
|
|
@ -130,8 +130,10 @@ public final class NioBufferedFileInputStream extends InputStream {
|
|||
StorageUtils.dispose(byteBuffer);
|
||||
}
|
||||
|
||||
//checkstyle.off: NoFinalizer
|
||||
@Override
|
||||
protected void finalize() throws IOException {
|
||||
close();
|
||||
}
|
||||
//checkstyle.on: NoFinalizer
|
||||
}
|
||||
|
|
|
@ -52,6 +52,20 @@
|
|||
<property name="file" value="dev/checkstyle-suppressions.xml"/>
|
||||
</module>
|
||||
|
||||
<!--
|
||||
If you wish to turn off checking for a section of code, you can put a comment in the source
|
||||
before and after the section, with the following syntax:
|
||||
|
||||
// checkstyle:off no.XXX (such as checkstyle.off: NoFinalizer)
|
||||
... // stuff that breaks the styles
|
||||
// checkstyle:on
|
||||
-->
|
||||
<module name="SuppressionCommentFilter">
|
||||
<property name="offCommentFormat" value="checkstyle.off\: ([\w\|]+)"/>
|
||||
<property name="onCommentFormat" value="checkstyle.on\: ([\w\|]+)"/>
|
||||
<property name="checkFormat" value="$1"/>
|
||||
</module>
|
||||
|
||||
<!-- Checks for whitespace -->
|
||||
<!-- See http://checkstyle.sf.net/config_whitespace.html -->
|
||||
<module name="FileTabCharacter">
|
||||
|
@ -168,5 +182,6 @@
|
|||
<module name="UnusedImports"/>
|
||||
<module name="RedundantImport"/>
|
||||
<module name="RedundantModifier"/>
|
||||
<module name="FileContentsHolder"/>
|
||||
</module>
|
||||
</module>
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.spark.examples.ml;
|
|||
|
||||
import org.apache.spark.ml.feature.Interaction;
|
||||
import org.apache.spark.ml.feature.VectorAssembler;
|
||||
import org.apache.spark.ml.linalg.Vectors;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.apache.spark.sql.types.Metadata;
|
||||
|
@ -48,7 +47,7 @@ public class JavaInteractionExample {
|
|||
RowFactory.create(5, 9, 2, 7, 10, 7, 3),
|
||||
RowFactory.create(6, 1, 1, 4, 2, 8, 4)
|
||||
);
|
||||
|
||||
|
||||
StructType schema = new StructType(new StructField[]{
|
||||
new StructField("id1", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
new StructField("id2", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
|
|
|
@ -60,8 +60,8 @@ public class JavaLogisticRegressionWithElasticNetExample {
|
|||
LogisticRegressionModel mlrModel = mlr.fit(training);
|
||||
|
||||
// Print the coefficients and intercepts for logistic regression with multinomial family
|
||||
System.out.println("Multinomial coefficients: "
|
||||
+ lrModel.coefficientMatrix() + "\nMultinomial intercepts: " + mlrModel.interceptVector());
|
||||
System.out.println("Multinomial coefficients: " + lrModel.coefficientMatrix()
|
||||
+ "\nMultinomial intercepts: " + mlrModel.interceptVector());
|
||||
// $example off$
|
||||
|
||||
spark.stop();
|
||||
|
|
|
@ -109,7 +109,8 @@ public final class UnsafeArrayData extends ArrayData {
|
|||
// Read the number of elements from the first 8 bytes.
|
||||
final long numElements = Platform.getLong(baseObject, baseOffset);
|
||||
assert numElements >= 0 : "numElements (" + numElements + ") should >= 0";
|
||||
assert numElements <= Integer.MAX_VALUE : "numElements (" + numElements + ") should <= Integer.MAX_VALUE";
|
||||
assert numElements <= Integer.MAX_VALUE :
|
||||
"numElements (" + numElements + ") should <= Integer.MAX_VALUE";
|
||||
|
||||
this.numElements = (int)numElements;
|
||||
this.baseObject = baseObject;
|
||||
|
|
|
@ -68,7 +68,8 @@ public final class UnsafeMapData extends MapData {
|
|||
// Read the numBytes of key array from the first 8 bytes.
|
||||
final long keyArraySize = Platform.getLong(baseObject, baseOffset);
|
||||
assert keyArraySize >= 0 : "keyArraySize (" + keyArraySize + ") should >= 0";
|
||||
assert keyArraySize <= Integer.MAX_VALUE : "keyArraySize (" + keyArraySize + ") should <= Integer.MAX_VALUE";
|
||||
assert keyArraySize <= Integer.MAX_VALUE :
|
||||
"keyArraySize (" + keyArraySize + ") should <= Integer.MAX_VALUE";
|
||||
final int valueArraySize = sizeInBytes - (int)keyArraySize - 8;
|
||||
assert valueArraySize >= 0 : "valueArraySize (" + valueArraySize + ") should >= 0";
|
||||
|
||||
|
|
|
@ -28,7 +28,6 @@ import java.util.Random;
|
|||
import java.util.Set;
|
||||
|
||||
public class HiveHasherSuite {
|
||||
private final static HiveHasher hasher = new HiveHasher();
|
||||
|
||||
@Test
|
||||
public void testKnownIntegerInputs() {
|
||||
|
|
Loading…
Reference in a new issue