From 60077255219b2a235846a348a0469880b804f773 Mon Sep 17 00:00:00 2001 From: Jan Hentschel Date: Fri, 22 Dec 2017 13:03:48 +0100 Subject: [PATCH] HBASE-19597 Fixed Checkstyle errors in hbase-spark and enabled Checkstyle to fail on violations --- hbase-spark/pom.xml | 16 ++++++ .../hbase/spark/SparkSQLPushDownFilter.java | 25 +++++----- .../JavaHBaseBulkDeleteExample.java | 6 +-- .../JavaHBaseBulkLoadExample.java | 32 +++++++----- .../JavaHBaseMapGetPutExample.java | 30 +++++------ .../hbase/spark/TestJavaHBaseContext.java | 50 +++++++++++-------- 6 files changed, 96 insertions(+), 63 deletions(-) diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml index a1ac466ea82..5397d999ace 100644 --- a/hbase-spark/pom.xml +++ b/hbase-spark/pom.xml @@ -713,6 +713,22 @@ + + org.apache.maven.plugins + maven-checkstyle-plugin + + + checkstyle + validate + + check + + + true + + + + diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java index 6643169aec6..e9bb511ad43 100644 --- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java +++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java @@ -17,10 +17,15 @@ package org.apache.hadoop.hbase.spark; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.hadoop.hbase.Cell; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.spark.datasources.BytesEncoder; @@ -29,16 +34,12 @@ import org.apache.hadoop.hbase.spark.protobuf.generated.SparkFilterProtos; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.spark.sql.datasources.hbase.Field; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import scala.collection.mutable.MutableList; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.ByteString; - /** * This filter will push down all qualifier logic given to us * by SparkSQL so that we have make the filters at the region server level @@ -179,7 +180,7 @@ public class SparkSQLPushDownFilter extends FilterBase{ /** * @param pbBytes A pb serialized instance * @return An instance of SparkSQLPushDownFilter - * @throws org.apache.hadoop.hbase.exceptions.DeserializationException + * @throws DeserializationException if the filter cannot be parsed from the given bytes */ @SuppressWarnings("unused") public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes) diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java index 97cf1404210..d7c424e30f9 100644 --- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java +++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkDeleteExample.java @@ -16,6 +16,9 @@ */ package org.apache.hadoop.hbase.spark.example.hbasecontext; +import java.util.ArrayList; +import java.util.List; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; @@ -27,9 +30,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; -import java.util.ArrayList; -import java.util.List; - /** * This is a simple example of deleting records in HBase * with the bulkDelete function. diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java index 54ff658ca91..f0f3e79f97c 100644 --- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java +++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseBulkLoadExample.java @@ -16,6 +16,10 @@ */ package org.apache.hadoop.hbase.spark.example.hbasecontext; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; @@ -25,23 +29,21 @@ import org.apache.hadoop.hbase.spark.JavaHBaseContext; import org.apache.hadoop.hbase.spark.KeyFamilyQualifier; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; - /** * Run this example using command below: * - * SPARK_HOME/bin/spark-submit --master local[2] --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample + * SPARK_HOME/bin/spark-submit --master local[2] + * --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample * path/to/hbase-spark.jar {path/to/output/HFiles} * * This example will output put hfiles in {path/to/output/HFiles}, and user can run - * 'hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to verify this example. + * 'hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to + * verify this example. */ final public class JavaHBaseBulkLoadExample { private JavaHBaseBulkLoadExample() {} @@ -85,17 +87,21 @@ final public class JavaHBaseBulkLoadExample { } } - public static class BulkLoadFunction implements Function> { - + public static class BulkLoadFunction + implements Function> { @Override public Pair call(String v1) throws Exception { - if (v1 == null) + if (v1 == null) { return null; + } + String[] strs = v1.split(","); - if(strs.length != 4) + if(strs.length != 4) { return null; - KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]), - Bytes.toBytes(strs[2])); + } + + KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), + Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2])); return new Pair(kfq, Bytes.toBytes(strs[3])); } } diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java index 316f8a101a3..9a1259ea346 100644 --- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java +++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/example/hbasecontext/JavaHBaseMapGetPutExample.java @@ -73,24 +73,24 @@ final public class JavaHBaseMapGetPutExample { hbaseContext.foreachPartition(rdd, new VoidFunction, Connection>>() { - public void call(Tuple2, Connection> t) - throws Exception { - Table table = t._2().getTable(TableName.valueOf(tableName)); - BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName)); + public void call(Tuple2, Connection> t) + throws Exception { + Table table = t._2().getTable(TableName.valueOf(tableName)); + BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName)); - while (t._1().hasNext()) { - byte[] b = t._1().next(); - Result r = table.get(new Get(b)); - if (r.getExists()) { - mutator.mutate(new Put(b)); + while (t._1().hasNext()) { + byte[] b = t._1().next(); + Result r = table.get(new Get(b)); + if (r.getExists()) { + mutator.mutate(new Put(b)); + } } - } - mutator.flush(); - mutator.close(); - table.close(); - } - }); + mutator.flush(); + mutator.close(); + table.close(); + } + }); } finally { jsc.stop(); } diff --git a/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java b/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java index f9a813dc295..c2ecbc87c0f 100644 --- a/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java +++ b/hbase-spark/src/test/java/org/apache/hadoop/hbase/spark/TestJavaHBaseContext.java @@ -20,16 +20,16 @@ import java.io.File; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; -import java.util.Iterator; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -41,12 +41,11 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkDeleteExample; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; - import org.apache.hadoop.hbase.util.Pair; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -61,6 +60,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.Tuple2; + import org.apache.hadoop.hbase.shaded.com.google.common.io.Files; @Category({MiscTests.class, MediumTests.class}) @@ -317,10 +317,12 @@ public class TestJavaHBaseContext implements Serializable { - hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(), output.toUri().getPath(), - new HashMap(), false, HConstants.DEFAULT_MAX_FILE_SIZE); + hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(), + output.toUri().getPath(), new HashMap(), false, + HConstants.DEFAULT_MAX_FILE_SIZE); - try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) { + try (Connection conn = ConnectionFactory.createConnection(conf); + Admin admin = conn.getAdmin()) { Table table = conn.getTable(TableName.valueOf(tableName)); // Do bulk load LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf); @@ -387,11 +389,13 @@ public class TestJavaHBaseContext implements Serializable { Configuration conf = htu.getConfiguration(); JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); - hbaseContext.bulkLoadThinRows(rdd, TableName.valueOf(tableName), new BulkLoadThinRowsFunction(), output.toString(), - new HashMap(), false, HConstants.DEFAULT_MAX_FILE_SIZE); + hbaseContext.bulkLoadThinRows(rdd, TableName.valueOf(tableName), new BulkLoadThinRowsFunction(), + output.toString(), new HashMap(), false, + HConstants.DEFAULT_MAX_FILE_SIZE); - try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) { + try (Connection conn = ConnectionFactory.createConnection(conf); + Admin admin = conn.getAdmin()) { Table table = conn.getTable(TableName.valueOf(tableName)); // Do bulk load LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf); @@ -429,25 +433,31 @@ public class TestJavaHBaseContext implements Serializable { } } - public static class BulkLoadFunction implements Function> { - + public static class BulkLoadFunction + implements Function> { @Override public Pair call(String v1) throws Exception { - if (v1 == null) + if (v1 == null) { return null; + } + String[] strs = v1.split(","); - if(strs.length != 4) + if(strs.length != 4) { return null; - KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]), - Bytes.toBytes(strs[2])); + } + + KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), + Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2])); return new Pair(kfq, Bytes.toBytes(strs[3])); } } - public static class BulkLoadThinRowsFunction implements Function, Pair> { - - @Override public Pair call(List list) throws Exception { - if (list == null) + public static class BulkLoadThinRowsFunction + implements Function, Pair> { + @Override public Pair call(List list) { + if (list == null) { return null; + } + ByteArrayWrapper rowKey = null; FamiliesQualifiersValues fqv = new FamiliesQualifiersValues(); for (String cell : list) {