diff --git a/.baseline/checkstyle/checkstyle-suppressions.xml b/.baseline/checkstyle/checkstyle-suppressions.xml new file mode 100644 index 000000000000..16adf10ba83c --- /dev/null +++ b/.baseline/checkstyle/checkstyle-suppressions.xml @@ -0,0 +1,30 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml new file mode 100644 index 000000000000..38bf19b08884 --- /dev/null +++ b/.baseline/checkstyle/checkstyle.xml @@ -0,0 +1,489 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/.baseline/copyright/001_apache-2.0.txt b/.baseline/copyright/001_apache-2.0.txt new file mode 100644 index 000000000000..60b675e31016 --- /dev/null +++ b/.baseline/copyright/001_apache-2.0.txt @@ -0,0 +1,16 @@ +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. diff --git a/.baseline/idea/intellij-java-palantir-style.xml b/.baseline/idea/intellij-java-palantir-style.xml new file mode 100644 index 000000000000..3fec69d8fcd6 --- /dev/null +++ b/.baseline/idea/intellij-java-palantir-style.xml @@ -0,0 +1,461 @@ + + + + + diff --git a/.gitignore b/.gitignore index 5550eda0dc5b..787d4746b888 100644 --- a/.gitignore +++ b/.gitignore @@ -9,3 +9,6 @@ build out # web site build site/site +# Expected by Baseline but we don't manage versions this way just yet +# TODO manage versions via baseline's versions plugin +versions.props diff --git a/api/src/main/java/com/netflix/iceberg/CombinedScanTask.java b/api/src/main/java/com/netflix/iceberg/CombinedScanTask.java index 045f89c1fdda..e4a5098ee9d1 100644 --- a/api/src/main/java/com/netflix/iceberg/CombinedScanTask.java +++ b/api/src/main/java/com/netflix/iceberg/CombinedScanTask.java @@ -27,7 +27,7 @@ public interface CombinedScanTask extends ScanTask { /** * Return the {@link FileScanTask tasks} in this combined task. - * @return a Collection of FileScanTask instances. + * @return a Collection of FileScanTask instances */ Collection files(); diff --git a/api/src/main/java/com/netflix/iceberg/DataFile.java b/api/src/main/java/com/netflix/iceberg/DataFile.java index d0e728ec5d37..dcb447bb4f52 100644 --- a/api/src/main/java/com/netflix/iceberg/DataFile.java +++ b/api/src/main/java/com/netflix/iceberg/DataFile.java @@ -19,6 +19,7 @@ package com.netflix.iceberg; +import com.netflix.iceberg.types.Types; import com.netflix.iceberg.types.Types.BinaryType; import com.netflix.iceberg.types.Types.IntegerType; import com.netflix.iceberg.types.Types.ListType; @@ -30,9 +31,6 @@ import java.util.List; import java.util.Map; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; - /** * Interface for files listed in a table manifest. */ @@ -40,23 +38,23 @@ public interface DataFile { static StructType getType(StructType partitionType) { // IDs start at 100 to leave room for changes to ManifestEntry return StructType.of( - required(100, "file_path", StringType.get()), - required(101, "file_format", StringType.get()), - required(102, "partition", partitionType), - required(103, "record_count", LongType.get()), - required(104, "file_size_in_bytes", LongType.get()), - required(105, "block_size_in_bytes", LongType.get()), - optional(106, "file_ordinal", IntegerType.get()), - optional(107, "sort_columns", ListType.ofRequired(112, IntegerType.get())), - optional(108, "column_sizes", MapType.ofRequired(117, 118, + Types.NestedField.required(100, "file_path", StringType.get()), + Types.NestedField.required(101, "file_format", StringType.get()), + Types.NestedField.required(102, "partition", partitionType), + Types.NestedField.required(103, "record_count", LongType.get()), + Types.NestedField.required(104, "file_size_in_bytes", LongType.get()), + Types.NestedField.required(105, "block_size_in_bytes", LongType.get()), + Types.NestedField.optional(106, "file_ordinal", IntegerType.get()), + Types.NestedField.optional(107, "sort_columns", ListType.ofRequired(112, IntegerType.get())), + Types.NestedField.optional(108, "column_sizes", MapType.ofRequired(117, 118, IntegerType.get(), LongType.get())), - optional(109, "value_counts", MapType.ofRequired(119, 120, + Types.NestedField.optional(109, "value_counts", MapType.ofRequired(119, 120, IntegerType.get(), LongType.get())), - optional(110, "null_value_counts", MapType.ofRequired(121, 122, + Types.NestedField.optional(110, "null_value_counts", MapType.ofRequired(121, 122, IntegerType.get(), LongType.get())), - optional(125, "lower_bounds", MapType.ofRequired(126, 127, + Types.NestedField.optional(125, "lower_bounds", MapType.ofRequired(126, 127, IntegerType.get(), BinaryType.get())), - optional(128, "upper_bounds", MapType.ofRequired(129, 130, + Types.NestedField.optional(128, "upper_bounds", MapType.ofRequired(129, 130, IntegerType.get(), BinaryType.get())) // NEXT ID TO ASSIGN: 131 ); diff --git a/api/src/main/java/com/netflix/iceberg/FileFormat.java b/api/src/main/java/com/netflix/iceberg/FileFormat.java index 0a6922af1289..5a243470bbba 100644 --- a/api/src/main/java/com/netflix/iceberg/FileFormat.java +++ b/api/src/main/java/com/netflix/iceberg/FileFormat.java @@ -65,9 +65,9 @@ public static FileFormat fromFileName(CharSequence filename) { return null; } - private static int lastIndexOf(char c, CharSequence seq) { + private static int lastIndexOf(char character, CharSequence seq) { for (int i = seq.length() - 1; i >= 0; i -= 1) { - if (seq.charAt(i) == c) { + if (seq.charAt(i) == character) { return i; } } diff --git a/api/src/main/java/com/netflix/iceberg/Files.java b/api/src/main/java/com/netflix/iceberg/Files.java index f7397511e02f..ce5dfbe3cf2a 100644 --- a/api/src/main/java/com/netflix/iceberg/Files.java +++ b/api/src/main/java/com/netflix/iceberg/Files.java @@ -32,6 +32,8 @@ public class Files { + private Files() {} + public static OutputFile localOutput(File file) { return new LocalOutputFile(file); } @@ -149,21 +151,21 @@ public int read() throws IOException { } @Override - public int read(byte[] b) throws IOException { - return stream.read(b); + public int read(byte[] buf) throws IOException { + return stream.read(buf); } @Override - public int read(byte[] b, int off, int len) throws IOException { - return stream.read(b, off, len); + public int read(byte[] buf, int off, int len) throws IOException { + return stream.read(buf, off, len); } @Override - public long skip(long n) throws IOException { - if (n > Integer.MAX_VALUE) { + public long skip(long numSkip) throws IOException { + if (numSkip > Integer.MAX_VALUE) { return stream.skipBytes(Integer.MAX_VALUE); } else { - return stream.skipBytes((int) n); + return stream.skipBytes((int) numSkip); } } @@ -186,23 +188,23 @@ public long getPos() throws IOException { } @Override - public void write(byte[] b) throws IOException { - stream.write(b); + public void write(byte[] value) throws IOException { + stream.write(value); } @Override - public void write(byte[] b, int off, int len) throws IOException { - stream.write(b, off, len); + public void write(byte[] values, int off, int len) throws IOException { + stream.write(values, off, len); } @Override - public void close() throws IOException { - stream.close(); + public void write(int value) throws IOException { + stream.write(value); } @Override - public void write(int b) throws IOException { - stream.write(b); + public void close() throws IOException { + stream.close(); } } } diff --git a/api/src/main/java/com/netflix/iceberg/ManifestFile.java b/api/src/main/java/com/netflix/iceberg/ManifestFile.java index b1d919b08d06..0cf40c02d432 100644 --- a/api/src/main/java/com/netflix/iceberg/ManifestFile.java +++ b/api/src/main/java/com/netflix/iceberg/ManifestFile.java @@ -20,29 +20,25 @@ package com.netflix.iceberg; import com.netflix.iceberg.types.Types; - import java.nio.ByteBuffer; import java.util.List; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; - /** * Represents a manifest file that can be scanned to find data files in a table. */ public interface ManifestFile { Schema SCHEMA = new Schema( - required(500, "manifest_path", Types.StringType.get()), - required(501, "manifest_length", Types.LongType.get()), - required(502, "partition_spec_id", Types.IntegerType.get()), - optional(503, "added_snapshot_id", Types.LongType.get()), - optional(504, "added_data_files_count", Types.IntegerType.get()), - optional(505, "existing_data_files_count", Types.IntegerType.get()), - optional(506, "deleted_data_files_count", Types.IntegerType.get()), - optional(507, "partitions", Types.ListType.ofRequired(508, Types.StructType.of( - required(509, "contains_null", Types.BooleanType.get()), - optional(510, "lower_bound", Types.BinaryType.get()), // null if no non-null values - optional(511, "upper_bound", Types.BinaryType.get()) + Types.NestedField.required(500, "manifest_path", Types.StringType.get()), + Types.NestedField.required(501, "manifest_length", Types.LongType.get()), + Types.NestedField.required(502, "partition_spec_id", Types.IntegerType.get()), + Types.NestedField.optional(503, "added_snapshot_id", Types.LongType.get()), + Types.NestedField.optional(504, "added_data_files_count", Types.IntegerType.get()), + Types.NestedField.optional(505, "existing_data_files_count", Types.IntegerType.get()), + Types.NestedField.optional(506, "deleted_data_files_count", Types.IntegerType.get()), + Types.NestedField.optional(507, "partitions", Types.ListType.ofRequired(508, Types.StructType.of( + Types.NestedField.required(509, "contains_null", Types.BooleanType.get()), + Types.NestedField.optional(510, "lower_bound", Types.BinaryType.get()), // null if no non-null values + Types.NestedField.optional(511, "upper_bound", Types.BinaryType.get()) )))); static Schema schema() { diff --git a/api/src/main/java/com/netflix/iceberg/Metrics.java b/api/src/main/java/com/netflix/iceberg/Metrics.java index 7ea9d33bc69d..5d6f41dd9482 100644 --- a/api/src/main/java/com/netflix/iceberg/Metrics.java +++ b/api/src/main/java/com/netflix/iceberg/Metrics.java @@ -23,8 +23,6 @@ import java.nio.ByteBuffer; import java.util.Map; -import static com.google.common.collect.ImmutableMap.copyOf; - public class Metrics implements Serializable { private Long rowCount = null; diff --git a/api/src/main/java/com/netflix/iceberg/PartitionField.java b/api/src/main/java/com/netflix/iceberg/PartitionField.java index 331f6dae65af..9e603543b800 100644 --- a/api/src/main/java/com/netflix/iceberg/PartitionField.java +++ b/api/src/main/java/com/netflix/iceberg/PartitionField.java @@ -68,11 +68,9 @@ public boolean equals(Object other) { } PartitionField that = (PartitionField) other; - return ( - sourceId == that.sourceId && - name.equals(that.name) && - transform.equals(that.transform) - ); + return sourceId == that.sourceId + && name.equals(that.name) + && transform.equals(that.transform); } @Override diff --git a/api/src/main/java/com/netflix/iceberg/PartitionSpec.java b/api/src/main/java/com/netflix/iceberg/PartitionSpec.java index b17e25b6edc4..0a915f83efd8 100644 --- a/api/src/main/java/com/netflix/iceberg/PartitionSpec.java +++ b/api/src/main/java/com/netflix/iceberg/PartitionSpec.java @@ -67,7 +67,7 @@ private PartitionSpec(Schema schema, int specId, List fields) { } /** - * @return the {@link Schema} for this spec. + * @return the {@link Schema} for this spec */ public Schema schema() { return schema; @@ -81,7 +81,7 @@ public int specId() { } /** - * @return the list of {@link PartitionField partition fields} for this spec. + * @return the list of {@link PartitionField partition fields} for this spec */ public List fields() { return lazyFieldList(); @@ -96,7 +96,7 @@ public PartitionField getFieldBySourceId(int fieldId) { } /** - * @return a {@link Types.StructType} for partition data defined by this spec. + * @return a {@link Types.StructType} for partition data defined by this spec */ public Types.StructType partitionType() { List structFields = Lists.newArrayListWithExpectedSize(fields.length); @@ -142,10 +142,10 @@ private String escape(String string) { public String partitionToPath(StructLike data) { StringBuilder sb = new StringBuilder(); - Class[] javaClasses = javaClasses(); - for (int i = 0; i < javaClasses.length; i += 1) { + Class[] classes = javaClasses(); + for (int i = 0; i < classes.length; i += 1) { PartitionField field = fields[i]; - String valueString = field.transform().toHumanString(get(data, i, javaClasses[i])); + String valueString = field.transform().toHumanString(get(data, i, classes[i])); if (i > 0) { sb.append("/"); @@ -160,7 +160,7 @@ public String partitionToPath(StructLike data) { * both specs have the same number of fields, field order, source columns, and transforms. * * @param other another PartitionSpec - * @return true if the specs have the same fields, source columns, and transforms. + * @return true if the specs have the same fields, source columns, and transforms */ public boolean compatibleWith(PartitionSpec other) { if (equals(other)) { @@ -174,8 +174,8 @@ public boolean compatibleWith(PartitionSpec other) { for (int i = 0; i < fields.length; i += 1) { PartitionField thisField = fields[i]; PartitionField thatField = other.fields[i]; - if (thisField.sourceId() != thatField.sourceId() || - !thisField.transform().toString().equals(thatField.transform().toString())) { + if (thisField.sourceId() != thatField.sourceId() + || !thisField.transform().toString().equals(thatField.transform().toString())) { return false; } } @@ -238,12 +238,12 @@ private Map lazyFieldsBySourceId() { /** * Returns the source field ids for identity partitions. * - * @return a set of source ids for the identity partitions. + * @return a set of source ids for the identity partitions */ public Set identitySourceIds() { Set sourceIds = Sets.newHashSet(); - List fields = this.fields(); - for (PartitionField field : fields) { + List resolvedFields = fields(); + for (PartitionField field : resolvedFields) { if ("identity".equals(field.transform().toString())) { sourceIds.add(field.sourceId()); } @@ -295,6 +295,8 @@ public static Builder builderFor(Schema schema) { *

