This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 4691809  [Baseline] Apply Baseline to iceberg-data #156 (#198)
4691809 is described below

commit 4691809bdf6e291bc0dae8daf946e7ac73322952
Author: Ratandeep Ratti <[email protected]>
AuthorDate: Tue Jun 11 09:40:55 2019 -0700

    [Baseline] Apply Baseline to iceberg-data #156 (#198)
---
 build.gradle                                       |  3 +-
 .../org/apache/iceberg/data/GenericRecord.java     | 26 ++++++-------
 .../org/apache/iceberg/data/IcebergGenerics.java   |  4 +-
 .../org/apache/iceberg/data/TableScanIterable.java | 12 +++---
 .../org/apache/iceberg/data/avro/DataReader.java   |  4 +-
 .../org/apache/iceberg/data/avro/DataWriter.java   |  8 ++--
 .../apache/iceberg/data/avro/GenericReaders.java   |  6 +--
 .../apache/iceberg/data/avro/IcebergDecoder.java   |  3 +-
 .../data/parquet/GenericParquetReaders.java        | 45 ++++++++++++----------
 .../iceberg/data/parquet/GenericParquetWriter.java | 37 ++++++++----------
 .../java/org/apache/iceberg/TestSplitScan.java     | 31 ++++++++++++---
 .../org/apache/iceberg/data/DataTestHelpers.java   |  2 +
 .../org/apache/iceberg/data/RandomGenericData.java |  8 +++-
 .../org/apache/iceberg/data/TestLocalScan.java     | 10 ++---
 14 files changed, 107 insertions(+), 92 deletions(-)