* Call {@link #builderFor(Schema)} to create a new builder. */ + // SuppressWarnings because builder classes should be allowed to copy names of passed in fields trivially. + @SuppressWarnings("checkstyle:HiddenField") public static class Builder { private final Schema schema; private final List fields = Lists.newArrayList(); diff --git a/api/src/main/java/com/netflix/iceberg/RewriteFiles.java b/api/src/main/java/com/netflix/iceberg/RewriteFiles.java index 1a8297298b7c..ca604e4cd8c7 100644 --- a/api/src/main/java/com/netflix/iceberg/RewriteFiles.java +++ b/api/src/main/java/com/netflix/iceberg/RewriteFiles.java @@ -34,12 +34,12 @@ * will throw a {@link ValidationException}. */ public interface RewriteFiles extends PendingUpdate { - /** - * Add a rewrite that replaces one set of files with another set that contains the same data. - * - * @param filesToDelete files that will be replaced (deleted), cannot be null or empty. - * @param filesToAdd files that will be added, cannot be null or empty. - * @return this for method chaining - */ - RewriteFiles rewriteFiles(Set filesToDelete, Set filesToAdd); + /** + * Add a rewrite that replaces one set of files with another set that contains the same data. + * + * @param filesToDelete files that will be replaced (deleted), cannot be null or empty. + * @param filesToAdd files that will be added, cannot be null or empty. + * @return this for method chaining + */ + RewriteFiles rewriteFiles(Set filesToDelete, Set filesToAdd); } diff --git a/api/src/main/java/com/netflix/iceberg/ScanTask.java b/api/src/main/java/com/netflix/iceberg/ScanTask.java index a15fc18114b3..8c474bcdb82d 100644 --- a/api/src/main/java/com/netflix/iceberg/ScanTask.java +++ b/api/src/main/java/com/netflix/iceberg/ScanTask.java @@ -26,7 +26,7 @@ */ public interface ScanTask extends Serializable { /** - * @return true if this is a {@link FileScanTask}, false otherwise. + * @return true if this is a {@link FileScanTask}, false otherwise */ default boolean isFileScanTask() { return false; diff --git a/api/src/main/java/com/netflix/iceberg/Schema.java b/api/src/main/java/com/netflix/iceberg/Schema.java index 2ef872a38545..848f9c9e3a4a 100644 --- a/api/src/main/java/com/netflix/iceberg/Schema.java +++ b/api/src/main/java/com/netflix/iceberg/Schema.java @@ -89,14 +89,14 @@ public Map getAliases() { /** * Returns the underlying {@link Types.StructType struct type} for this schema. * - * @return the StructType version of this schema. + * @return the StructType version of this schema */ public Types.StructType asStruct() { return struct; } /** - * @return a List of the {@link Types.NestedField columns} in this Schema. + * @return a List of the {@link Types.NestedField columns} in this Schema */ public List columns() { return struct.fields(); diff --git a/api/src/main/java/com/netflix/iceberg/Snapshot.java b/api/src/main/java/com/netflix/iceberg/Snapshot.java index 89542dc3e32d..bd93aab42561 100644 --- a/api/src/main/java/com/netflix/iceberg/Snapshot.java +++ b/api/src/main/java/com/netflix/iceberg/Snapshot.java @@ -68,7 +68,7 @@ public interface Snapshot { * The files returned include the following columns: file_path, file_format, partition, * record_count, and file_size_in_bytes. Other columns will be null. * - * @return all files added to the table in this snapshot. + * @return all files added to the table in this snapshot */ Iterable addedFiles(); @@ -78,7 +78,7 @@ public interface Snapshot { * The files returned include the following columns: file_path, file_format, partition, * record_count, and file_size_in_bytes. Other columns will be null. * - * @return all files deleted from the table in this snapshot. + * @return all files deleted from the table in this snapshot */ Iterable deletedFiles(); diff --git a/api/src/main/java/com/netflix/iceberg/Table.java b/api/src/main/java/com/netflix/iceberg/Table.java index c785f8e0d2ce..30d53b445ecf 100644 --- a/api/src/main/java/com/netflix/iceberg/Table.java +++ b/api/src/main/java/com/netflix/iceberg/Table.java @@ -71,14 +71,14 @@ public interface Table { /** * Get the current {@link Snapshot snapshot} for this table. * - * @return the current table Snapshot. + * @return the current table Snapshot */ Snapshot currentSnapshot(); /** * Get the {@link Snapshot snapshots} of this table. * - * @return an Iterable of snapshots of this table. + * @return an Iterable of snapshots of this table */ Iterable snapshots(); diff --git a/api/src/main/java/com/netflix/iceberg/TableScan.java b/api/src/main/java/com/netflix/iceberg/TableScan.java index 11df3be609ad..f02e985aaa43 100644 --- a/api/src/main/java/com/netflix/iceberg/TableScan.java +++ b/api/src/main/java/com/netflix/iceberg/TableScan.java @@ -96,6 +96,13 @@ default TableScan select(String... columns) { */ TableScan filter(Expression expr); + /** + * Returns this scan's filter {@link Expression}. + * + * @return this scan's filter expression + */ + Expression filter(); + /** * Plan the {@link FileScanTask files} that will be read by this scan. *

@@ -129,11 +136,4 @@ default TableScan select(String... columns) { * @return this scan's projection schema */ Schema schema(); - - /** - * Returns this scan's filter {@link Expression}. - * - * @return this scan's filter expression - */ - Expression filter(); } diff --git a/api/src/main/java/com/netflix/iceberg/exceptions/RuntimeIOException.java b/api/src/main/java/com/netflix/iceberg/exceptions/RuntimeIOException.java index 837b018d1cd4..8c3a4386d971 100644 --- a/api/src/main/java/com/netflix/iceberg/exceptions/RuntimeIOException.java +++ b/api/src/main/java/com/netflix/iceberg/exceptions/RuntimeIOException.java @@ -25,12 +25,12 @@ * Exception used to wrap {@link IOException} as a {@link RuntimeException} and add context. */ public class RuntimeIOException extends RuntimeException { - public RuntimeIOException(IOException e) { - super(e); + public RuntimeIOException(IOException cause) { + super(cause); } - public RuntimeIOException(IOException e, String message, Object... args) { - super(String.format(message, args), e); + public RuntimeIOException(IOException cause, String message, Object... args) { + super(String.format(message, args), cause); } public RuntimeIOException(String message, Object...args) { diff --git a/api/src/main/java/com/netflix/iceberg/expressions/BoundReference.java b/api/src/main/java/com/netflix/iceberg/expressions/BoundReference.java index 5a836504bcad..4458ec80f55c 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/BoundReference.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/BoundReference.java @@ -36,15 +36,15 @@ public class BoundReference implements Reference { this.type = struct.fields().get(pos).type(); } - private int find(int fieldId, Types.StructType struct) { + private int find(int fieldIdToFind, Types.StructType struct) { List fields = struct.fields(); for (int i = 0; i < fields.size(); i += 1) { - if (fields.get(i).fieldId() == fieldId) { + if (fields.get(i).fieldId() == fieldIdToFind) { return i; } } throw new ValidationException( - "Cannot find top-level field id %d in struct: %s", fieldId, struct); + "Cannot find top-level field id %d in struct: %s", fieldIdToFind, struct); } public Type type() { diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Expression.java b/api/src/main/java/com/netflix/iceberg/expressions/Expression.java index d499b7651aa1..7e727eadd9a3 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Expression.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/Expression.java @@ -100,12 +100,12 @@ public Operation flipLR() { } /** - * @return the operation for an expression node. + * @return the operation for an expression node */ Operation op(); /** - * @return the negation of this expression, equivalent to not(this). + * @return the negation of this expression, equivalent to not(this) */ default Expression negate() { throw new UnsupportedOperationException(String.format("%s cannot be negated", this)); diff --git a/api/src/main/java/com/netflix/iceberg/expressions/ExpressionVisitors.java b/api/src/main/java/com/netflix/iceberg/expressions/ExpressionVisitors.java index 4531b8cbe78d..c05858681cd1 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/ExpressionVisitors.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/ExpressionVisitors.java @@ -23,6 +23,9 @@ * Utils for traversing {@link Expression expressions}. */ public class ExpressionVisitors { + + private ExpressionVisitors() {} + public abstract static class ExpressionVisitor { public R alwaysTrue() { return null; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Expressions.java b/api/src/main/java/com/netflix/iceberg/expressions/Expressions.java index 670789b97381..38aaff3494a6 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Expressions.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/Expressions.java @@ -19,6 +19,15 @@ package com.netflix.iceberg.expressions; +import static com.netflix.iceberg.expressions.Expression.Operation.EQ; +import static com.netflix.iceberg.expressions.Expression.Operation.GT; +import static com.netflix.iceberg.expressions.Expression.Operation.GT_EQ; +import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; +import static com.netflix.iceberg.expressions.Expression.Operation.LT; +import static com.netflix.iceberg.expressions.Expression.Operation.LT_EQ; +import static com.netflix.iceberg.expressions.Expression.Operation.NOT_EQ; +import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; + import com.google.common.base.Preconditions; import com.netflix.iceberg.expressions.Expression.Operation; @@ -68,51 +77,51 @@ public static Expression not(Expression child) { } public static UnboundPredicate isNull(String name) { - return new UnboundPredicate<>(Expression.Operation.IS_NULL, ref(name)); + return new UnboundPredicate<>(IS_NULL, ref(name)); } public static UnboundPredicate notNull(String name) { - return new UnboundPredicate<>(Expression.Operation.NOT_NULL, ref(name)); + return new UnboundPredicate<>(NOT_NULL, ref(name)); } public static UnboundPredicate lessThan(String name, T value) { - return new UnboundPredicate<>(Expression.Operation.LT, ref(name), value); + return new UnboundPredicate<>(LT, ref(name), value); } public static UnboundPredicate lessThanOrEqual(String name, T value) { - return new UnboundPredicate<>(Expression.Operation.LT_EQ, ref(name), value); + return new UnboundPredicate<>(LT_EQ, ref(name), value); } public static UnboundPredicate greaterThan(String name, T value) { - return new UnboundPredicate<>(Expression.Operation.GT, ref(name), value); + return new UnboundPredicate<>(GT, ref(name), value); } public static UnboundPredicate greaterThanOrEqual(String name, T value) { - return new UnboundPredicate<>(Expression.Operation.GT_EQ, ref(name), value); + return new UnboundPredicate<>(GT_EQ, ref(name), value); } public static UnboundPredicate equal(String name, T value) { - return new UnboundPredicate<>(Expression.Operation.EQ, ref(name), value); + return new UnboundPredicate<>(EQ, ref(name), value); } public static UnboundPredicate notEqual(String name, T value) { - return new UnboundPredicate<>(Expression.Operation.NOT_EQ, ref(name), value); + return new UnboundPredicate<>(NOT_EQ, ref(name), value); } public static UnboundPredicate predicate(Operation op, String name, T value) { - Preconditions.checkArgument(op != Operation.IS_NULL && op != Operation.NOT_NULL, + Preconditions.checkArgument(op != IS_NULL && op != NOT_NULL, "Cannot create %s predicate inclusive a value", op); return new UnboundPredicate<>(op, ref(name), value); } public static UnboundPredicate predicate(Operation op, String name, Literal lit) { - Preconditions.checkArgument(op != Operation.IS_NULL && op != Operation.NOT_NULL, + Preconditions.checkArgument(op != IS_NULL && op != NOT_NULL, "Cannot create %s predicate inclusive a value", op); return new UnboundPredicate<>(op, ref(name), lit); } public static UnboundPredicate predicate(Operation op, String name) { - Preconditions.checkArgument(op == Operation.IS_NULL || op == Operation.NOT_NULL, + Preconditions.checkArgument(op == IS_NULL || op == NOT_NULL, "Cannot create %s predicate without a value", op); return new UnboundPredicate<>(op, ref(name)); } diff --git a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveManifestEvaluator.java b/api/src/main/java/com/netflix/iceberg/expressions/InclusiveManifestEvaluator.java index cac617d777a1..10285fa3adb3 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveManifestEvaluator.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/InclusiveManifestEvaluator.java @@ -19,6 +19,8 @@ package com.netflix.iceberg.expressions; +import static com.netflix.iceberg.expressions.Expressions.rewriteNot; + import com.netflix.iceberg.ManifestFile; import com.netflix.iceberg.ManifestFile.PartitionFieldSummary; import com.netflix.iceberg.PartitionSpec; @@ -28,8 +30,6 @@ import java.nio.ByteBuffer; import java.util.List; -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; - /** * Evaluates an {@link Expression} on a {@link ManifestFile} to test whether the file contains * matching partitions. @@ -61,7 +61,7 @@ public InclusiveManifestEvaluator(PartitionSpec spec, Expression rowFilter) { * Test whether the file may contain records that match the expression. * * @param manifest a manifest file - * @return false if the file cannot contain rows that match the expression, true otherwise. + * @return false if the file cannot contain rows that match the expression, true otherwise */ public boolean eval(ManifestFile manifest) { return visitor().eval(manifest); diff --git a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveMetricsEvaluator.java b/api/src/main/java/com/netflix/iceberg/expressions/InclusiveMetricsEvaluator.java index 26c17e4f6078..8876da7e2ec9 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveMetricsEvaluator.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/InclusiveMetricsEvaluator.java @@ -29,8 +29,6 @@ import java.nio.ByteBuffer; import java.util.Map; -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; - /** * Evaluates an {@link Expression} on a {@link DataFile} to test whether rows in the file may match. *

@@ -56,14 +54,14 @@ private MetricsEvalVisitor visitor() { public InclusiveMetricsEvaluator(Schema schema, Expression unbound) { this.schema = schema; this.struct = schema.asStruct(); - this.expr = Binder.bind(struct, rewriteNot(unbound)); + this.expr = Binder.bind(struct, Expressions.rewriteNot(unbound)); } /** * Test whether the file may contain records that match the expression. * * @param file a data file - * @return false if the file cannot contain rows that match the expression, true otherwise. + * @return false if the file cannot contain rows that match the expression, true otherwise */ public boolean eval(DataFile file) { // TODO: detect the case where a column is missing from the file using file's max field id. @@ -140,9 +138,11 @@ public Boolean notNull(BoundReference ref) { Preconditions.checkNotNull(struct.field(id), "Cannot filter by nested column: %s", schema.findField(id)); - if (valueCounts != null && valueCounts.containsKey(id) && - nullCounts != null && nullCounts.containsKey(id) && - valueCounts.get(id) - nullCounts.get(id) == 0) { + if (valueCounts != null + && valueCounts.containsKey(id) + && nullCounts != null + && nullCounts.containsKey(id) + && valueCounts.get(id) - nullCounts.get(id) == 0) { return ROWS_CANNOT_MATCH; } diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Literal.java b/api/src/main/java/com/netflix/iceberg/expressions/Literal.java index 69a2623303ae..abf4c5e0651f 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Literal.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/Literal.java @@ -27,7 +27,7 @@ import java.util.UUID; /** - * Represents a literal fixed value in an expression predicate + * Represents a literal fixed value in an expression predicate. * @param The Java type of the value wrapped by a {@link Literal} */ public interface Literal extends Serializable { @@ -56,7 +56,7 @@ static Literal of(CharSequence value) { } static Literal of(UUID value) { - return new Literals.UUIDLiteral(value); + return new Literals.UuidLiteral(value); } static Literal of(byte[] value) { diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Literals.java b/api/src/main/java/com/netflix/iceberg/expressions/Literals.java index f4e5d4e74446..e3ca899e8ed1 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Literals.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/Literals.java @@ -52,7 +52,8 @@ private Literals() { * @param Java type of value * @return a Literal for the given value */ - @SuppressWarnings("unchecked") + // Ignore cyclomatic complexity because of the instanceof branches. + @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) static Literal from(T value) { Preconditions.checkNotNull(value, "Cannot create expression literal from null"); @@ -69,7 +70,7 @@ static Literal from(T value) { } else if (value instanceof CharSequence) { return (Literal) new Literals.StringLiteral((CharSequence) value); } else if (value instanceof UUID) { - return (Literal) new Literals.UUIDLiteral((UUID) value); + return (Literal) new UuidLiteral((UUID) value); } else if (value instanceof byte[]) { return (Literal) new Literals.FixedLiteral(ByteBuffer.wrap((byte[]) value)); } else if (value instanceof ByteBuffer) { @@ -116,7 +117,7 @@ private abstract static class ComparableLiteral> extends private static final Comparator CMP = Comparators.nullsFirst().thenComparing(Comparator.naturalOrder()); - public ComparableLiteral(C value) { + ComparableLiteral(C value) { super(value); } @@ -340,7 +341,7 @@ static class TimeLiteral extends ComparableLiteral { @SuppressWarnings("unchecked") public Literal to(Type type) { if (type.typeId() == Type.TypeID.TIME) { - return (Literal) this ; + return (Literal) this; } return null; } @@ -425,7 +426,7 @@ public Literal to(Type type) { return (Literal) this; case UUID: - return (Literal) new UUIDLiteral(UUID.fromString(value().toString())); + return (Literal) new UuidLiteral(UUID.fromString(value().toString())); case DECIMAL: int scale = ((Types.DecimalType) type).scale(); @@ -451,8 +452,8 @@ public String toString() { } } - static class UUIDLiteral extends ComparableLiteral { - UUIDLiteral(UUID value) { + static class UuidLiteral extends ComparableLiteral { + UuidLiteral(UUID value) { super(value); } diff --git a/api/src/main/java/com/netflix/iceberg/expressions/NamedReference.java b/api/src/main/java/com/netflix/iceberg/expressions/NamedReference.java index 5e4ec1e7a592..ce7c964be2f8 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/NamedReference.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/NamedReference.java @@ -22,7 +22,7 @@ import com.google.common.base.Preconditions; public class NamedReference implements Reference { - public final String name; + private final String name; NamedReference(String name) { Preconditions.checkNotNull(name, "Name cannot be null"); diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Projections.java b/api/src/main/java/com/netflix/iceberg/expressions/Projections.java index d9da0532c82f..266ce5448936 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Projections.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/Projections.java @@ -19,9 +19,9 @@ package com.netflix.iceberg.expressions; -import com.netflix.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; import com.netflix.iceberg.PartitionField; import com.netflix.iceberg.PartitionSpec; +import com.netflix.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; import com.netflix.iceberg.transforms.Transform; /** @@ -43,7 +43,7 @@ private Projections() { * A strict projection guarantees that if a partition matches a projected expression, then all * rows in that partition will match the original expression. */ - public static abstract class ProjectionEvaluator extends ExpressionVisitor { + public abstract static class ProjectionEvaluator extends ExpressionVisitor { /** * Project the given row expression to a partition expression. * @@ -92,7 +92,7 @@ public static ProjectionEvaluator strict(PartitionSpec spec) { } private static class BaseProjectionEvaluator extends ProjectionEvaluator { - final PartitionSpec spec; + private final PartitionSpec spec; private BaseProjectionEvaluator(PartitionSpec spec) { this.spec = spec; @@ -143,6 +143,10 @@ public Expression predicate(UnboundPredicate pred) { return bound; } + + protected PartitionSpec spec() { + return spec; + } } private static class InclusiveProjection extends BaseProjectionEvaluator { @@ -153,7 +157,7 @@ private InclusiveProjection(PartitionSpec spec) { @Override @SuppressWarnings("unchecked") public Expression predicate(BoundPredicate pred) { - PartitionField part = spec.getFieldBySourceId(pred.ref().fieldId()); + PartitionField part = spec().getFieldBySourceId(pred.ref().fieldId()); if (part == null) { // the predicate has no partition column return alwaysTrue(); @@ -178,7 +182,7 @@ private StrictProjection(PartitionSpec spec) { @Override @SuppressWarnings("unchecked") public Expression predicate(BoundPredicate pred) { - PartitionField part = spec.getFieldBySourceId(pred.ref().fieldId()); + PartitionField part = spec().getFieldBySourceId(pred.ref().fieldId()); if (part == null) { // the predicate has no partition column return alwaysFalse(); diff --git a/api/src/main/java/com/netflix/iceberg/expressions/ResidualEvaluator.java b/api/src/main/java/com/netflix/iceberg/expressions/ResidualEvaluator.java index 290ae4afe034..80a99b07c778 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/ResidualEvaluator.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/ResidualEvaluator.java @@ -19,9 +19,9 @@ package com.netflix.iceberg.expressions; -import com.netflix.iceberg.StructLike; import com.netflix.iceberg.PartitionField; import com.netflix.iceberg.PartitionSpec; +import com.netflix.iceberg.StructLike; import com.netflix.iceberg.transforms.Transform; import java.io.Serializable; import java.util.Comparator; @@ -76,8 +76,8 @@ public Expression residualFor(StructLike partitionData) { private class ResidualVisitor extends ExpressionVisitors.BoundExpressionVisitor { private StructLike struct; - private Expression eval(StructLike struct) { - this.struct = struct; + private Expression eval(StructLike newStruct) { + this.struct = newStruct; return ExpressionVisitors.visit(expr, this); } diff --git a/api/src/main/java/com/netflix/iceberg/expressions/SerializationProxies.java b/api/src/main/java/com/netflix/iceberg/expressions/SerializationProxies.java index deac29ee170a..eb90a8fc41fc 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/SerializationProxies.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/SerializationProxies.java @@ -35,10 +35,8 @@ static class ConstantExpressionProxy implements Serializable { /** * Constructor for Java serialization. */ - public ConstantExpressionProxy() { - } - public ConstantExpressionProxy(boolean trueOrFalse) { + ConstantExpressionProxy(boolean trueOrFalse) { this.trueOrFalse = trueOrFalse; } @@ -63,7 +61,7 @@ static class BinaryLiteralProxy extends FixedLiteralProxy { } Object readResolve() throws ObjectStreamException { - return new Literals.BinaryLiteral(ByteBuffer.wrap(bytes)); + return new Literals.BinaryLiteral(ByteBuffer.wrap(bytes())); } } @@ -71,7 +69,7 @@ Object readResolve() throws ObjectStreamException { * Replacement for FixedLiteral in Java Serialization. */ static class FixedLiteralProxy implements Serializable { - protected byte[] bytes; + private byte[] bytes; /** * Constructor for Java serialization. @@ -87,5 +85,9 @@ static class FixedLiteralProxy implements Serializable { Object readResolve() throws ObjectStreamException { return new Literals.FixedLiteral(ByteBuffer.wrap(bytes)); } + + protected byte[] bytes() { + return bytes; + } } } diff --git a/api/src/main/java/com/netflix/iceberg/expressions/StrictMetricsEvaluator.java b/api/src/main/java/com/netflix/iceberg/expressions/StrictMetricsEvaluator.java index d3fa1df5321a..1b6ccb67eb8a 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/StrictMetricsEvaluator.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/StrictMetricsEvaluator.java @@ -29,8 +29,6 @@ import java.nio.ByteBuffer; import java.util.Map; -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; - /** * Evaluates an {@link Expression} on a {@link DataFile} to test whether all rows in the file match. *

@@ -57,14 +55,14 @@ private MetricsEvalVisitor visitor() { public StrictMetricsEvaluator(Schema schema, Expression unbound) { this.schema = schema; this.struct = schema.asStruct(); - this.expr = Binder.bind(struct, rewriteNot(unbound)); + this.expr = Binder.bind(struct, Expressions.rewriteNot(unbound)); } /** * Test whether the file may contain records that match the expression. * * @param file a data file - * @return false if the file cannot contain rows that match the expression, true otherwise. + * @return false if the file cannot contain rows that match the expression, true otherwise */ public boolean eval(DataFile file) { // TODO: detect the case where a column is missing from the file using file's max field id. @@ -126,9 +124,11 @@ public Boolean isNull(BoundReference ref) { Preconditions.checkNotNull(struct.field(id), "Cannot filter by nested column: %s", schema.findField(id)); - if (valueCounts != null && valueCounts.containsKey(id) && - nullCounts != null && nullCounts.containsKey(id) && - valueCounts.get(id) - nullCounts.get(id) == 0) { + if (valueCounts != null + && valueCounts.containsKey(id) + && nullCounts != null + && nullCounts.containsKey(id) + && valueCounts.get(id) - nullCounts.get(id) == 0) { return ROWS_MUST_MATCH; } @@ -233,8 +233,10 @@ public Boolean eq(BoundReference ref, Literal lit) { Types.NestedField field = struct.field(id); Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); - if (lowerBounds != null && lowerBounds.containsKey(id) && - upperBounds != null && upperBounds.containsKey(id)) { + if (lowerBounds != null + && lowerBounds.containsKey(id) + && upperBounds != null + && upperBounds.containsKey(id)) { T lower = Conversions.fromByteBuffer(struct.field(id).type(), lowerBounds.get(id)); int cmp = lit.comparator().compare(lower, lit.value()); diff --git a/api/src/main/java/com/netflix/iceberg/expressions/UnboundPredicate.java b/api/src/main/java/com/netflix/iceberg/expressions/UnboundPredicate.java index 3523d1d4f9c0..6246c0081dac 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/UnboundPredicate.java +++ b/api/src/main/java/com/netflix/iceberg/expressions/UnboundPredicate.java @@ -22,9 +22,6 @@ import com.netflix.iceberg.exceptions.ValidationException; import com.netflix.iceberg.types.Types; -import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; -import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; - public class UnboundPredicate extends Predicate { UnboundPredicate(Operation op, NamedReference namedRef, T value) { @@ -55,12 +52,12 @@ public Expression bind(Types.StructType struct) { if (field.isRequired()) { return Expressions.alwaysFalse(); } - return new BoundPredicate<>(IS_NULL, new BoundReference<>(struct, field.fieldId())); + return new BoundPredicate<>(Expression.Operation.IS_NULL, new BoundReference<>(struct, field.fieldId())); case NOT_NULL: if (field.isRequired()) { return Expressions.alwaysTrue(); } - return new BoundPredicate<>(NOT_NULL, new BoundReference<>(struct, field.fieldId())); + return new BoundPredicate<>(Expression.Operation.NOT_NULL, new BoundReference<>(struct, field.fieldId())); default: throw new ValidationException("Operation must be IS_NULL or NOT_NULL"); } diff --git a/api/src/main/java/com/netflix/iceberg/io/CloseableGroup.java b/api/src/main/java/com/netflix/iceberg/io/CloseableGroup.java index 1f6224691cf0..c774dbd835a5 100644 --- a/api/src/main/java/com/netflix/iceberg/io/CloseableGroup.java +++ b/api/src/main/java/com/netflix/iceberg/io/CloseableGroup.java @@ -23,10 +23,10 @@ import java.io.Closeable; import java.io.IOException; import java.util.Iterator; -import java.util.LinkedList; +import java.util.Queue; public abstract class CloseableGroup implements Closeable { - private final LinkedList closeables = Lists.newLinkedList(); + private final Queue closeables = Lists.newLinkedList(); protected void addCloseable(Closeable closeable) { closeables.add(closeable); @@ -35,7 +35,7 @@ protected void addCloseable(Closeable closeable) { @Override public void close() throws IOException { while (!closeables.isEmpty()) { - Closeable toClose = closeables.removeFirst(); + Closeable toClose = closeables.remove(); if (toClose != null) { toClose.close(); } @@ -45,7 +45,7 @@ public void close() throws IOException { static class ClosingIterable extends CloseableGroup implements CloseableIterable { private final Iterable iterable; - public ClosingIterable(Iterable iterable, Iterable closeables) { + ClosingIterable(Iterable iterable, Iterable closeables) { this.iterable = iterable; for (Closeable closeable : closeables) { addCloseable(closeable); diff --git a/api/src/main/java/com/netflix/iceberg/io/FileAppender.java b/api/src/main/java/com/netflix/iceberg/io/FileAppender.java index 6f5188697f2a..59f6b8b85507 100644 --- a/api/src/main/java/com/netflix/iceberg/io/FileAppender.java +++ b/api/src/main/java/com/netflix/iceberg/io/FileAppender.java @@ -37,7 +37,7 @@ default void addAll(Iterable values) { } /** - * @return {@link Metrics} for this file. Only valid after the file is closed. + * @return {@link Metrics} for this file. Only valid after the file is closed */ Metrics metrics(); } diff --git a/api/src/main/java/com/netflix/iceberg/io/InputFile.java b/api/src/main/java/com/netflix/iceberg/io/InputFile.java index 91444f726b65..f323f81a4f77 100644 --- a/api/src/main/java/com/netflix/iceberg/io/InputFile.java +++ b/api/src/main/java/com/netflix/iceberg/io/InputFile.java @@ -35,7 +35,7 @@ public interface InputFile { long getLength(); /** - * Opens a new {@link SeekableInputStream} for the underlying data file + * Opens a new {@link SeekableInputStream} for the underlying data file. * * @return a seekable stream for reading the file * @throws RuntimeIOException If the implementation throws an {@link IOException} diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Bucket.java b/api/src/main/java/com/netflix/iceberg/transforms/Bucket.java index 4f9c2fda5f49..862dd9406d14 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Bucket.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/Bucket.java @@ -20,7 +20,6 @@ package com.netflix.iceberg.transforms; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; import com.google.common.base.Objects; import com.google.common.collect.Sets; import com.google.common.hash.HashFunction; @@ -33,46 +32,45 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; import java.util.Set; import java.util.UUID; -import static com.netflix.iceberg.types.Type.TypeID; - abstract class Bucket implements Transform { private static final HashFunction MURMUR3 = Hashing.murmur3_32(); @SuppressWarnings("unchecked") - static Bucket get(Type type, int N) { + static Bucket get(Type type, int numBuckets) { switch (type.typeId()) { case DATE: case INTEGER: - return (Bucket) new BucketInteger(N); + return (Bucket) new BucketInteger(numBuckets); case TIME: case TIMESTAMP: case LONG: - return (Bucket) new BucketLong(N); + return (Bucket) new BucketLong(numBuckets); case DECIMAL: - return (Bucket) new BucketDecimal(N); + return (Bucket) new BucketDecimal(numBuckets); case STRING: - return (Bucket) new BucketString(N); + return (Bucket) new BucketString(numBuckets); case FIXED: case BINARY: - return (Bucket) new BucketByteBuffer(N); + return (Bucket) new BucketByteBuffer(numBuckets); case UUID: - return (Bucket) new BucketUUID(N); + return (Bucket) new BucketUuid(numBuckets); default: throw new IllegalArgumentException("Cannot bucket by type: " + type); } } - private final int N; + private final int numBuckets; - private Bucket(int N) { - this.N = N; + private Bucket(int numBuckets) { + this.numBuckets = numBuckets; } public Integer numBuckets() { - return N; + return numBuckets; } @VisibleForTesting @@ -80,30 +78,30 @@ public Integer numBuckets() { @Override public Integer apply(T value) { - return (hash(value) & Integer.MAX_VALUE) % N; + return (hash(value) & Integer.MAX_VALUE) % numBuckets; } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; } - if (o == null || getClass() != o.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } - Bucket bucket = (Bucket) o; - return N == bucket.N; + Bucket bucket = (Bucket) other; + return numBuckets == bucket.numBuckets; } @Override public int hashCode() { - return Objects.hashCode(N); + return Objects.hashCode(numBuckets); } @Override public String toString() { - return "bucket[" + N + "]"; + return "bucket[" + numBuckets + "]"; } @Override @@ -141,8 +139,8 @@ public Type getResultType(Type sourceType) { } private static class BucketInteger extends Bucket { - private BucketInteger(int N) { - super(N); + private BucketInteger(int numBuckets) { + super(numBuckets); } public int hash(Integer value) { @@ -151,13 +149,13 @@ public int hash(Integer value) { @Override public boolean canTransform(Type type) { - return type.typeId() == TypeID.INTEGER || type.typeId() == TypeID.DATE; + return type.typeId() == Type.TypeID.INTEGER || type.typeId() == Type.TypeID.DATE; } } private static class BucketLong extends Bucket { - private BucketLong(int N) { - super(N); + private BucketLong(int numBuckets) { + super(numBuckets); } public int hash(Long value) { @@ -166,19 +164,17 @@ public int hash(Long value) { @Override public boolean canTransform(Type type) { - return ( - type.typeId() == TypeID.LONG || - type.typeId() == TypeID.TIME || - type.typeId() == TypeID.TIMESTAMP - ); + return type.typeId() == Type.TypeID.LONG + || type.typeId() == Type.TypeID.TIME + || type.typeId() == Type.TypeID.TIMESTAMP; } } // bucketing by Double is not allowed by the spec, but this has the float hash implementation static class BucketFloat extends Bucket { // used by tests because the factory method will not instantiate a bucket function for floats - BucketFloat(int N) { - super(N); + BucketFloat(int numBuckets) { + super(numBuckets); } public int hash(Float value) { @@ -187,15 +183,15 @@ public int hash(Float value) { @Override public boolean canTransform(Type type) { - return type.typeId() == TypeID.FLOAT; + return type.typeId() == Type.TypeID.FLOAT; } } // bucketing by Double is not allowed by the spec, but this has the double hash implementation static class BucketDouble extends Bucket { // used by tests because the factory method will not instantiate a bucket function for doubles - BucketDouble(int N) { - super(N); + BucketDouble(int numBuckets) { + super(numBuckets); } public int hash(Double value) { @@ -204,31 +200,31 @@ public int hash(Double value) { @Override public boolean canTransform(Type type) { - return type.typeId() == TypeID.DOUBLE; + return type.typeId() == Type.TypeID.DOUBLE; } } private static class BucketString extends Bucket { - private BucketString(int N) { - super(N); + private BucketString(int numBuckets) { + super(numBuckets); } public int hash(CharSequence value) { - return MURMUR3.hashString(value, Charsets.UTF_8).asInt(); + return MURMUR3.hashString(value, StandardCharsets.UTF_8).asInt(); } @Override public boolean canTransform(Type type) { - return type.typeId() == TypeID.STRING; + return type.typeId() == Type.TypeID.STRING; } } private static class BucketBytes extends Bucket { - private static final Set SUPPORTED_TYPES = Sets.newHashSet( - TypeID.BINARY, TypeID.FIXED); + private static final Set SUPPORTED_TYPES = Sets.newHashSet( + Type.TypeID.BINARY, Type.TypeID.FIXED); - private BucketBytes(int N) { - super(N); + private BucketBytes(int numBuckets) { + super(numBuckets); } public int hash(byte[] value) { @@ -242,11 +238,11 @@ public boolean canTransform(Type type) { } private static class BucketByteBuffer extends Bucket { - private static final Set SUPPORTED_TYPES = Sets.newHashSet( - TypeID.BINARY, TypeID.FIXED); + private static final Set SUPPORTED_TYPES = Sets.newHashSet( + Type.TypeID.BINARY, Type.TypeID.FIXED); - private BucketByteBuffer(int N) { - super(N); + private BucketByteBuffer(int numBuckets) { + super(numBuckets); } public int hash(ByteBuffer value) { @@ -273,15 +269,15 @@ public boolean canTransform(Type type) { } } - private static class BucketUUID extends Bucket { + private static class BucketUuid extends Bucket { private static final ThreadLocal BUFFER = ThreadLocal.withInitial(() -> { ByteBuffer buffer = ByteBuffer.allocate(16); buffer.order(ByteOrder.BIG_ENDIAN); return buffer; }); - private BucketUUID(int N) { - super(N); + private BucketUuid(int numBuckets) { + super(numBuckets); } public int hash(UUID value) { @@ -294,13 +290,13 @@ public int hash(UUID value) { @Override public boolean canTransform(Type type) { - return type.typeId() == TypeID.UUID; + return type.typeId() == Type.TypeID.UUID; } } private static class BucketDecimal extends Bucket { - private BucketDecimal(int N) { - super(N); + private BucketDecimal(int numBuckets) { + super(numBuckets); } public int hash(BigDecimal value) { @@ -309,7 +305,7 @@ public int hash(BigDecimal value) { @Override public boolean canTransform(Type type) { - return type.typeId() == TypeID.DECIMAL; + return type.typeId() == Type.TypeID.DECIMAL; } } } diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Dates.java b/api/src/main/java/com/netflix/iceberg/transforms/Dates.java index 0b45f0b0ebba..c3facb1f1890 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Dates.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/Dates.java @@ -19,6 +19,9 @@ package com.netflix.iceberg.transforms; +import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; +import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; + import com.netflix.iceberg.expressions.BoundPredicate; import com.netflix.iceberg.expressions.Expressions; import com.netflix.iceberg.expressions.UnboundPredicate; @@ -29,9 +32,6 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; -import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; -import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; - enum Dates implements Transform { YEAR(ChronoUnit.YEARS, "year"), MONTH(ChronoUnit.MONTHS, "month"), @@ -65,15 +65,15 @@ public Type getResultType(Type sourceType) { } @Override - public UnboundPredicate project(String name, BoundPredicate pred) { + public UnboundPredicate project(String nameToProject, BoundPredicate pred) { if (pred.op() == NOT_NULL || pred.op() == IS_NULL) { - return Expressions.predicate(pred.op(), name); + return Expressions.predicate(pred.op(), nameToProject); } - return ProjectionUtil.truncateInteger(name, pred, this); + return ProjectionUtil.truncateInteger(nameToProject, pred, this); } @Override - public UnboundPredicate projectStrict(String name, BoundPredicate predicate) { + public UnboundPredicate projectStrict(String nameToProject, BoundPredicate predicate) { return null; } diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Identity.java b/api/src/main/java/com/netflix/iceberg/transforms/Identity.java index c1620ed17901..e95b5df95d2c 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Identity.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/Identity.java @@ -45,8 +45,8 @@ public T apply(T value) { } @Override - public boolean canTransform(Type type) { - return type.isPrimitiveType(); + public boolean canTransform(Type typeToTransform) { + return typeToTransform.isPrimitiveType(); } @Override @@ -105,15 +105,15 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; } - if (o == null || getClass() != o.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } - Identity that = (Identity) o; + Identity that = (Identity) other; return type.equals(that.type); } diff --git a/api/src/main/java/com/netflix/iceberg/transforms/PartitionSpecVisitor.java b/api/src/main/java/com/netflix/iceberg/transforms/PartitionSpecVisitor.java index 0de992adfba2..c05b90af87af 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/PartitionSpecVisitor.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/PartitionSpecVisitor.java @@ -40,6 +40,8 @@ public interface PartitionSpecVisitor { T hour(String sourceName, int sourceId); + // Suppressing cyclomatic complexity because of the instanceof checks and inherent branching. + @SuppressWarnings("checkstyle:CyclomaticComplexity") static List visit(Schema schema, PartitionSpec spec, PartitionSpecVisitor visitor) { List results = Lists.newArrayListWithExpectedSize(spec.fields().size()); diff --git a/api/src/main/java/com/netflix/iceberg/transforms/ProjectionUtil.java b/api/src/main/java/com/netflix/iceberg/transforms/ProjectionUtil.java index 06f0067fb49e..e269250b3c2b 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/ProjectionUtil.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/ProjectionUtil.java @@ -21,29 +21,31 @@ import com.netflix.iceberg.expressions.BoundPredicate; import com.netflix.iceberg.expressions.Expression; +import com.netflix.iceberg.expressions.Expressions; import com.netflix.iceberg.expressions.UnboundPredicate; import java.math.BigDecimal; import java.math.BigInteger; -import static com.netflix.iceberg.expressions.Expressions.predicate; - class ProjectionUtil { + + private ProjectionUtil() {} + static UnboundPredicate truncateInteger( String name, BoundPredicate pred, Transform transform) { int boundary = pred.literal().value(); switch (pred.op()) { case LT: // adjust closed and then transform ltEq - return predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary - 1)); + return Expressions.predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary - 1)); case LT_EQ: - return predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); case GT: // adjust closed and then transform gtEq - return predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary + 1)); + return Expressions.predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary + 1)); case GT_EQ: - return predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); case EQ: - return predicate(pred.op(), name, transform.apply(boundary)); + return Expressions.predicate(pred.op(), name, transform.apply(boundary)); default: return null; } @@ -55,16 +57,16 @@ static UnboundPredicate truncateLong( switch (pred.op()) { case LT: // adjust closed and then transform ltEq - return predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary - 1L)); + return Expressions.predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary - 1L)); case LT_EQ: - return predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); case GT: // adjust closed and then transform gtEq - return predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary + 1L)); + return Expressions.predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary + 1L)); case GT_EQ: - return predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); case EQ: - return predicate(pred.op(), name, transform.apply(boundary)); + return Expressions.predicate(pred.op(), name, transform.apply(boundary)); default: return null; } @@ -80,19 +82,19 @@ static UnboundPredicate truncateDecimal( BigDecimal minusOne = new BigDecimal( boundary.unscaledValue().subtract(BigInteger.ONE), boundary.scale()); - return predicate(Expression.Operation.LT_EQ, name, transform.apply(minusOne)); + return Expressions.predicate(Expression.Operation.LT_EQ, name, transform.apply(minusOne)); case LT_EQ: - return predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); case GT: // adjust closed and then transform gtEq BigDecimal plusOne = new BigDecimal( boundary.unscaledValue().add(BigInteger.ONE), boundary.scale()); - return predicate(Expression.Operation.GT_EQ, name, transform.apply(plusOne)); + return Expressions.predicate(Expression.Operation.GT_EQ, name, transform.apply(plusOne)); case GT_EQ: - return predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); case EQ: - return predicate(pred.op(), name, transform.apply(boundary)); + return Expressions.predicate(pred.op(), name, transform.apply(boundary)); default: return null; } @@ -104,12 +106,12 @@ static UnboundPredicate truncateArray( switch (pred.op()) { case LT: case LT_EQ: - return predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.LT_EQ, name, transform.apply(boundary)); case GT: case GT_EQ: - return predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.GT_EQ, name, transform.apply(boundary)); case EQ: - return predicate(Expression.Operation.EQ, name, transform.apply(boundary)); + return Expressions.predicate(Expression.Operation.EQ, name, transform.apply(boundary)); // case IN: // TODO // return Expressions.predicate(Operation.IN, name, transform.apply(boundary)); default: diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Timestamps.java b/api/src/main/java/com/netflix/iceberg/transforms/Timestamps.java index 7977e5bb97e0..82b3d59fc938 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Timestamps.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/Timestamps.java @@ -19,6 +19,9 @@ package com.netflix.iceberg.transforms; +import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; +import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; + import com.netflix.iceberg.expressions.BoundPredicate; import com.netflix.iceberg.expressions.Expressions; import com.netflix.iceberg.expressions.UnboundPredicate; @@ -29,9 +32,6 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; -import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; -import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; - enum Timestamps implements Transform { YEAR(ChronoUnit.YEARS, "year"), MONTH(ChronoUnit.MONTHS, "month"), @@ -53,7 +53,8 @@ public Integer apply(Long timestampMicros) { OffsetDateTime timestamp = Instant .ofEpochSecond(timestampMicros / 1_000_000) .atOffset(ZoneOffset.UTC); - return (int) granularity.between(EPOCH, timestamp); + Integer year = Long.valueOf(granularity.between(EPOCH, timestamp)).intValue(); + return year; } @Override @@ -67,15 +68,15 @@ public Type getResultType(Type sourceType) { } @Override - public UnboundPredicate project(String name, BoundPredicate pred) { + public UnboundPredicate project(String nameToProject, BoundPredicate pred) { if (pred.op() == NOT_NULL || pred.op() == IS_NULL) { - return Expressions.predicate(pred.op(), name); + return Expressions.predicate(pred.op(), nameToProject); } - return ProjectionUtil.truncateLong(name, pred, this); + return ProjectionUtil.truncateLong(nameToProject, pred, this); } @Override - public UnboundPredicate projectStrict(String name, BoundPredicate predicate) { + public UnboundPredicate projectStrict(String nameToProject, BoundPredicate predicate) { return null; } diff --git a/api/src/main/java/com/netflix/iceberg/transforms/TransformUtil.java b/api/src/main/java/com/netflix/iceberg/transforms/TransformUtil.java index 8a71d44659b5..bbfc5d98e244 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/TransformUtil.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/TransformUtil.java @@ -19,8 +19,8 @@ package com.netflix.iceberg.transforms; -import com.google.common.base.Charsets; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.LocalTime; import java.time.OffsetDateTime; @@ -29,6 +29,8 @@ import java.util.Base64; class TransformUtil { + private TransformUtil() {} + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); private static final int EPOCH_YEAR = EPOCH.getYear(); @@ -66,6 +68,6 @@ static String humanHour(int hourOrdinal) { static String base64encode(ByteBuffer buffer) { // use direct encoding because all of the encoded bytes are in ASCII - return Charsets.ISO_8859_1.decode(Base64.getEncoder().encode(buffer)).toString(); + return StandardCharsets.ISO_8859_1.decode(Base64.getEncoder().encode(buffer)).toString(); } } diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Transforms.java b/api/src/main/java/com/netflix/iceberg/transforms/Transforms.java index 044fe8928662..969a725dde7b 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Transforms.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/Transforms.java @@ -20,8 +20,8 @@ package com.netflix.iceberg.transforms; import com.google.common.base.Preconditions; -import com.netflix.iceberg.Schema; import com.netflix.iceberg.PartitionSpec; +import com.netflix.iceberg.Schema; import com.netflix.iceberg.types.Type; import java.util.Locale; import java.util.regex.Matcher; @@ -45,11 +45,11 @@ private Transforms() { Matcher width = HAS_WIDTH.matcher(transform); if (width.matches()) { String name = width.group(1); - int w = Integer.parseInt(width.group(2)); + int parsedWidth = Integer.parseInt(width.group(2)); if (name.equalsIgnoreCase("truncate")) { - return Truncate.get(type, w); + return Truncate.get(type, parsedWidth); } else if (name.equals("bucket")) { - return Bucket.get(type, w); + return Bucket.get(type, parsedWidth); } } diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Truncate.java b/api/src/main/java/com/netflix/iceberg/transforms/Truncate.java index d2542079f366..bb9bd83fce76 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Truncate.java +++ b/api/src/main/java/com/netflix/iceberg/transforms/Truncate.java @@ -19,6 +19,11 @@ package com.netflix.iceberg.transforms; +import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; +import static com.netflix.iceberg.expressions.Expression.Operation.LT; +import static com.netflix.iceberg.expressions.Expression.Operation.LT_EQ; +import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; + import com.google.common.base.Objects; import com.netflix.iceberg.expressions.BoundPredicate; import com.netflix.iceberg.expressions.Expressions; @@ -28,11 +33,6 @@ import java.math.BigInteger; import java.nio.ByteBuffer; -import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; -import static com.netflix.iceberg.expressions.Expression.Operation.LT; -import static com.netflix.iceberg.expressions.Expression.Operation.LT_EQ; -import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; - abstract class Truncate implements Transform { @SuppressWarnings("unchecked") static Truncate get(Type type, int width) { @@ -53,10 +53,10 @@ static Truncate get(Type type, int width) { } } - abstract public Integer width(); + public abstract Integer width(); @Override - abstract public T apply(T value); + public abstract T apply(T value); @Override public Type getResultType(Type sourceType) { @@ -64,20 +64,20 @@ public Type getResultType(Type sourceType) { } private static class TruncateInteger extends Truncate { - private final int W; + private final int width; private TruncateInteger(int width) { - this.W = width; + this.width = width; } @Override public Integer width() { - return W; + return width; } @Override public Integer apply(Integer value) { - return value - (((value % W) + W) % W); + return value - (((value % width) + width) % width); } @Override @@ -137,44 +137,44 @@ public UnboundPredicate projectStrict(String name, BoundPredicate { - private final int W; + private final int width; private TruncateLong(int width) { - this.W = width; + this.width = width; } @Override public Integer width() { - return W; + return width; } @Override public Long apply(Long value) { - return value - (((value % W) + W) % W); + return value - (((value % width) + width) % width); } @Override @@ -196,44 +196,44 @@ public UnboundPredicate projectStrict(String name, BoundPredicate pr } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; } - if (o == null || getClass() != o.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } - TruncateLong that = (TruncateLong) o; - return W == that.W; + TruncateLong that = (TruncateLong) other; + return width == that.width; } @Override public int hashCode() { - return Objects.hashCode(W); + return Objects.hashCode(width); } @Override public String toString() { - return "truncate[" + W + "]"; + return "truncate[" + width + "]"; } } private static class TruncateString extends Truncate { - private final int L; + private final int length; private TruncateString(int length) { - this.L = length; + this.length = length; } @Override public Integer width() { - return L; + return length; } @Override public CharSequence apply(CharSequence value) { - return value.subSequence(0, Math.min(value.length(), L)); + return value.subSequence(0, Math.min(value.length(), length)); } @Override @@ -257,45 +257,45 @@ public UnboundPredicate projectStrict(String name, } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; } - if (o == null || getClass() != o.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } - TruncateString that = (TruncateString) o; - return L == that.L; + TruncateString that = (TruncateString) other; + return length == that.length; } @Override public int hashCode() { - return Objects.hashCode(L); + return Objects.hashCode(length); } @Override public String toString() { - return "truncate[" + L + "]"; + return "truncate[" + length + "]"; } } private static class TruncateByteBuffer extends Truncate { - private final int L; + private final int length; private TruncateByteBuffer(int length) { - this.L = length; + this.length = length; } @Override public Integer width() { - return L; + return length; } @Override public ByteBuffer apply(ByteBuffer value) { ByteBuffer ret = value.duplicate(); - ret.limit(Math.min(value.limit(), value.position() + L)); + ret.limit(Math.min(value.limit(), value.position() + length)); return ret; } @@ -320,21 +320,21 @@ public UnboundPredicate projectStrict(String name, } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; } - if (o == null || getClass() != o.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } - TruncateByteBuffer that = (TruncateByteBuffer) o; - return L == that.L; + TruncateByteBuffer that = (TruncateByteBuffer) other; + return length == that.length; } @Override public int hashCode() { - return Objects.hashCode(L); + return Objects.hashCode(length); } @Override @@ -344,7 +344,7 @@ public String toHumanString(ByteBuffer value) { @Override public String toString() { - return "truncate[" + L + "]"; + return "truncate[" + length + "]"; } } @@ -392,15 +392,15 @@ public UnboundPredicate projectStrict(String name, } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; } - if (o == null || getClass() != o.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } - TruncateDecimal that = (TruncateDecimal) o; + TruncateDecimal that = (TruncateDecimal) other; return unscaledWidth.equals(that.unscaledWidth); } diff --git a/api/src/main/java/com/netflix/iceberg/types/CheckCompatibility.java b/api/src/main/java/com/netflix/iceberg/types/CheckCompatibility.java index 48ebb8b71b3c..4467756946b8 100644 --- a/api/src/main/java/com/netflix/iceberg/types/CheckCompatibility.java +++ b/api/src/main/java/com/netflix/iceberg/types/CheckCompatibility.java @@ -29,8 +29,6 @@ import java.util.Map; import java.util.function.Supplier; -import static com.netflix.iceberg.types.TypeUtil.isPromotionAllowed; - public class CheckCompatibility extends TypeUtil.CustomOrderSchemaVisitor> { /** * Returns a list of compatibility errors for writing with the given write schema. @@ -217,7 +215,7 @@ public List primitive(Type.PrimitiveType readPrimitive) { currentType.typeId().toString().toLowerCase(Locale.ENGLISH), readPrimitive)); } - if (!isPromotionAllowed(currentType.asPrimitiveType(), readPrimitive)) { + if (!TypeUtil.isPromotionAllowed(currentType.asPrimitiveType(), readPrimitive)) { return ImmutableList.of(String.format(": %s cannot be promoted to %s", currentType, readPrimitive)); } diff --git a/api/src/main/java/com/netflix/iceberg/types/Comparators.java b/api/src/main/java/com/netflix/iceberg/types/Comparators.java index 6680f7dd16ff..5fc6fc4cdf57 100644 --- a/api/src/main/java/com/netflix/iceberg/types/Comparators.java +++ b/api/src/main/java/com/netflix/iceberg/types/Comparators.java @@ -24,6 +24,9 @@ import java.util.Comparator; public class Comparators { + + private Comparators() {} + private static final ImmutableMap> COMPARATORS = ImmutableMap .>builder() .put(Types.BooleanType.get(), Comparator.naturalOrder()) @@ -130,7 +133,7 @@ private static class NullSafeChainedComparator implements Comparator { private final Comparator first; private final Comparator second; - public NullSafeChainedComparator(Comparator first, Comparator second) { + NullSafeChainedComparator(Comparator first, Comparator second) { this.first = first; this.second = second; } diff --git a/api/src/main/java/com/netflix/iceberg/types/Conversions.java b/api/src/main/java/com/netflix/iceberg/types/Conversions.java index cfef71d133da..cf6597ca25b4 100644 --- a/api/src/main/java/com/netflix/iceberg/types/Conversions.java +++ b/api/src/main/java/com/netflix/iceberg/types/Conversions.java @@ -19,7 +19,6 @@ package com.netflix.iceberg.types; -import com.google.common.base.Charsets; import com.netflix.iceberg.exceptions.RuntimeIOException; import java.math.BigDecimal; import java.math.BigInteger; @@ -29,10 +28,14 @@ import java.nio.charset.CharacterCodingException; import java.nio.charset.CharsetDecoder; import java.nio.charset.CharsetEncoder; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.UUID; public class Conversions { + + private Conversions() {} + private static final String HIVE_NULL = "__HIVE_DEFAULT_PARTITION__"; public static Object fromPartitionString(Type type, String asString) { @@ -58,9 +61,9 @@ public static Object fromPartitionString(Type type, String asString) { case FIXED: Types.FixedType fixed = (Types.FixedType) type; return Arrays.copyOf( - asString.getBytes(Charsets.UTF_8), fixed.length()); + asString.getBytes(StandardCharsets.UTF_8), fixed.length()); case BINARY: - return asString.getBytes(Charsets.UTF_8); + return asString.getBytes(StandardCharsets.UTF_8); case DECIMAL: return new BigDecimal(asString); default: @@ -70,9 +73,9 @@ public static Object fromPartitionString(Type type, String asString) { } private static final ThreadLocal ENCODER = - ThreadLocal.withInitial(Charsets.UTF_8::newEncoder); + ThreadLocal.withInitial(StandardCharsets.UTF_8::newEncoder); private static final ThreadLocal DECODER = - ThreadLocal.withInitial(Charsets.UTF_8::newDecoder); + ThreadLocal.withInitial(StandardCharsets.UTF_8::newDecoder); public static ByteBuffer toByteBuffer(Type type, Object value) { switch (type.typeId()) { @@ -120,7 +123,7 @@ private static Object internalFromByteBuffer(Type type, ByteBuffer buffer) { ByteBuffer tmp = buffer.duplicate().order(ByteOrder.LITTLE_ENDIAN); switch (type.typeId()) { case BOOLEAN: - return (tmp.get() != 0x00); + return tmp.get() != 0x00; case INTEGER: case DATE: return tmp.getInt(); diff --git a/api/src/main/java/com/netflix/iceberg/types/IndexById.java b/api/src/main/java/com/netflix/iceberg/types/IndexById.java index 84d8061e19ed..ef47485b7786 100644 --- a/api/src/main/java/com/netflix/iceberg/types/IndexById.java +++ b/api/src/main/java/com/netflix/iceberg/types/IndexById.java @@ -33,7 +33,8 @@ public Map schema(Schema schema, Map struct(Types.StructType struct, List> fieldResults) { + public Map struct( + Types.StructType struct, List> fieldResults) { return index; } @@ -52,7 +53,8 @@ public Map list(Types.ListType list, Map map(Types.MapType map, Map keyResult, Map valueResult) { + public Map map( + Types.MapType map, Map keyResult, Map valueResult) { for (Types.NestedField field : map.fields()) { index.put(field.fieldId(), field); } diff --git a/api/src/main/java/com/netflix/iceberg/types/IndexByName.java b/api/src/main/java/com/netflix/iceberg/types/IndexByName.java index e98a3ba8e99b..95b666188e43 100644 --- a/api/src/main/java/com/netflix/iceberg/types/IndexByName.java +++ b/api/src/main/java/com/netflix/iceberg/types/IndexByName.java @@ -64,8 +64,8 @@ public Map map(Types.MapType map, Map keyResul private void addField(String name, int fieldId) { String fullName = name; - if (!fieldNames.isEmpty()) { - fullName = DOT.join(DOT.join(fieldNames.descendingIterator()), name); + if (!fieldNames().isEmpty()) { + fullName = DOT.join(DOT.join(fieldNames().descendingIterator()), name); } nameToId.put(fullName, fieldId); } diff --git a/api/src/main/java/com/netflix/iceberg/types/PruneColumns.java b/api/src/main/java/com/netflix/iceberg/types/PruneColumns.java index c4300e726dc1..ea699f6b76b4 100644 --- a/api/src/main/java/com/netflix/iceberg/types/PruneColumns.java +++ b/api/src/main/java/com/netflix/iceberg/types/PruneColumns.java @@ -27,7 +27,7 @@ class PruneColumns extends TypeUtil.SchemaVisitor { private final Set selected; - public PruneColumns(Set selected) { + PruneColumns(Set selected) { this.selected = selected; } diff --git a/api/src/main/java/com/netflix/iceberg/types/ReassignIds.java b/api/src/main/java/com/netflix/iceberg/types/ReassignIds.java index a81c2e18ab3a..4fee09173f98 100644 --- a/api/src/main/java/com/netflix/iceberg/types/ReassignIds.java +++ b/api/src/main/java/com/netflix/iceberg/types/ReassignIds.java @@ -46,7 +46,7 @@ public Type schema(Schema schema, Supplier future) { @Override public Type struct(Types.StructType struct, Iterable fieldTypes) { Preconditions.checkNotNull(sourceType, "Evaluation must start with a schema."); - Preconditions.checkArgument(sourceType.isStructType(), "Not a struct: " + sourceType); + Preconditions.checkArgument(sourceType.isStructType(), "Not a struct: %s", sourceType); Types.StructType sourceStruct = sourceType.asStructType(); List fields = struct.fields(); @@ -69,7 +69,7 @@ public Type struct(Types.StructType struct, Iterable fieldTypes) { @Override public Type field(Types.NestedField field, Supplier future) { - Preconditions.checkArgument(sourceType.isStructType(), "Not a struct: " + sourceType); + Preconditions.checkArgument(sourceType.isStructType(), "Not a struct: %s", sourceType); Types.StructType sourceStruct = sourceType.asStructType(); Types.NestedField sourceField = sourceStruct.field(field.name()); @@ -84,7 +84,7 @@ public Type field(Types.NestedField field, Supplier future) { @Override public Type list(Types.ListType list, Supplier elementTypeFuture) { - Preconditions.checkArgument(sourceType.isListType(), "Not a list: " + sourceType); + Preconditions.checkArgument(sourceType.isListType(), "Not a list: %s", sourceType); Types.ListType sourceList = sourceType.asListType(); int sourceElementId = sourceList.elementId(); @@ -104,7 +104,7 @@ public Type list(Types.ListType list, Supplier elementTypeFuture) { @Override public Type map(Types.MapType map, Supplier keyTypeFuture, Supplier valueTypeFuture) { - Preconditions.checkArgument(sourceType.isMapType(), "Not a map: " + sourceType); + Preconditions.checkArgument(sourceType.isMapType(), "Not a map: %s", sourceType); Types.MapType sourceMap = sourceType.asMapType(); int sourceKeyId = sourceMap.keyId(); diff --git a/api/src/main/java/com/netflix/iceberg/types/TypeUtil.java b/api/src/main/java/com/netflix/iceberg/types/TypeUtil.java index 6f2ca7886ad8..a78bcb347b37 100644 --- a/api/src/main/java/com/netflix/iceberg/types/TypeUtil.java +++ b/api/src/main/java/com/netflix/iceberg/types/TypeUtil.java @@ -25,7 +25,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.netflix.iceberg.Schema; -import java.util.LinkedList; +import java.util.Deque; import java.util.List; import java.util.Map; import java.util.Set; @@ -33,6 +33,9 @@ import java.util.function.Supplier; public class TypeUtil { + + private TypeUtil() {} + public static Schema select(Schema schema, Set fieldIds) { Preconditions.checkNotNull(schema, "Schema cannot be null"); Preconditions.checkNotNull(fieldIds, "Field ids cannot be null"); @@ -151,8 +154,8 @@ public static boolean isPromotionAllowed(Type from, Type.PrimitiveType to) { } Types.DecimalType toDecimal = (Types.DecimalType) to; - return (fromDecimal.scale() == toDecimal.scale() && - fromDecimal.precision() <= toDecimal.precision()); + return fromDecimal.scale() == toDecimal.scale() + && fromDecimal.precision() <= toDecimal.precision(); } return false; @@ -166,8 +169,16 @@ public interface NextID { } public static class SchemaVisitor { - protected LinkedList fieldNames = Lists.newLinkedList(); - protected LinkedList fieldIds = Lists.newLinkedList(); + private final Deque fieldNames = Lists.newLinkedList(); + private final Deque fieldIds = Lists.newLinkedList(); + + protected Deque fieldNames() { + return fieldNames; + } + + protected Deque fieldIds() { + return fieldIds; + } public T schema(Schema schema, T structResult) { return null; @@ -360,23 +371,23 @@ public static T visit(Type type, CustomOrderSchemaVisitor visitor) { static int decimalMaxPrecision(int numBytes) { Preconditions.checkArgument(numBytes >= 0 && numBytes < 24, - "Unsupported decimal length: " + numBytes); + "Unsupported decimal length: %d", numBytes); return MAX_PRECISION[numBytes]; } public static int decimalRequriedBytes(int precision) { Preconditions.checkArgument(precision >= 0 && precision < 40, - "Unsupported decimal precision: " + precision); + "Unsupported decimal precision: %d", precision); return REQUIRED_LENGTH[precision]; } - private static int[] MAX_PRECISION = new int[24]; - private static int[] REQUIRED_LENGTH = new int[40]; + private static final int[] MAX_PRECISION = new int[24]; + private static final int[] REQUIRED_LENGTH = new int[40]; static { // for each length, calculate the max precision for (int len = 0; len < MAX_PRECISION.length; len += 1) { - MAX_PRECISION[len] = (int) Math.floor(Math.log10(Math.pow(2, 8*len - 1) - 1)); + MAX_PRECISION[len] = (int) Math.floor(Math.log10(Math.pow(2, 8 * len - 1) - 1)); } // for each precision, find the first length that can hold it diff --git a/api/src/main/java/com/netflix/iceberg/types/Types.java b/api/src/main/java/com/netflix/iceberg/types/Types.java index f917a195bd44..7719bc5c4937 100644 --- a/api/src/main/java/com/netflix/iceberg/types/Types.java +++ b/api/src/main/java/com/netflix/iceberg/types/Types.java @@ -35,6 +35,9 @@ import java.util.regex.Pattern; public class Types { + + private Types() {} + private static final ImmutableMap TYPES = ImmutableMap .builder() .put(BooleanType.get().toString(), BooleanType.get()) @@ -241,14 +244,14 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (other == null || getClass() != other.getClass()) { return false; } - TimestampType timestampType = (TimestampType) o; + TimestampType timestampType = (TimestampType) other; return adjustToUTC == timestampType.adjustToUTC; } @@ -320,14 +323,14 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (other == null || getClass() != other.getClass()) { return false; } - FixedType fixedType = (FixedType) o; + FixedType fixedType = (FixedType) other; return length == fixedType.length; } @@ -389,14 +392,14 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (other == null || getClass() != other.getClass()) { return false; } - DecimalType that = (DecimalType) o; + DecimalType that = (DecimalType) other; if (scale != that.scale) { return false; } @@ -459,14 +462,14 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (other == null || getClass() != other.getClass()) { return false; } - NestedField that = (NestedField) o; + NestedField that = (NestedField) other; if (isOptional != that.isOptional) { return false; } else if (id != that.id) { @@ -518,6 +521,11 @@ public NestedField field(String name) { return lazyFieldsByName().get(name); } + @Override + public NestedField field(int id) { + return lazyFieldsById().get(id); + } + @Override public Type fieldType(String name) { NestedField field = field(name); @@ -527,11 +535,6 @@ public Type fieldType(String name) { return null; } - @Override - public NestedField field(int id) { - return lazyFieldsById().get(id); - } - @Override public TypeID typeId() { return TypeID.STRUCT; @@ -553,14 +556,14 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (other == null || getClass() != other.getClass()) { return false; } - StructType that = (StructType) o; + StructType that = (StructType) other; return Arrays.equals(fields, that.fields); } @@ -678,14 +681,14 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (other == null || getClass() != other.getClass()) { return false; } - ListType listType = (ListType) o; + ListType listType = (ListType) other; return elementField.equals(listType.elementField); } @@ -796,14 +799,14 @@ public String toString() { } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object other) { + if (this == other) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (other == null || getClass() != other.getClass()) { return false; } - MapType mapType = (MapType) o; + MapType mapType = (MapType) other; if (!keyField.equals(mapType.keyField)) { return false; } diff --git a/api/src/test/java/com/netflix/iceberg/TestHelpers.java b/api/src/test/java/com/netflix/iceberg/TestHelpers.java index ceb1eed280cd..4a9e3b059084 100644 --- a/api/src/test/java/com/netflix/iceberg/TestHelpers.java +++ b/api/src/test/java/com/netflix/iceberg/TestHelpers.java @@ -23,7 +23,6 @@ import com.netflix.iceberg.expressions.Expression; import com.netflix.iceberg.expressions.ExpressionVisitors; import com.netflix.iceberg.expressions.UnboundPredicate; -import org.junit.Assert; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -33,8 +32,12 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; +import org.junit.Assert; public class TestHelpers { + + private TestHelpers() {} + public static T assertAndUnwrap(Expression expr, Class expected) { Assert.assertTrue("Expression should have expected type: " + expected, expected.isInstance(expr)); @@ -75,7 +78,7 @@ public static T roundTripSerialize(T type) throws IOException, ClassNotFound private static class CheckReferencesBound extends ExpressionVisitors.ExpressionVisitor { private final String message; - public CheckReferencesBound(String message) { + CheckReferencesBound(String message) { this.message = message; } @@ -131,8 +134,8 @@ public static void assertThrows(String message, Callable callable) { try { callable.call(); - Assert.fail("No exception was thrown (" + message + "), expected: " + - expected.getName()); + Assert.fail("No exception was thrown (" + message + "), expected: " + + expected.getName()); } catch (Exception actual) { handleException(message, expected, containedInMessage, actual); } @@ -152,8 +155,8 @@ public static void assertThrows(String message, Runnable runnable) { try { runnable.run(); - Assert.fail("No exception was thrown (" + message + "), expected: " + - expected.getName()); + Assert.fail("No exception was thrown (" + message + "), expected: " + + expected.getName()); } catch (Exception actual) { handleException(message, expected, containedInMessage, actual); } @@ -166,8 +169,8 @@ private static void handleException(String message, try { Assert.assertEquals(message, expected, actual.getClass()); Assert.assertTrue( - "Expected exception message (" + containedInMessage + ") missing: " + - actual.getMessage(), + "Expected exception message (" + containedInMessage + ") missing: " + + actual.getMessage(), actual.getMessage().contains(containedInMessage) ); } catch (AssertionError e) { diff --git a/api/src/test/java/com/netflix/iceberg/events/TestListeners.java b/api/src/test/java/com/netflix/iceberg/events/TestListeners.java index 2f2052796db4..bee9576e73e5 100644 --- a/api/src/test/java/com/netflix/iceberg/events/TestListeners.java +++ b/api/src/test/java/com/netflix/iceberg/events/TestListeners.java @@ -36,19 +36,20 @@ public static class Event2 { public static class TestListener { private static final TestListener INSTANCE = new TestListener(); + public static TestListener get() { return INSTANCE; } - private Event1 e1 = null; - private Event2 e2 = null; + private Event1 event1 = null; + private Event2 event2 = null; - public void event1(Event1 e) { - this.e1 = e; + public void event1(Event1 receivedEvent1) { + this.event1 = receivedEvent1; } - public void event2(Event2 e) { - this.e2 = e; + public void event2(Event2 receivedEvent2) { + this.event2 = receivedEvent2; } } @@ -58,7 +59,7 @@ public void testEvent1() { Listeners.notifyAll(e1); - Assert.assertEquals(e1, TestListener.get().e1); + Assert.assertEquals(e1, TestListener.get().event1); } @Test @@ -67,7 +68,7 @@ public void testEvent2() { Listeners.notifyAll(e2); - Assert.assertEquals(e2, TestListener.get().e2); + Assert.assertEquals(e2, TestListener.get().event2); } @Test @@ -79,7 +80,7 @@ public void testMultipleListeners() { Listeners.notifyAll(e1); - Assert.assertEquals(e1, TestListener.get().e1); - Assert.assertEquals(e1, other.e1); + Assert.assertEquals(e1, TestListener.get().event1); + Assert.assertEquals(e1, other.event1); } } diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestEvaluatior.java b/api/src/test/java/com/netflix/iceberg/expressions/TestEvaluatior.java index 4723f83a0977..119840c35b28 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestEvaluatior.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestEvaluatior.java @@ -19,13 +19,6 @@ package com.netflix.iceberg.expressions; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; -import org.apache.avro.util.Utf8; -import org.junit.Assert; -import org.junit.Test; - import static com.netflix.iceberg.expressions.Expressions.alwaysFalse; import static com.netflix.iceberg.expressions.Expressions.alwaysTrue; import static com.netflix.iceberg.expressions.Expressions.and; @@ -42,6 +35,13 @@ import static com.netflix.iceberg.types.Types.NestedField.optional; import static com.netflix.iceberg.types.Types.NestedField.required; +import com.netflix.iceberg.TestHelpers; +import com.netflix.iceberg.types.Types; +import com.netflix.iceberg.types.Types.StructType; +import org.apache.avro.util.Utf8; +import org.junit.Assert; +import org.junit.Test; + public class TestEvaluatior { private static final StructType STRUCT = StructType.of( required(13, "x", Types.IntegerType.get()), diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionBinding.java b/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionBinding.java index 14b95b0760d5..da24ce07bdc4 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionBinding.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionBinding.java @@ -19,13 +19,6 @@ package com.netflix.iceberg.expressions; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; -import org.junit.Assert; -import org.junit.Test; - import static com.netflix.iceberg.expressions.Expressions.alwaysFalse; import static com.netflix.iceberg.expressions.Expressions.alwaysTrue; import static com.netflix.iceberg.expressions.Expressions.and; @@ -36,6 +29,13 @@ import static com.netflix.iceberg.expressions.Expressions.or; import static com.netflix.iceberg.types.Types.NestedField.required; +import com.netflix.iceberg.TestHelpers; +import com.netflix.iceberg.exceptions.ValidationException; +import com.netflix.iceberg.types.Types; +import com.netflix.iceberg.types.Types.StructType; +import org.junit.Assert; +import org.junit.Test; + public class TestExpressionBinding { private static final StructType STRUCT = StructType.of( required(0, "x", Types.IntegerType.get()), diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionHelpers.java b/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionHelpers.java index e67d31caa6ef..c3cea8cb59bd 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionHelpers.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionHelpers.java @@ -19,10 +19,6 @@ package com.netflix.iceberg.expressions; -import com.netflix.iceberg.TestHelpers; -import org.junit.Assert; -import org.junit.Test; - import static com.netflix.iceberg.expressions.Expressions.alwaysFalse; import static com.netflix.iceberg.expressions.Expressions.alwaysTrue; import static com.netflix.iceberg.expressions.Expressions.and; @@ -31,6 +27,10 @@ import static com.netflix.iceberg.expressions.Expressions.not; import static com.netflix.iceberg.expressions.Expressions.or; +import com.netflix.iceberg.TestHelpers; +import org.junit.Assert; +import org.junit.Test; + public class TestExpressionHelpers { private final UnboundPredicate pred = lessThan("x", 7); diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionSerialization.java b/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionSerialization.java index 1c6c564e237a..cf9dbfb64758 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionSerialization.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionSerialization.java @@ -82,15 +82,11 @@ private static boolean equals(Expression left, Expression right) { case NOT: return equals(((Not) left).child(), ((Not) right).child()); case AND: - return ( - equals(((And) left).left(), ((And) right).left()) && - equals(((And) left).right(), ((And) right).right()) - ); + return equals(((And) left).left(), ((And) right).left()) + && equals(((And) left).right(), ((And) right).right()); case OR: - return ( - equals(((Or) left).left(), ((Or) right).left()) && - equals(((Or) left).right(), ((Or) right).right()) - ); + return equals(((Or) left).left(), ((Or) right).left()) + && equals(((Or) left).right(), ((Or) right).right()); default: return false; } @@ -123,7 +119,7 @@ private static boolean equals(Reference left, Reference right) { NamedReference lref = (NamedReference) left; NamedReference rref = (NamedReference) right; - return lref.name.equals(rref.name); + return lref.name().equals(rref.name()); } else if (left instanceof BoundReference) { if (!(right instanceof BoundReference)) { @@ -133,10 +129,8 @@ private static boolean equals(Reference left, Reference right) { BoundReference lref = (BoundReference) left; BoundReference rref = (BoundReference) right; - return ( - lref.fieldId() == rref.fieldId() && - lref.type().equals(rref.type()) - ); + return lref.fieldId() == rref.fieldId() + && lref.type().equals(rref.type()); } return false; diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveManifestEvaluator.java b/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveManifestEvaluator.java index f92f70014fe1..d3d56120fed1 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveManifestEvaluator.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveManifestEvaluator.java @@ -19,18 +19,6 @@ package com.netflix.iceberg.expressions; -import com.google.common.collect.ImmutableList; -import com.netflix.iceberg.ManifestFile; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; - -import java.nio.ByteBuffer; - import static com.netflix.iceberg.expressions.Expressions.and; import static com.netflix.iceberg.expressions.Expressions.equal; import static com.netflix.iceberg.expressions.Expressions.greaterThan; @@ -46,6 +34,17 @@ import static com.netflix.iceberg.types.Types.NestedField.optional; import static com.netflix.iceberg.types.Types.NestedField.required; +import com.google.common.collect.ImmutableList; +import com.netflix.iceberg.ManifestFile; +import com.netflix.iceberg.PartitionSpec; +import com.netflix.iceberg.Schema; +import com.netflix.iceberg.TestHelpers; +import com.netflix.iceberg.exceptions.ValidationException; +import com.netflix.iceberg.types.Types; +import java.nio.ByteBuffer; +import org.junit.Assert; +import org.junit.Test; + public class TestInclusiveManifestEvaluator { private static final Schema SCHEMA = new Schema( required(1, "id", Types.IntegerType.get()), diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveMetricsEvaluator.java b/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveMetricsEvaluator.java index bbc96ceb52da..6f7425218264 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveMetricsEvaluator.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveMetricsEvaluator.java @@ -19,19 +19,6 @@ package com.netflix.iceberg.expressions; -import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.TestHelpers.Row; -import com.netflix.iceberg.TestHelpers.TestDataFile; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.IntegerType; -import org.junit.Assert; -import org.junit.Test; - - import static com.netflix.iceberg.expressions.Expressions.and; import static com.netflix.iceberg.expressions.Expressions.equal; import static com.netflix.iceberg.expressions.Expressions.greaterThan; @@ -47,6 +34,18 @@ import static com.netflix.iceberg.types.Types.NestedField.optional; import static com.netflix.iceberg.types.Types.NestedField.required; +import com.google.common.collect.ImmutableMap; +import com.netflix.iceberg.DataFile; +import com.netflix.iceberg.Schema; +import com.netflix.iceberg.TestHelpers; +import com.netflix.iceberg.TestHelpers.Row; +import com.netflix.iceberg.TestHelpers.TestDataFile; +import com.netflix.iceberg.exceptions.ValidationException; +import com.netflix.iceberg.types.Types; +import com.netflix.iceberg.types.Types.IntegerType; +import org.junit.Assert; +import org.junit.Test; + public class TestInclusiveMetricsEvaluator { private static final Schema SCHEMA = new Schema( required(1, "id", IntegerType.get()), diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestLiteralSerialization.java b/api/src/test/java/com/netflix/iceberg/expressions/TestLiteralSerialization.java index 5108c84cb3bb..4c058e6b76fa 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestLiteralSerialization.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestLiteralSerialization.java @@ -21,10 +21,10 @@ import com.netflix.iceberg.TestHelpers; import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; import java.math.BigDecimal; import java.util.UUID; +import org.junit.Assert; +import org.junit.Test; public class TestLiteralSerialization { @Test diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestMiscLiteralConversions.java b/api/src/test/java/com/netflix/iceberg/expressions/TestMiscLiteralConversions.java index 3947b1197dbb..2f971650a528 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestMiscLiteralConversions.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestMiscLiteralConversions.java @@ -21,13 +21,13 @@ import com.netflix.iceberg.types.Type; import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.UUID; +import org.junit.Assert; +import org.junit.Test; public class TestMiscLiteralConversions { @Test @@ -321,24 +321,24 @@ private void testInvalidConversions(Literal lit, Type... invalidTypes) { } private static class Pair { - public static Pair of(X x, Y y) { - return new Pair<>(x, y); + public static Pair of(X first, Y second) { + return new Pair<>(first, second); } - private final X x; - private final Y y; + private final X first; + private final Y second; - private Pair(X x, Y y) { - this.x = x; - this.y = y; + private Pair(X first, Y second) { + this.first = first; + this.second = second; } - public X first() { - return x; + X first() { + return first; } - public Y second() { - return y; + Y second() { + return second; } } } diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestNumericLiteralConversions.java b/api/src/test/java/com/netflix/iceberg/expressions/TestNumericLiteralConversions.java index 4bf37951c638..8a1ab9993243 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestNumericLiteralConversions.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestNumericLiteralConversions.java @@ -20,9 +20,9 @@ package com.netflix.iceberg.expressions; import com.netflix.iceberg.types.Types; +import java.math.BigDecimal; import org.junit.Assert; import org.junit.Test; -import java.math.BigDecimal; public class TestNumericLiteralConversions { @Test diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestPredicateBinding.java b/api/src/test/java/com/netflix/iceberg/expressions/TestPredicateBinding.java index 433d20e1aa8a..14f516fe61df 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestPredicateBinding.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestPredicateBinding.java @@ -19,15 +19,7 @@ package com.netflix.iceberg.expressions; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; -import org.junit.Assert; -import org.junit.Test; -import java.math.BigDecimal; -import java.util.Arrays; -import java.util.List; - +import static com.netflix.iceberg.TestHelpers.assertAndUnwrap; import static com.netflix.iceberg.expressions.Expression.Operation.EQ; import static com.netflix.iceberg.expressions.Expression.Operation.GT; import static com.netflix.iceberg.expressions.Expression.Operation.GT_EQ; @@ -37,12 +29,20 @@ import static com.netflix.iceberg.expressions.Expression.Operation.NOT_EQ; import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; import static com.netflix.iceberg.expressions.Expressions.ref; -import static com.netflix.iceberg.TestHelpers.assertAndUnwrap; import static com.netflix.iceberg.types.Types.NestedField.optional; import static com.netflix.iceberg.types.Types.NestedField.required; +import com.netflix.iceberg.exceptions.ValidationException; +import com.netflix.iceberg.types.Types; +import com.netflix.iceberg.types.Types.StructType; +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; +import org.junit.Assert; +import org.junit.Test; + public class TestPredicateBinding { - private static List COMPARISONS = Arrays.asList( + private static final List COMPARISONS = Arrays.asList( LT, LT_EQ, GT, GT_EQ, EQ, NOT_EQ); @Test diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestStrictMetricsEvaluator.java b/api/src/test/java/com/netflix/iceberg/expressions/TestStrictMetricsEvaluator.java index da785f8a0e7c..71ff7b7df6e1 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestStrictMetricsEvaluator.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestStrictMetricsEvaluator.java @@ -19,20 +19,6 @@ package com.netflix.iceberg.expressions; -import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.TestHelpers.Row; -import com.netflix.iceberg.TestHelpers.TestDataFile; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.StringType; -import org.junit.Assert; -import org.junit.Ignore; -import org.junit.Test; - import static com.netflix.iceberg.expressions.Expressions.and; import static com.netflix.iceberg.expressions.Expressions.equal; import static com.netflix.iceberg.expressions.Expressions.greaterThan; @@ -48,6 +34,18 @@ import static com.netflix.iceberg.types.Types.NestedField.optional; import static com.netflix.iceberg.types.Types.NestedField.required; +import com.google.common.collect.ImmutableMap; +import com.netflix.iceberg.DataFile; +import com.netflix.iceberg.Schema; +import com.netflix.iceberg.TestHelpers; +import com.netflix.iceberg.TestHelpers.Row; +import com.netflix.iceberg.TestHelpers.TestDataFile; +import com.netflix.iceberg.exceptions.ValidationException; +import com.netflix.iceberg.types.Types.IntegerType; +import com.netflix.iceberg.types.Types.StringType; +import org.junit.Assert; +import org.junit.Test; + public class TestStrictMetricsEvaluator { private static final Schema SCHEMA = new Schema( required(1, "id", IntegerType.get()), diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestStringLiteralConversions.java b/api/src/test/java/com/netflix/iceberg/expressions/TestStringLiteralConversions.java index bb1b7ce10960..9f2db3bcba01 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestStringLiteralConversions.java +++ b/api/src/test/java/com/netflix/iceberg/expressions/TestStringLiteralConversions.java @@ -20,6 +20,9 @@ package com.netflix.iceberg.expressions; import com.netflix.iceberg.types.Types; +import java.math.BigDecimal; +import java.time.DateTimeException; +import java.util.UUID; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.data.TimeConversions; @@ -29,9 +32,6 @@ import org.joda.time.LocalTime; import org.junit.Assert; import org.junit.Test; -import java.math.BigDecimal; -import java.time.DateTimeException; -import java.util.UUID; public class TestStringLiteralConversions { diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestBucketing.java b/api/src/test/java/com/netflix/iceberg/transforms/TestBucketing.java index 04db9e9060e3..4dfc33d324eb 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/com/netflix/iceberg/transforms/TestBucketing.java @@ -19,23 +19,23 @@ package com.netflix.iceberg.transforms; -import com.google.common.base.Charsets; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.netflix.iceberg.expressions.Literal; import com.netflix.iceberg.types.Types; -import org.apache.avro.util.Utf8; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; import java.util.Random; import java.util.UUID; +import org.apache.avro.util.Utf8; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; public class TestBucketing { private static final HashFunction MURMUR3 = Hashing.murmur3_32(); @@ -73,10 +73,10 @@ public void testSpecValues() { -142385009, new Bucket.BucketDouble(100).hash(1.0D)); Assert.assertEquals("Spec example: hash(decimal2(14.20)) = -500754589", -500754589, - Bucket.get(Types.DecimalType.of(9,2), 100).hash(new BigDecimal("14.20"))); + Bucket.get(Types.DecimalType.of(9, 2), 100).hash(new BigDecimal("14.20"))); Assert.assertEquals("Spec example: hash(decimal2(14.20)) = -500754589", -500754589, - Bucket.get(Types.DecimalType.of(9,2), 100).hash(new BigDecimal("14.20"))); + Bucket.get(Types.DecimalType.of(9, 2), 100).hash(new BigDecimal("14.20"))); Literal date = Literal.of("2017-11-16").to(Types.DateType.get()); Assert.assertEquals("Spec example: hash(2017-11-16) = -653330422", @@ -148,10 +148,10 @@ public void testIntegerTypePromotion() { Bucket bucketInts = Bucket.get(Types.IntegerType.get(), 100); Bucket bucketLongs = Bucket.get(Types.LongType.get(), 100); - int r = testRandom.nextInt(); + int randomBuckets = testRandom.nextInt(); Assert.assertEquals("Integer and Long bucket results should match", - bucketInts.apply(r), bucketLongs.apply((long) r)); + bucketInts.apply(randomBuckets), bucketLongs.apply((long) randomBuckets)); } @Test @@ -185,10 +185,10 @@ public void testFloatTypePromotion() { Bucket bucketFloats = new Bucket.BucketFloat(100); Bucket bucketDoubles = new Bucket.BucketDouble(100); - float r = testRandom.nextFloat(); + float randomFloat = testRandom.nextFloat(); Assert.assertEquals("Float and Double bucket results should match", - bucketFloats.apply(r), bucketDoubles.apply((double) r)); + bucketFloats.apply(randomFloat), bucketDoubles.apply((double) randomFloat)); } @Test @@ -206,7 +206,7 @@ public void testDecimal() { @Test public void testString() { String string = "string to test murmur3 hash"; - byte[] asBytes = string.getBytes(Charsets.UTF_8); + byte[] asBytes = string.getBytes(StandardCharsets.UTF_8); Bucket bucketFunc = Bucket.get(Types.StringType.get(), 100); @@ -217,7 +217,7 @@ public void testString() { @Test public void testUtf8() { Utf8 utf8 = new Utf8("string to test murmur3 hash"); - byte[] asBytes = utf8.toString().getBytes(Charsets.UTF_8); + byte[] asBytes = utf8.toString().getBytes(StandardCharsets.UTF_8); Bucket bucketFunc = Bucket.get(Types.StringType.get(), 100); diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestIdentity.java b/api/src/test/java/com/netflix/iceberg/transforms/TestIdentity.java index 93afd4bceabd..eb1ab96f012b 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestIdentity.java +++ b/api/src/test/java/com/netflix/iceberg/transforms/TestIdentity.java @@ -21,10 +21,10 @@ import com.netflix.iceberg.expressions.Literal; import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; import java.math.BigDecimal; import java.nio.ByteBuffer; +import org.junit.Assert; +import org.junit.Test; public class TestIdentity { @Test @@ -60,10 +60,10 @@ public void testDateHumanString() { Transform identity = Transforms.identity(date); String dateString = "2017-12-01"; - Literal d = Literal.of(dateString).to(date); + Literal dateLiteral = Literal.of(dateString).to(date); Assert.assertEquals("Should produce identical date", - dateString, identity.toHumanString(d.value())); + dateString, identity.toHumanString(dateLiteral.value())); } @Test @@ -72,10 +72,10 @@ public void testTimeHumanString() { Transform identity = Transforms.identity(time); String timeString = "10:12:55.038194"; - Literal d = Literal.of(timeString).to(time); + Literal timeLiteral = Literal.of(timeString).to(time); Assert.assertEquals("Should produce identical time", - timeString, identity.toHumanString(d.value())); + timeString, identity.toHumanString(timeLiteral.value())); } @Test @@ -127,7 +127,7 @@ public void testBigDecimalToHumanString() { Transform identity = Transforms.identity(decimal); String decimalString = "-1.50"; - BigDecimal d = new BigDecimal(decimalString); - Assert.assertEquals("Should not modify Strings", decimalString, identity.toHumanString(d)); + BigDecimal decimalLiteral = new BigDecimal(decimalString); + Assert.assertEquals("Should not modify Strings", decimalString, identity.toHumanString(decimalLiteral)); } } diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestProjection.java b/api/src/test/java/com/netflix/iceberg/transforms/TestProjection.java index 8d8a34d46a5c..e332ed6b1b12 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestProjection.java +++ b/api/src/test/java/com/netflix/iceberg/transforms/TestProjection.java @@ -19,7 +19,18 @@ package com.netflix.iceberg.transforms; +import static com.netflix.iceberg.TestHelpers.assertAndUnwrap; +import static com.netflix.iceberg.TestHelpers.assertAndUnwrapUnbound; +import static com.netflix.iceberg.expressions.Expressions.and; +import static com.netflix.iceberg.expressions.Expressions.equal; +import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; +import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; +import static com.netflix.iceberg.expressions.Expressions.or; +import static com.netflix.iceberg.types.Types.NestedField.optional; +import static com.netflix.iceberg.types.Types.NestedField.required; + import com.google.common.collect.Lists; +import com.netflix.iceberg.PartitionSpec; import com.netflix.iceberg.Schema; import com.netflix.iceberg.expressions.BoundPredicate; import com.netflix.iceberg.expressions.Expression; @@ -27,21 +38,10 @@ import com.netflix.iceberg.expressions.Or; import com.netflix.iceberg.expressions.Projections; import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.PartitionSpec; import com.netflix.iceberg.types.Types; +import java.util.List; import org.junit.Assert; import org.junit.Test; -import java.util.List; - -import static com.netflix.iceberg.TestHelpers.assertAndUnwrap; -import static com.netflix.iceberg.TestHelpers.assertAndUnwrapUnbound; -import static com.netflix.iceberg.expressions.Expressions.and; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.or; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; public class TestProjection { private static final Schema SCHEMA = new Schema( diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestResiduals.java b/api/src/test/java/com/netflix/iceberg/transforms/TestResiduals.java index 27b1c11604a3..c76553160e4a 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestResiduals.java +++ b/api/src/test/java/com/netflix/iceberg/transforms/TestResiduals.java @@ -19,16 +19,6 @@ package com.netflix.iceberg.transforms; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers.Row; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.ResidualEvaluator; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; - import static com.netflix.iceberg.TestHelpers.assertAndUnwrapUnbound; import static com.netflix.iceberg.expressions.Expression.Operation.GT; import static com.netflix.iceberg.expressions.Expression.Operation.LT; @@ -40,6 +30,16 @@ import static com.netflix.iceberg.expressions.Expressions.lessThan; import static com.netflix.iceberg.expressions.Expressions.or; +import com.netflix.iceberg.PartitionSpec; +import com.netflix.iceberg.Schema; +import com.netflix.iceberg.TestHelpers.Row; +import com.netflix.iceberg.expressions.Expression; +import com.netflix.iceberg.expressions.ResidualEvaluator; +import com.netflix.iceberg.expressions.UnboundPredicate; +import com.netflix.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; + public class TestResiduals { @Test public void testIdentityTransformResiduals() { diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestTimestamps.java b/api/src/test/java/com/netflix/iceberg/transforms/TestTimestamps.java index c05aff33b83f..652e8ed902a5 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestTimestamps.java +++ b/api/src/test/java/com/netflix/iceberg/transforms/TestTimestamps.java @@ -25,24 +25,25 @@ import org.junit.Test; public class TestTimestamps { + @Test public void testTimestampWithoutZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withoutZone(); - Literal date = Literal.of("2017-12-01T10:12:55.038194").to(type); + Literal date = Literal.of("2017-12-01T10:12:55.038194").to(type); - Transform year = Transforms.year(type); + Transform year = Transforms.year(type); Assert.assertEquals("Should produce the correct Human string", "2017", year.toHumanString(year.apply(date.value()))); - Transform month = Transforms.month(type); + Transform month = Transforms.month(type); Assert.assertEquals("Should produce the correct Human string", "2017-12", month.toHumanString(month.apply(date.value()))); - Transform day = Transforms.day(type); + Transform day = Transforms.day(type); Assert.assertEquals("Should produce the correct Human string", "2017-12-01", day.toHumanString(day.apply(date.value()))); - Transform hour = Transforms.hour(type); + Transform hour = Transforms.hour(type); Assert.assertEquals("Should produce the correct Human string", "2017-12-01-10", hour.toHumanString(hour.apply(date.value()))); } @@ -50,22 +51,22 @@ public void testTimestampWithoutZoneToHumanString() { @Test public void testTimestampWithZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withZone(); - Literal date = Literal.of("2017-12-01T10:12:55.038194-08:00").to(type); + Literal date = Literal.of("2017-12-01T10:12:55.038194-08:00").to(type); - Transform year = Transforms.year(type); + Transform year = Transforms.year(type); Assert.assertEquals("Should produce the correct Human string", "2017", year.toHumanString(year.apply(date.value()))); - Transform month = Transforms.month(type); + Transform month = Transforms.month(type); Assert.assertEquals("Should produce the correct Human string", "2017-12", month.toHumanString(month.apply(date.value()))); - Transform day = Transforms.day(type); + Transform day = Transforms.day(type); Assert.assertEquals("Should produce the correct Human string", "2017-12-01", day.toHumanString(day.apply(date.value()))); // the hour is 18 because the value is always UTC - Transform hour = Transforms.hour(type); + Transform hour = Transforms.hour(type); Assert.assertEquals("Should produce the correct Human string", "2017-12-01-18", hour.toHumanString(hour.apply(date.value()))); } diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestTruncate.java b/api/src/test/java/com/netflix/iceberg/transforms/TestTruncate.java index d42d43e15633..a52ae30e71f1 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestTruncate.java +++ b/api/src/test/java/com/netflix/iceberg/transforms/TestTruncate.java @@ -20,10 +20,10 @@ package com.netflix.iceberg.transforms; import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; import java.math.BigDecimal; import java.nio.ByteBuffer; +import org.junit.Assert; +import org.junit.Test; public class TestTruncate { @Test diff --git a/api/src/test/java/com/netflix/iceberg/types/TestBinaryComparator.java b/api/src/test/java/com/netflix/iceberg/types/TestBinaryComparator.java index 9efa2d0536be..4c39e26ef6c1 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestBinaryComparator.java +++ b/api/src/test/java/com/netflix/iceberg/types/TestBinaryComparator.java @@ -20,10 +20,10 @@ package com.netflix.iceberg.types; import com.netflix.iceberg.expressions.Literal; -import org.junit.Assert; -import org.junit.Test; import java.nio.ByteBuffer; import java.util.Comparator; +import org.junit.Assert; +import org.junit.Test; /** * Tests the comparator returned by binary and fixed literals. diff --git a/api/src/test/java/com/netflix/iceberg/types/TestCharSeqComparator.java b/api/src/test/java/com/netflix/iceberg/types/TestCharSeqComparator.java index 4e3b4a3565cb..bec5c3d31bae 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestCharSeqComparator.java +++ b/api/src/test/java/com/netflix/iceberg/types/TestCharSeqComparator.java @@ -20,10 +20,10 @@ package com.netflix.iceberg.types; import com.netflix.iceberg.expressions.Literal; +import java.util.Comparator; import org.apache.avro.util.Utf8; import org.junit.Assert; import org.junit.Test; -import java.util.Comparator; /** * Tests the comparator returned by CharSequence literals. diff --git a/api/src/test/java/com/netflix/iceberg/types/TestComparableComparator.java b/api/src/test/java/com/netflix/iceberg/types/TestComparableComparator.java index 1f3f6e39995a..633a39d1a8ec 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestComparableComparator.java +++ b/api/src/test/java/com/netflix/iceberg/types/TestComparableComparator.java @@ -20,9 +20,9 @@ package com.netflix.iceberg.types; import com.netflix.iceberg.expressions.Literal; +import java.util.Comparator; import org.junit.Assert; import org.junit.Test; -import java.util.Comparator; /** * This tests the Comparator returned by ComparableLiteral, which is used for most types. diff --git a/api/src/test/java/com/netflix/iceberg/types/TestReadabilityChecks.java b/api/src/test/java/com/netflix/iceberg/types/TestReadabilityChecks.java index 6b743386a6a3..5cf0cbabd544 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestReadabilityChecks.java +++ b/api/src/test/java/com/netflix/iceberg/types/TestReadabilityChecks.java @@ -19,13 +19,13 @@ package com.netflix.iceberg.types; +import static com.netflix.iceberg.types.Types.NestedField.optional; +import static com.netflix.iceberg.types.Types.NestedField.required; + import com.netflix.iceberg.Schema; +import java.util.List; import org.junit.Assert; import org.junit.Test; -import java.util.List; - -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; public class TestReadabilityChecks { private static final Type.PrimitiveType[] PRIMITIVES = new Type.PrimitiveType[] { @@ -66,49 +66,41 @@ public void testPrimitiveTypes() { } } - { - Schema structSchema = new Schema(required(1, "struct_field", Types.StructType.of( - required(2, "from", from)) - )); + Schema structSchema = new Schema(required(1, "struct_field", Types.StructType.of( + required(2, "from", from)) + )); - List errors = CheckCompatibility.writeCompatibilityErrors(structSchema, fromSchema); - Assert.assertEquals("Should produce 1 error message", 1, errors.size()); + List errors = CheckCompatibility.writeCompatibilityErrors(structSchema, fromSchema); + Assert.assertEquals("Should produce 1 error message", 1, errors.size()); - Assert.assertTrue("Should complain that primitive to struct is not allowed", - errors.get(0).contains("cannot be read as a struct")); - } + Assert.assertTrue("Should complain that primitive to struct is not allowed", + errors.get(0).contains("cannot be read as a struct")); - { - Schema listSchema = new Schema(required(1, "list_field", Types.ListType.ofRequired(2, from))); + Schema listSchema = new Schema(required(1, "list_field", Types.ListType.ofRequired(2, from))); - List errors = CheckCompatibility.writeCompatibilityErrors(listSchema, fromSchema); - Assert.assertEquals("Should produce 1 error message", 1, errors.size()); + errors = CheckCompatibility.writeCompatibilityErrors(listSchema, fromSchema); + Assert.assertEquals("Should produce 1 error message", 1, errors.size()); - Assert.assertTrue("Should complain that primitive to list is not allowed", - errors.get(0).contains("cannot be read as a list")); - } + Assert.assertTrue("Should complain that primitive to list is not allowed", + errors.get(0).contains("cannot be read as a list")); - { - Schema mapSchema = new Schema(required(1, "map_field", - Types.MapType.ofRequired(2, 3, Types.StringType.get(), from))); + Schema mapSchema = new Schema(required(1, "map_field", + Types.MapType.ofRequired(2, 3, Types.StringType.get(), from))); - List errors = CheckCompatibility.writeCompatibilityErrors(mapSchema, fromSchema); - Assert.assertEquals("Should produce 1 error message", 1, errors.size()); + errors = CheckCompatibility.writeCompatibilityErrors(mapSchema, fromSchema); + Assert.assertEquals("Should produce 1 error message", 1, errors.size()); - Assert.assertTrue("Should complain that primitive to map is not allowed", - errors.get(0).contains("cannot be read as a map")); - } + Assert.assertTrue("Should complain that primitive to map is not allowed", + errors.get(0).contains("cannot be read as a map")); - { - Schema mapSchema = new Schema(required(1, "map_field", - Types.MapType.ofRequired(2, 3, from, Types.StringType.get()))); + mapSchema = new Schema(required(1, "map_field", + Types.MapType.ofRequired(2, 3, from, Types.StringType.get()))); - List errors = CheckCompatibility.writeCompatibilityErrors(mapSchema, fromSchema); - Assert.assertEquals("Should produce 1 error message", 1, errors.size()); + errors = CheckCompatibility.writeCompatibilityErrors(mapSchema, fromSchema); + Assert.assertEquals("Should produce 1 error message", 1, errors.size()); - Assert.assertTrue("Should complain that primitive to map is not allowed", - errors.get(0).contains("cannot be read as a map")); - } + Assert.assertTrue("Should complain that primitive to map is not allowed", + errors.get(0).contains("cannot be read as a map")); } } diff --git a/api/src/test/java/com/netflix/iceberg/types/TestSerializableTypes.java b/api/src/test/java/com/netflix/iceberg/types/TestSerializableTypes.java index 757bd72b728b..fe2eebdb33fb 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/com/netflix/iceberg/types/TestSerializableTypes.java @@ -19,14 +19,14 @@ package com.netflix.iceberg.types; +import static com.netflix.iceberg.types.Types.NestedField.optional; +import static com.netflix.iceberg.types.Types.NestedField.required; + import com.netflix.iceberg.Schema; import com.netflix.iceberg.TestHelpers; import org.junit.Assert; import org.junit.Test; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; - public class TestSerializableTypes { @Test public void testIdentityTypes() throws Exception { diff --git a/build.gradle b/build.gradle index d41eb137de8c..597605b453b3 100644 --- a/build.gradle +++ b/build.gradle @@ -18,11 +18,16 @@ */ buildscript { - repositories { jcenter() } + repositories { + jcenter() + maven { url "http://palantir.bintray.com/releases" } + } dependencies { classpath 'com.github.jengelman.gradle.plugins:shadow:2.0.0' classpath 'com.netflix.nebula:gradle-aggregate-javadocs-plugin:2.2.+' classpath 'com.netflix.nebula:nebula-publishing-plugin:5.1.5' + classpath 'com.palantir.baseline:gradle-baseline-java:0.42.0' + classpath 'gradle.plugin.org.inferred:gradle-processors:2.1.0' } } @@ -37,11 +42,20 @@ if (JavaVersion.current() != JavaVersion.VERSION_1_8) { allprojects { group = "com.netflix.iceberg" - apply plugin: 'idea' version = gitVersion() + apply plugin: 'com.palantir.baseline-idea' } apply plugin: 'nebula-aggregate-javadocs' +apply plugin: 'org.inferred.processors' + +allprojects { + repositories { + mavenCentral() + mavenLocal() + maven { url "http://palantir.bintray.com/releases" } + } +} subprojects { apply plugin: 'nebula.javadoc-jar' @@ -50,11 +64,6 @@ subprojects { apply plugin: 'maven' // make pom files for deployment apply plugin: 'nebula.maven-base-publish' - repositories { - mavenCentral() - mavenLocal() - } - configurations { testCompile.extendsFrom compileOnly all { @@ -99,6 +108,7 @@ subprojects { } project(':iceberg-api') { + apply plugin: 'com.palantir.baseline-checkstyle' dependencies { testCompile "org.apache.avro:avro:$avroVersion" testCompile 'joda-time:joda-time:2.9.9' diff --git a/core/src/main/java/com/netflix/iceberg/avro/TypeToSchema.java b/core/src/main/java/com/netflix/iceberg/avro/TypeToSchema.java index 085caa8479d1..9747a615ed6e 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/TypeToSchema.java +++ b/core/src/main/java/com/netflix/iceberg/avro/TypeToSchema.java @@ -81,7 +81,7 @@ public Schema struct(Types.StructType struct, List fieldSchemas) { String recordName = names.get(struct); if (recordName == null) { - recordName = "r" + fieldIds.peek(); + recordName = "r" + fieldIds().peek(); } List structFields = struct.fields(); diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 976f7d27fb1f..dee7f0485c17 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -21,4 +21,4 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-4.4-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-5.0-bin.zip