diff --git a/build.gradle b/build.gradle
index 51d4382..68cf864 100644
--- a/build.gradle
+++ b/build.gradle
@@ -112,7 +112,8 @@ subprojects {
 
 // We enable baseline-idea everywhere so that everyone can use IntelliJ to 
build code against the
 // Baseline style guide.
-def baselineProjects = [ project("iceberg-api"), project("iceberg-common"), 
project("iceberg-core")]
+def baselineProjects = [ project("iceberg-api"), project("iceberg-common"), 
project("iceberg-core"),
+                         project("iceberg-data") ]
 
 configure(subprojects - baselineProjects) {
   // error-prone is brought in with baseline-idea, but we're not prepared to 
handle error-prone
diff --git a/data/src/main/java/org/apache/iceberg/data/GenericRecord.java 
b/data/src/main/java/org/apache/iceberg/data/GenericRecord.java
index 14d5fe8..73129e1 100644
--- a/data/src/main/java/org/apache/iceberg/data/GenericRecord.java
+++ b/data/src/main/java/org/apache/iceberg/data/GenericRecord.java
@@ -19,11 +19,10 @@
 
 package org.apache.iceberg.data;
 
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
 import com.google.common.collect.Maps;
 import java.util.Arrays;
 import java.util.List;
@@ -35,18 +34,15 @@ import org.apache.iceberg.types.Types.StructType;
 
 public class GenericRecord implements Record, StructLike {
   private static final LoadingCache<StructType, Map<String, Integer>> 
NAME_MAP_CACHE =
-      CacheBuilder.newBuilder()
+      Caffeine.newBuilder()
       .weakKeys()
-      .build(new CacheLoader<StructType, Map<String, Integer>>() {
-        @Override
-        public Map<String, Integer> load(StructType struct) {
-          Map<String, Integer> idToPos = Maps.newHashMap();
-          List<Types.NestedField> fields = struct.fields();
-          for (int i = 0; i < fields.size(); i += 1) {
-            idToPos.put(fields.get(i).name(), i);
-          }
-          return idToPos;
+      .build(struct -> {
+        Map<String, Integer> idToPos = Maps.newHashMap();
+        List<Types.NestedField> fields = struct.fields();
+        for (int i = 0; i < fields.size(); i += 1) {
+          idToPos.put(fields.get(i).name(), i);
         }
+        return idToPos;
       });
 
   public static GenericRecord create(Schema schema) {
@@ -66,7 +62,7 @@ public class GenericRecord implements Record, StructLike {
     this.struct = struct;
     this.size = struct.fields().size();
     this.values = new Object[size];
-    this.nameToPos = NAME_MAP_CACHE.getUnchecked(struct);
+    this.nameToPos = NAME_MAP_CACHE.get(struct);
   }
 
   private GenericRecord(GenericRecord toCopy) {
@@ -104,7 +100,7 @@ public class GenericRecord implements Record, StructLike {
   @Override
   public void setField(String name, Object value) {
     Integer pos = nameToPos.get(name);
-    Preconditions.checkArgument(pos != null, "Cannot set unknown field named: 
" + name);
+    Preconditions.checkArgument(pos != null, "Cannot set unknown field named: 
%s", name);
     values[pos] = value;
   }
 
diff --git a/data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java 
b/data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java
index 9e784af..eca1c84 100644
--- a/data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java
+++ b/data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java
@@ -65,8 +65,8 @@ public class IcebergGenerics {
       return this;
     }
 
-    public ScanBuilder select(String... columns) {
-      this.columns = ImmutableList.copyOf(columns);
+    public ScanBuilder select(String... selectedColumns) {
+      this.columns = ImmutableList.copyOf(selectedColumns);
       return this;
     }
 
diff --git a/data/src/main/java/org/apache/iceberg/data/TableScanIterable.java 
b/data/src/main/java/org/apache/iceberg/data/TableScanIterable.java
index 4f74935..27a625d 100644
--- a/data/src/main/java/org/apache/iceberg/data/TableScanIterable.java
+++ b/data/src/main/java/org/apache/iceberg/data/TableScanIterable.java
@@ -24,6 +24,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 import org.apache.iceberg.CombinedScanTask;
@@ -34,6 +35,7 @@ import org.apache.iceberg.TableOperations;
 import org.apache.iceberg.TableScan;
 import org.apache.iceberg.avro.Avro;
 import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.expressions.Evaluator;
 import org.apache.iceberg.expressions.Expressions;
@@ -42,10 +44,6 @@ import org.apache.iceberg.io.CloseableIterable;
 import org.apache.iceberg.io.InputFile;
 import org.apache.iceberg.parquet.Parquet;
 
-import static com.google.common.collect.Iterables.filter;
-import static java.util.Collections.emptyIterator;
-import static 
org.apache.iceberg.data.parquet.GenericParquetReaders.buildReader;
-
 class TableScanIterable extends CloseableGroup implements 
CloseableIterable<Record> {
   private final TableOperations ops;
   private final Schema projection;
@@ -92,7 +90,7 @@ class TableScanIterable extends CloseableGroup implements 
CloseableIterable<Reco
       case PARQUET:
         Parquet.ReadBuilder parquet = Parquet.read(input)
             .project(projection)
-            .createReaderFunc(fileSchema -> buildReader(projection, 
fileSchema))
+            .createReaderFunc(fileSchema -> 
GenericParquetReaders.buildReader(projection, fileSchema))
             .split(task.start(), task.length());
 
         if (reuseContainers) {
@@ -117,7 +115,7 @@ class TableScanIterable extends CloseableGroup implements 
CloseableIterable<Reco
     private final Iterator<FileScanTask> tasks;
     private final boolean caseSensitive;
     private Closeable currentCloseable = null;
-    private Iterator<Record> currentIterator = emptyIterator();
+    private Iterator<Record> currentIterator = Collections.emptyIterator();
 
     private ScanIterator(CloseableIterable<CombinedScanTask> tasks, boolean 
caseSensitive) {
       this.tasks = Lists.newArrayList(Iterables.concat(
@@ -146,7 +144,7 @@ class TableScanIterable extends CloseableGroup implements 
CloseableIterable<Reco
 
           if (task.residual() != null && task.residual() != 
Expressions.alwaysTrue()) {
             Evaluator filter = new Evaluator(projection.asStruct(), 
task.residual(), caseSensitive);
-            this.currentIterator = filter(reader, filter::eval).iterator();
+            this.currentIterator = Iterables.filter(reader, 
filter::eval).iterator();
           } else {
             this.currentIterator = reader.iterator();
           }
diff --git a/data/src/main/java/org/apache/iceberg/data/avro/DataReader.java 
b/data/src/main/java/org/apache/iceberg/data/avro/DataReader.java
index d340ded..122173d 100644
--- a/data/src/main/java/org/apache/iceberg/data/avro/DataReader.java
+++ b/data/src/main/java/org/apache/iceberg/data/avro/DataReader.java
@@ -58,8 +58,8 @@ public class DataReader<T> implements DatumReader<T> {
   }
 
   @Override
-  public void setSchema(Schema fileSchema) {
-    this.fileSchema = Schema.applyAliases(fileSchema, readSchema);
+  public void setSchema(Schema newFileSchema) {
+    this.fileSchema = Schema.applyAliases(newFileSchema, readSchema);
   }
 
   @Override
diff --git a/data/src/main/java/org/apache/iceberg/data/avro/DataWriter.java 
b/data/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
index 543e4b4..322588b 100644
--- a/data/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
+++ b/data/src/main/java/org/apache/iceberg/data/avro/DataWriter.java
@@ -33,8 +33,6 @@ import org.apache.iceberg.avro.LogicalMap;
 import org.apache.iceberg.avro.ValueWriter;
 import org.apache.iceberg.avro.ValueWriters;
 
-import static org.apache.iceberg.avro.AvroSchemaVisitor.visit;
-
 public class DataWriter<T> implements DatumWriter<T> {
   private ValueWriter<T> writer = null;
 
@@ -49,7 +47,7 @@ public class DataWriter<T> implements DatumWriter<T> {
   @Override
   @SuppressWarnings("unchecked")
   public void setSchema(Schema schema) {
-    this.writer = (ValueWriter<T>) visit(schema, new WriteBuilder());
+    this.writer = (ValueWriter<T>) AvroSchemaVisitor.visit(schema, new 
WriteBuilder());
   }
 
   @Override
@@ -69,9 +67,9 @@ public class DataWriter<T> implements DatumWriter<T> {
     @Override
     public ValueWriter<?> union(Schema union, List<ValueWriter<?>> options) {
       Preconditions.checkArgument(options.contains(ValueWriters.nulls()),
-          "Cannot create writer for non-option union: " + union);
+          "Cannot create writer for non-option union: %s", union);
       Preconditions.checkArgument(options.size() == 2,
-          "Cannot create writer for non-option union: " + union);
+          "Cannot create writer for non-option union: %s", union);
       if (union.getTypes().get(0).getType() == Schema.Type.NULL) {
         return ValueWriters.option(0, options.get(1));
       } else {
diff --git 
a/data/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java 
b/data/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java
index 1eceeda..14e4269 100644
--- a/data/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java
+++ b/data/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java
@@ -111,11 +111,11 @@ class GenericReaders {
   }
 
   private static class GenericRecordReader extends 
ValueReaders.StructReader<Record> {
-    private final StructType struct;
+    private final StructType structType;
 
     private GenericRecordReader(List<ValueReader<?>> readers, StructType 
struct) {
       super(readers);
-      this.struct = struct;
+      this.structType = struct;
     }
 
     @Override
@@ -123,7 +123,7 @@ class GenericReaders {
       if (reuse instanceof Record) {
         return (Record) reuse;
       } else {
-        return GenericRecord.create(struct);
+        return GenericRecord.create(structType);
       }
     }
 
diff --git 
a/data/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java 
b/data/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java
index 94285d8..c6b1aaf 100644
--- a/data/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java
+++ b/data/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java
@@ -190,6 +190,7 @@ public class IcebergDecoder<D> extends 
MessageDecoder.BaseDecoder<D> {
    * @return true if the buffer is complete, false otherwise (stream ended)
    * @throws IOException if there is an error while reading
    */
+  @SuppressWarnings("checkstyle:InnerAssignment")
   private boolean readFully(InputStream stream, byte[] bytes)
       throws IOException {
     int pos = 0;
@@ -198,6 +199,6 @@ public class IcebergDecoder<D> extends 
MessageDecoder.BaseDecoder<D> {
         (bytesRead = stream.read(bytes, pos, bytes.length - pos)) > 0) {
       pos += bytesRead;
     }
-    return (pos == bytes.length);
+    return pos == bytes.length;
   }
 }
diff --git 
a/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java 
b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java
index 6257694..c4d2a35 100644
--- 
a/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java
+++ 
b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java
@@ -34,6 +34,7 @@ import java.util.Map;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.GenericRecord;
 import org.apache.iceberg.data.Record;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
 import org.apache.iceberg.parquet.ParquetValueReader;
 import org.apache.iceberg.parquet.ParquetValueReaders;
 import org.apache.iceberg.parquet.ParquetValueReaders.BinaryAsDecimalReader;
@@ -59,9 +60,6 @@ import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-import static org.apache.iceberg.parquet.ParquetSchemaUtil.hasIds;
-import static org.apache.iceberg.parquet.ParquetValueReaders.option;
-
 public class GenericParquetReaders {
   private GenericParquetReaders() {
   }
@@ -69,7 +67,7 @@ public class GenericParquetReaders {
   @SuppressWarnings("unchecked")
   public static ParquetValueReader<GenericRecord> buildReader(Schema 
expectedSchema,
                                                               MessageType 
fileSchema) {
-    if (hasIds(fileSchema)) {
+    if (ParquetSchemaUtil.hasIds(fileSchema)) {
       return (ParquetValueReader<GenericRecord>)
           TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema,
               new ReadBuilder(fileSchema));
@@ -102,8 +100,8 @@ public class GenericParquetReaders {
       List<Type> fields = struct.getFields();
       for (int i = 0; i < fields.size(); i += 1) {
         Type fieldType = fields.get(i);
-        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName()))-1;
-        newFields.add(option(fieldType, fieldD, fieldReaders.get(i)));
+        int fieldD = type().getMaxDefinitionLevel(path(fieldType.getName())) - 
1;
+        newFields.add(ParquetValueReaders.option(fieldType, fieldD, 
fieldReaders.get(i)));
         types.add(fieldType);
       }
 
@@ -112,7 +110,7 @@ public class GenericParquetReaders {
   }
 
   private static class ReadBuilder extends 
TypeWithSchemaVisitor<ParquetValueReader<?>> {
-    final MessageType type;
+    private final MessageType type;
 
     ReadBuilder(MessageType type) {
       this.type = type;
@@ -133,9 +131,9 @@ public class GenericParquetReaders {
       List<Type> fields = struct.getFields();
       for (int i = 0; i < fields.size(); i += 1) {
         Type fieldType = fields.get(i);
-        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName()))-1;
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
         int id = fieldType.getId().intValue();
-        readersById.put(id, option(fieldType, fieldD, fieldReaders.get(i)));
+        readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, 
fieldReaders.get(i)));
         typesById.put(id, fieldType);
       }
 
@@ -165,13 +163,13 @@ public class GenericParquetReaders {
       GroupType repeated = array.getFields().get(0).asGroupType();
       String[] repeatedPath = currentPath();
 
-      int repeatedD = type.getMaxDefinitionLevel(repeatedPath)-1;
-      int repeatedR = type.getMaxRepetitionLevel(repeatedPath)-1;
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
 
       Type elementType = repeated.getType(0);
-      int elementD = type.getMaxDefinitionLevel(path(elementType.getName()))-1;
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 
1;
 
-      return new ListReader<>(repeatedD, repeatedR, option(elementType, 
elementD, elementReader));
+      return new ListReader<>(repeatedD, repeatedR, 
ParquetValueReaders.option(elementType, elementD, elementReader));
     }
 
     @Override
@@ -181,16 +179,17 @@ public class GenericParquetReaders {
       GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
       String[] repeatedPath = currentPath();
 
-      int repeatedD = type.getMaxDefinitionLevel(repeatedPath)-1;
-      int repeatedR = type.getMaxRepetitionLevel(repeatedPath)-1;
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1;
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1;
 
       Type keyType = repeatedKeyValue.getType(0);
-      int keyD = type.getMaxDefinitionLevel(path(keyType.getName()))-1;
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1;
       Type valueType = repeatedKeyValue.getType(1);
-      int valueD = type.getMaxDefinitionLevel(path(valueType.getName()))-1;
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1;
 
       return new MapReader<>(repeatedD, repeatedR,
-          option(keyType, keyD, keyReader), option(valueType, valueD, 
valueReader));
+          ParquetValueReaders.option(keyType, keyD, keyReader),
+          ParquetValueReaders.option(valueType, valueD, valueReader));
     }
 
     @Override
@@ -278,6 +277,10 @@ public class GenericParquetReaders {
       }
     }
 
+    MessageType type() {
+      return type;
+    }
+
     private String[] currentPath() {
       String[] path = new String[fieldNames.size()];
       if (!fieldNames.isEmpty()) {
@@ -380,13 +383,13 @@ public class GenericParquetReaders {
   }
 
   static class RecordReader extends StructReader<Record, Record> {
-    private final StructType struct;
+    private final StructType structType;
 
     RecordReader(List<Type> types,
                  List<ParquetValueReader<?>> readers,
                  StructType struct) {
       super(types, readers);
-      this.struct = struct;
+      this.structType = struct;
     }
 
     @Override
@@ -394,7 +397,7 @@ public class GenericParquetReaders {
       if (reuse != null) {
         return reuse;
       } else {
-        return GenericRecord.create(struct);
+        return GenericRecord.create(structType);
       }
     }
 
diff --git 
a/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java 
b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java
index f0e061a..922a970 100644
--- 
a/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java
+++ 
b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java
@@ -32,6 +32,7 @@ import java.util.List;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.parquet.ParquetTypeVisitor;
 import org.apache.iceberg.parquet.ParquetValueWriter;
+import org.apache.iceberg.parquet.ParquetValueWriters;
 import org.apache.iceberg.parquet.ParquetValueWriters.PrimitiveWriter;
 import org.apache.iceberg.parquet.ParquetValueWriters.StructWriter;
 import org.apache.parquet.column.ColumnDescriptor;
@@ -42,16 +43,6 @@ import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 
-import static org.apache.iceberg.parquet.ParquetValueWriters.byteBuffers;
-import static org.apache.iceberg.parquet.ParquetValueWriters.collections;
-import static org.apache.iceberg.parquet.ParquetValueWriters.decimalAsFixed;
-import static org.apache.iceberg.parquet.ParquetValueWriters.decimalAsInteger;
-import static org.apache.iceberg.parquet.ParquetValueWriters.decimalAsLong;
-import static org.apache.iceberg.parquet.ParquetValueWriters.maps;
-import static org.apache.iceberg.parquet.ParquetValueWriters.option;
-import static org.apache.iceberg.parquet.ParquetValueWriters.strings;
-import static org.apache.iceberg.parquet.ParquetValueWriters.unboxed;
-
 public class GenericParquetWriter {
   private GenericParquetWriter() {
   }
@@ -82,7 +73,7 @@ public class GenericParquetWriter {
       for (int i = 0; i < fields.size(); i += 1) {
         Type fieldType = struct.getType(i);
         int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName()));
-        writers.add(option(fieldType, fieldD, fieldWriters.get(i)));
+        writers.add(ParquetValueWriters.option(fieldType, fieldD, 
fieldWriters.get(i)));
       }
 
       return new RecordWriter(writers);
@@ -99,7 +90,8 @@ public class GenericParquetWriter {
       org.apache.parquet.schema.Type elementType = repeated.getType(0);
       int elementD = type.getMaxDefinitionLevel(path(elementType.getName()));
 
-      return collections(repeatedD, repeatedR, option(elementType, elementD, 
elementWriter));
+      return ParquetValueWriters.collections(repeatedD, repeatedR,
+          ParquetValueWriters.option(elementType, elementD, elementWriter));
     }
 
     @Override
@@ -117,8 +109,9 @@ public class GenericParquetWriter {
       org.apache.parquet.schema.Type valueType = repeatedKeyValue.getType(1);
       int valueD = type.getMaxDefinitionLevel(path(valueType.getName()));
 
-      return maps(repeatedD, repeatedR,
-          option(keyType, keyD, keyWriter), option(valueType, valueD, 
valueWriter));
+      return ParquetValueWriters.maps(repeatedD, repeatedR,
+          ParquetValueWriters.option(keyType, keyD, keyWriter),
+          ParquetValueWriters.option(valueType, valueD, valueWriter));
     }
 
     @Override
@@ -130,12 +123,12 @@ public class GenericParquetWriter {
           case ENUM:
           case JSON:
           case UTF8:
-            return strings(desc);
+            return ParquetValueWriters.strings(desc);
           case INT_8:
           case INT_16:
           case INT_32:
           case INT_64:
-            return unboxed(desc);
+            return ParquetValueWriters.unboxed(desc);
           case DATE:
             return new DateWriter(desc);
           case TIME_MICROS:
@@ -146,18 +139,18 @@ public class GenericParquetWriter {
             DecimalMetadata decimal = primitive.getDecimalMetadata();
             switch (primitive.getPrimitiveTypeName()) {
               case INT32:
-                return decimalAsInteger(desc, decimal.getPrecision(), 
decimal.getScale());
+                return ParquetValueWriters.decimalAsInteger(desc, 
decimal.getPrecision(), decimal.getScale());
               case INT64:
-                return decimalAsLong(desc, decimal.getPrecision(), 
decimal.getScale());
+                return ParquetValueWriters.decimalAsLong(desc, 
decimal.getPrecision(), decimal.getScale());
               case BINARY:
               case FIXED_LEN_BYTE_ARRAY:
-                return decimalAsFixed(desc, decimal.getPrecision(), 
decimal.getScale());
+                return ParquetValueWriters.decimalAsFixed(desc, 
decimal.getPrecision(), decimal.getScale());
               default:
                 throw new UnsupportedOperationException(
                     "Unsupported base type for decimal: " + 
primitive.getPrimitiveTypeName());
             }
           case BSON:
-            return byteBuffers(desc);
+            return ParquetValueWriters.byteBuffers(desc);
           default:
             throw new UnsupportedOperationException(
                 "Unsupported logical type: " + primitive.getOriginalType());
@@ -168,13 +161,13 @@ public class GenericParquetWriter {
         case FIXED_LEN_BYTE_ARRAY:
           return new FixedWriter(desc);
         case BINARY:
-          return byteBuffers(desc);
+          return ParquetValueWriters.byteBuffers(desc);
         case BOOLEAN:
         case INT32:
         case INT64:
         case FLOAT:
         case DOUBLE:
-          return unboxed(desc);
+          return ParquetValueWriters.unboxed(desc);
         default:
           throw new UnsupportedOperationException("Unsupported type: " + 
primitive);
       }
diff --git a/data/src/test/java/org/apache/iceberg/TestSplitScan.java 
b/data/src/test/java/org/apache/iceberg/TestSplitScan.java
index c0e3572..3683dab 100644
--- a/data/src/test/java/org/apache/iceberg/TestSplitScan.java
+++ b/data/src/test/java/org/apache/iceberg/TestSplitScan.java
@@ -1,3 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
 package org.apache.iceberg;
 
 import com.google.common.collect.Lists;
@@ -60,7 +79,7 @@ public class TestSplitScan {
   }
 
   @Before
-  public void setup() throws IOException {
+  public void before() throws IOException {
     tableLocation = new File(temp.newFolder(), "table");
     setupTable();
   }
@@ -99,16 +118,16 @@ public class TestSplitScan {
     table.newAppend().appendFile(dataFile).commit();
   }
 
-  private File writeToFile(List<Record> records, FileFormat format) throws 
IOException {
+  private File writeToFile(List<Record> records, FileFormat fileFormat) throws 
IOException {
     File file = temp.newFile();
     Assert.assertTrue(file.delete());
 
-    switch (format) {
+    switch (fileFormat) {
       case AVRO:
         try (FileAppender<Record> appender = 
Avro.write(Files.localOutput(file))
             .schema(SCHEMA)
             .createWriterFunc(DataWriter::create)
-            .named(format.name())
+            .named(fileFormat.name())
             .build()) {
           appender.addAll(records);
         }
@@ -117,14 +136,14 @@ public class TestSplitScan {
         try (FileAppender<Record> appender = 
Parquet.write(Files.localOutput(file))
             .schema(SCHEMA)
             .createWriterFunc(GenericParquetWriter::buildWriter)
-            .named(format.name())
+            .named(fileFormat.name())
             .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, 
String.valueOf(SPLIT_SIZE))
             .build()) {
           appender.addAll(records);
         }
         break;
       default:
-        throw new UnsupportedOperationException("Cannot write format: " + 
format);
+        throw new UnsupportedOperationException("Cannot write format: " + 
fileFormat);
     }
     return file;
   }
diff --git a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java 
b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java
index bc92cfb..8806649 100644
--- a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java
+++ b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java
@@ -26,6 +26,8 @@ import org.apache.iceberg.types.Types;
 import org.junit.Assert;
 
 public class DataTestHelpers {
+  private DataTestHelpers() {}
+
   public static void assertEquals(Types.StructType struct, Record expected, 
Record actual) {
     List<Types.NestedField> fields = struct.fields();
     for (int i = 0; i < fields.size(); i += 1) {
diff --git a/data/src/test/java/org/apache/iceberg/data/RandomGenericData.java 
b/data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
index 8c6668a..d6e2f8f 100644
--- a/data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
+++ b/data/src/test/java/org/apache/iceberg/data/RandomGenericData.java
@@ -19,13 +19,13 @@
 
 package org.apache.iceberg.data;
 
-import com.google.common.base.Charsets;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.time.Instant;
 import java.time.LocalDate;
 import java.time.LocalTime;
@@ -45,6 +45,8 @@ import org.apache.iceberg.types.Types;
 import static java.time.temporal.ChronoUnit.MICROS;
 
 public class RandomGenericData {
+  private RandomGenericData() {}
+
   public static List<Record> generate(Schema schema, int numRecords, long 
seed) {
     RandomDataGenerator generator = new RandomDataGenerator(seed);
     List<Record> records = Lists.newArrayListWithExpectedSize(numRecords);
@@ -152,6 +154,7 @@ public class RandomGenericData {
     }
   }
 
+  @SuppressWarnings("RandomModInteger")
   private static Object generatePrimitive(Type.PrimitiveType primitive, Random 
random) {
     int choice = random.nextInt(20);
 
@@ -291,10 +294,11 @@ public class RandomGenericData {
       buffer[i] = (byte) CHARS.charAt(random.nextInt(CHARS.length()));
     }
 
-    return new String(buffer, Charsets.UTF_8);
+    return new String(buffer, StandardCharsets.UTF_8);
   }
 
   private static final String DIGITS = "0123456789";
+
   private static BigInteger randomUnscaled(int precision, Random random) {
     int length = random.nextInt(precision);
     if (length == 0) {
diff --git a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java 
b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java
index 388e1cf..44b57f6 100644
--- a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java
+++ b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java
@@ -281,14 +281,14 @@ public class TestLocalScan {
 
   private InputFile writeFile(String location, String filename, List<Record> 
records) throws IOException {
     Path path = new Path(location, filename);
-    FileFormat format = FileFormat.fromFileName(filename);
-    Preconditions.checkNotNull(format, "Cannot determine format for file: %s", 
filename);
-    switch (format) {
+    FileFormat fileFormat = FileFormat.fromFileName(filename);
+    Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: 
%s", filename);
+    switch (fileFormat) {
       case AVRO:
         try (FileAppender<Record> appender = Avro.write(fromPath(path, CONF))
             .schema(SCHEMA)
             .createWriterFunc(DataWriter::create)
-            .named(format.name())
+            .named(fileFormat.name())
             .build()) {
           appender.addAll(records);
         }
@@ -306,7 +306,7 @@ public class TestLocalScan {
         return HadoopInputFile.fromPath(path, CONF);
 
       default:
-        throw new UnsupportedOperationException("Cannot write format: " + 
format);
+        throw new UnsupportedOperationException("Cannot write format: " + 
fileFormat);
     }
   }
 

Reply via email to