diff --git a/api/src/main/java/com/netflix/iceberg/AppendFiles.java b/api/src/main/java/org/apache/iceberg/AppendFiles.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/AppendFiles.java rename to api/src/main/java/org/apache/iceberg/AppendFiles.java index d9734325f54f..2a9192a5fe31 100644 --- a/api/src/main/java/com/netflix/iceberg/AppendFiles.java +++ b/api/src/main/java/org/apache/iceberg/AppendFiles.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; /** * API for appending new files in a table. diff --git a/api/src/main/java/com/netflix/iceberg/CombinedScanTask.java b/api/src/main/java/org/apache/iceberg/CombinedScanTask.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/CombinedScanTask.java rename to api/src/main/java/org/apache/iceberg/CombinedScanTask.java index 045f89c1fdda..87a5495ba7da 100644 --- a/api/src/main/java/com/netflix/iceberg/CombinedScanTask.java +++ b/api/src/main/java/org/apache/iceberg/CombinedScanTask.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import java.util.Collection; diff --git a/api/src/main/java/com/netflix/iceberg/DataFile.java b/api/src/main/java/org/apache/iceberg/DataFile.java similarity index 87% rename from api/src/main/java/com/netflix/iceberg/DataFile.java rename to api/src/main/java/org/apache/iceberg/DataFile.java index 7e432c9bdbb6..1a86e1290d20 100644 --- a/api/src/main/java/com/netflix/iceberg/DataFile.java +++ b/api/src/main/java/org/apache/iceberg/DataFile.java @@ -17,22 +17,21 @@ * under the License. */ -package com.netflix.iceberg; - -import com.netflix.iceberg.encryption.EncryptionKeyMetadata; -import com.netflix.iceberg.types.Types.BinaryType; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.ListType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.MapType; -import com.netflix.iceberg.types.Types.StringType; -import com.netflix.iceberg.types.Types.StructType; +package org.apache.iceberg; + import java.nio.ByteBuffer; 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; +import org.apache.iceberg.types.Types.BinaryType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; /** * Interface for files listed in a table manifest. diff --git a/api/src/main/java/com/netflix/iceberg/DataOperations.java b/api/src/main/java/org/apache/iceberg/DataOperations.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/DataOperations.java rename to api/src/main/java/org/apache/iceberg/DataOperations.java index c273e66d2a8c..143f16a7b3cb 100644 --- a/api/src/main/java/com/netflix/iceberg/DataOperations.java +++ b/api/src/main/java/org/apache/iceberg/DataOperations.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; /** * Data operations that produce snapshots. diff --git a/api/src/main/java/com/netflix/iceberg/DeleteFiles.java b/api/src/main/java/org/apache/iceberg/DeleteFiles.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/DeleteFiles.java rename to api/src/main/java/org/apache/iceberg/DeleteFiles.java index 69f8f756d9d1..e93dfb47a445 100644 --- a/api/src/main/java/com/netflix/iceberg/DeleteFiles.java +++ b/api/src/main/java/org/apache/iceberg/DeleteFiles.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Projections; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Projections; /** * API for deleting files from a table. diff --git a/api/src/main/java/com/netflix/iceberg/ExpireSnapshots.java b/api/src/main/java/org/apache/iceberg/ExpireSnapshots.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/ExpireSnapshots.java rename to api/src/main/java/org/apache/iceberg/ExpireSnapshots.java index 64659fa82a95..fb53cd62b8ec 100644 --- a/api/src/main/java/com/netflix/iceberg/ExpireSnapshots.java +++ b/api/src/main/java/org/apache/iceberg/ExpireSnapshots.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import java.util.List; import java.util.function.Consumer; diff --git a/api/src/main/java/com/netflix/iceberg/FileFormat.java b/api/src/main/java/org/apache/iceberg/FileFormat.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/FileFormat.java rename to api/src/main/java/org/apache/iceberg/FileFormat.java index 274a867d4e0d..2f17abca8e8c 100644 --- a/api/src/main/java/com/netflix/iceberg/FileFormat.java +++ b/api/src/main/java/org/apache/iceberg/FileFormat.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.types.Comparators; +import org.apache.iceberg.types.Comparators; /** * Enum of supported file formats. diff --git a/api/src/main/java/com/netflix/iceberg/FileScanTask.java b/api/src/main/java/org/apache/iceberg/FileScanTask.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/FileScanTask.java rename to api/src/main/java/org/apache/iceberg/FileScanTask.java index 6d124d1fbd48..18a2002be485 100644 --- a/api/src/main/java/com/netflix/iceberg/FileScanTask.java +++ b/api/src/main/java/org/apache/iceberg/FileScanTask.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression; /** * A scan task over a range of a single file. diff --git a/api/src/main/java/com/netflix/iceberg/Files.java b/api/src/main/java/org/apache/iceberg/Files.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/Files.java rename to api/src/main/java/org/apache/iceberg/Files.java index 0a2805f1a2a2..0d9aeb1e741b 100644 --- a/api/src/main/java/com/netflix/iceberg/Files.java +++ b/api/src/main/java/org/apache/iceberg/Files.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg; - -import com.netflix.iceberg.exceptions.AlreadyExistsException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.io.PositionOutputStream; -import com.netflix.iceberg.io.SeekableInputStream; +package org.apache.iceberg; + import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.io.RandomAccessFile; import java.nio.file.Paths; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; +import org.apache.iceberg.io.SeekableInputStream; public class Files { diff --git a/api/src/main/java/com/netflix/iceberg/Filterable.java b/api/src/main/java/org/apache/iceberg/Filterable.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/Filterable.java rename to api/src/main/java/org/apache/iceberg/Filterable.java index ac7df63270ef..4eb8a50a362b 100644 --- a/api/src/main/java/com/netflix/iceberg/Filterable.java +++ b/api/src/main/java/org/apache/iceberg/Filterable.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; -import com.netflix.iceberg.expressions.Expression; import java.util.Collection; +import org.apache.iceberg.expressions.Expression; /** * Methods to filter files in a snapshot or manifest when reading. diff --git a/api/src/main/java/com/netflix/iceberg/ManifestFile.java b/api/src/main/java/org/apache/iceberg/ManifestFile.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/ManifestFile.java rename to api/src/main/java/org/apache/iceberg/ManifestFile.java index b1d919b08d06..26180375b487 100644 --- a/api/src/main/java/com/netflix/iceberg/ManifestFile.java +++ b/api/src/main/java/org/apache/iceberg/ManifestFile.java @@ -17,15 +17,14 @@ * under the License. */ -package com.netflix.iceberg; - -import com.netflix.iceberg.types.Types; +package org.apache.iceberg; import java.nio.ByteBuffer; import java.util.List; +import org.apache.iceberg.types.Types; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; /** * Represents a manifest file that can be scanned to find data files in a table. diff --git a/api/src/main/java/com/netflix/iceberg/Metrics.java b/api/src/main/java/org/apache/iceberg/Metrics.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/Metrics.java rename to api/src/main/java/org/apache/iceberg/Metrics.java index 7ea9d33bc69d..765c6b919f05 100644 --- a/api/src/main/java/com/netflix/iceberg/Metrics.java +++ b/api/src/main/java/org/apache/iceberg/Metrics.java @@ -17,14 +17,12 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import java.io.Serializable; 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/OverwriteFiles.java b/api/src/main/java/org/apache/iceberg/OverwriteFiles.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/OverwriteFiles.java rename to api/src/main/java/org/apache/iceberg/OverwriteFiles.java index 46ae011e8243..8d8e38c7a69f 100644 --- a/api/src/main/java/com/netflix/iceberg/OverwriteFiles.java +++ b/api/src/main/java/org/apache/iceberg/OverwriteFiles.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Projections; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Projections; /** * API for overwriting files in a table by filter expression. diff --git a/api/src/main/java/com/netflix/iceberg/PartitionField.java b/api/src/main/java/org/apache/iceberg/PartitionField.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/PartitionField.java rename to api/src/main/java/org/apache/iceberg/PartitionField.java index 331f6dae65af..4f60dcc223c2 100644 --- a/api/src/main/java/com/netflix/iceberg/PartitionField.java +++ b/api/src/main/java/org/apache/iceberg/PartitionField.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; -import com.netflix.iceberg.transforms.Transform; import java.io.Serializable; +import org.apache.iceberg.transforms.Transform; /** * Represents a single field in a {@link PartitionSpec}. diff --git a/api/src/main/java/com/netflix/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/PartitionSpec.java rename to api/src/main/java/org/apache/iceberg/PartitionSpec.java index b17e25b6edc4..63f7a87ef968 100644 --- a/api/src/main/java/com/netflix/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -17,17 +17,13 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.transforms.Transforms; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.io.Serializable; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; @@ -36,6 +32,10 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; /** * Represents how to produce partition data for a table. diff --git a/api/src/main/java/com/netflix/iceberg/PendingUpdate.java b/api/src/main/java/org/apache/iceberg/PendingUpdate.java similarity index 92% rename from api/src/main/java/com/netflix/iceberg/PendingUpdate.java rename to api/src/main/java/org/apache/iceberg/PendingUpdate.java index 1bc0d2962b64..dfd43120c7f9 100644 --- a/api/src/main/java/com/netflix/iceberg/PendingUpdate.java +++ b/api/src/main/java/org/apache/iceberg/PendingUpdate.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.ValidationException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; /** * API for table metadata changes. diff --git a/api/src/main/java/com/netflix/iceberg/ReplacePartitions.java b/api/src/main/java/org/apache/iceberg/ReplacePartitions.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/ReplacePartitions.java rename to api/src/main/java/org/apache/iceberg/ReplacePartitions.java index b66581735121..07108e36adf4 100644 --- a/api/src/main/java/com/netflix/iceberg/ReplacePartitions.java +++ b/api/src/main/java/org/apache/iceberg/ReplacePartitions.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; /** * Not recommended: API for overwriting files in a table by partition. diff --git a/api/src/main/java/com/netflix/iceberg/RewriteFiles.java b/api/src/main/java/org/apache/iceberg/RewriteFiles.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/RewriteFiles.java rename to api/src/main/java/org/apache/iceberg/RewriteFiles.java index 1a8297298b7c..51057bf07e14 100644 --- a/api/src/main/java/com/netflix/iceberg/RewriteFiles.java +++ b/api/src/main/java/org/apache/iceberg/RewriteFiles.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.ValidationException; import java.util.Set; +import org.apache.iceberg.exceptions.ValidationException; /** * API for replacing files in a table. diff --git a/api/src/main/java/com/netflix/iceberg/Rollback.java b/api/src/main/java/org/apache/iceberg/Rollback.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/Rollback.java rename to api/src/main/java/org/apache/iceberg/Rollback.java index ac4e7779e014..0992969ce13b 100644 --- a/api/src/main/java/com/netflix/iceberg/Rollback.java +++ b/api/src/main/java/org/apache/iceberg/Rollback.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitFailedException; /** * API for rolling table data back to the state at an older table {@link Snapshot snapshot}. diff --git a/api/src/main/java/com/netflix/iceberg/ScanTask.java b/api/src/main/java/org/apache/iceberg/ScanTask.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/ScanTask.java rename to api/src/main/java/org/apache/iceberg/ScanTask.java index a15fc18114b3..7c7350eabed6 100644 --- a/api/src/main/java/com/netflix/iceberg/ScanTask.java +++ b/api/src/main/java/org/apache/iceberg/ScanTask.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import java.io.Serializable; diff --git a/api/src/main/java/com/netflix/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/Schema.java rename to api/src/main/java/org/apache/iceberg/Schema.java index ee21439d038a..191572817425 100644 --- a/api/src/main/java/com/netflix/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -17,16 +17,13 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.BiMap; import com.google.common.collect.ImmutableBiMap; import com.google.common.collect.Sets; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; import java.io.Serializable; import java.util.Arrays; import java.util.Collection; @@ -35,6 +32,9 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; /** * The schema of a data table. diff --git a/api/src/main/java/com/netflix/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java similarity index 99% rename from api/src/main/java/com/netflix/iceberg/Snapshot.java rename to api/src/main/java/org/apache/iceberg/Snapshot.java index c2071d6133a5..25822c967852 100644 --- a/api/src/main/java/com/netflix/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import java.util.List; import java.util.Map; diff --git a/api/src/main/java/com/netflix/iceberg/StructLike.java b/api/src/main/java/org/apache/iceberg/StructLike.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/StructLike.java rename to api/src/main/java/org/apache/iceberg/StructLike.java index d8d178626c58..0d3ddb40c788 100644 --- a/api/src/main/java/com/netflix/iceberg/StructLike.java +++ b/api/src/main/java/org/apache/iceberg/StructLike.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; /** * Interface for accessing data by position in a schema. diff --git a/api/src/main/java/com/netflix/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/Table.java rename to api/src/main/java/org/apache/iceberg/Table.java index e0ce18958187..d0b5821f78ce 100644 --- a/api/src/main/java/com/netflix/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.encryption.EncryptionManager; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.io.LocationProvider; import java.util.Map; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; /** * Represents a table. @@ -188,7 +188,7 @@ default AppendFiles newFastAppend() { FileIO io(); /** - * @return an {@link com.netflix.iceberg.encryption.EncryptionManager} to encrypt and decrypt + * @return an {@link org.apache.iceberg.encryption.EncryptionManager} to encrypt and decrypt * data files. */ EncryptionManager encryption(); diff --git a/api/src/main/java/com/netflix/iceberg/TableScan.java b/api/src/main/java/org/apache/iceberg/TableScan.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/TableScan.java rename to api/src/main/java/org/apache/iceberg/TableScan.java index eaab97858ad7..4f189aab9137 100644 --- a/api/src/main/java/com/netflix/iceberg/TableScan.java +++ b/api/src/main/java/org/apache/iceberg/TableScan.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.io.CloseableIterable; import java.util.Collection; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; /** * API for configuring a table scan. diff --git a/api/src/main/java/com/netflix/iceberg/Tables.java b/api/src/main/java/org/apache/iceberg/Tables.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/Tables.java rename to api/src/main/java/org/apache/iceberg/Tables.java index 3308c650246b..4e23a08f6016 100644 --- a/api/src/main/java/com/netflix/iceberg/Tables.java +++ b/api/src/main/java/org/apache/iceberg/Tables.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableMap; import java.util.Map; diff --git a/api/src/main/java/com/netflix/iceberg/Transaction.java b/api/src/main/java/org/apache/iceberg/Transaction.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/Transaction.java rename to api/src/main/java/org/apache/iceberg/Transaction.java index 22d7e5a62154..d2e6320e20ed 100644 --- a/api/src/main/java/com/netflix/iceberg/Transaction.java +++ b/api/src/main/java/org/apache/iceberg/Transaction.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.ValidationException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; /** * A transaction for performing multiple updates to a table. diff --git a/api/src/main/java/com/netflix/iceberg/UpdateLocation.java b/api/src/main/java/org/apache/iceberg/UpdateLocation.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/UpdateLocation.java rename to api/src/main/java/org/apache/iceberg/UpdateLocation.java index e5085313df94..4513749b2cf0 100644 --- a/api/src/main/java/com/netflix/iceberg/UpdateLocation.java +++ b/api/src/main/java/org/apache/iceberg/UpdateLocation.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; /** * API for setting a table's base location. diff --git a/api/src/main/java/com/netflix/iceberg/UpdateProperties.java b/api/src/main/java/org/apache/iceberg/UpdateProperties.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/UpdateProperties.java rename to api/src/main/java/org/apache/iceberg/UpdateProperties.java index bca807fa1746..28c0675078ea 100644 --- a/api/src/main/java/com/netflix/iceberg/UpdateProperties.java +++ b/api/src/main/java/org/apache/iceberg/UpdateProperties.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import java.util.Map; diff --git a/api/src/main/java/com/netflix/iceberg/UpdateSchema.java b/api/src/main/java/org/apache/iceberg/UpdateSchema.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/UpdateSchema.java rename to api/src/main/java/org/apache/iceberg/UpdateSchema.java index 70aac5ed4cee..8d9b17943d0a 100644 --- a/api/src/main/java/com/netflix/iceberg/UpdateSchema.java +++ b/api/src/main/java/org/apache/iceberg/UpdateSchema.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.types.Type; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.types.Type; /** * API for schema evolution. diff --git a/api/src/main/java/com/netflix/iceberg/encryption/EncryptedInputFile.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptedInputFile.java similarity index 93% rename from api/src/main/java/com/netflix/iceberg/encryption/EncryptedInputFile.java rename to api/src/main/java/org/apache/iceberg/encryption/EncryptedInputFile.java index eff540c7c919..e990d1f5bf3a 100644 --- a/api/src/main/java/com/netflix/iceberg/encryption/EncryptedInputFile.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptedInputFile.java @@ -17,11 +17,9 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; -import com.netflix.iceberg.io.InputFile; - -import java.nio.ByteBuffer; +import org.apache.iceberg.io.InputFile; /** * Thin wrapper around an {@link InputFile} instance that is encrypted. diff --git a/api/src/main/java/com/netflix/iceberg/encryption/EncryptedOutputFile.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java similarity index 93% rename from api/src/main/java/com/netflix/iceberg/encryption/EncryptedOutputFile.java rename to api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java index 53dd863450a6..d05033ebe150 100644 --- a/api/src/main/java/com/netflix/iceberg/encryption/EncryptedOutputFile.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java @@ -17,11 +17,9 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; -import com.netflix.iceberg.io.OutputFile; - -import java.nio.ByteBuffer; +import org.apache.iceberg.io.OutputFile; /** * Thin wrapper around a {@link OutputFile} that is encrypting bytes written to the underlying diff --git a/api/src/main/java/com/netflix/iceberg/encryption/EncryptionKeyMetadata.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptionKeyMetadata.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/encryption/EncryptionKeyMetadata.java rename to api/src/main/java/org/apache/iceberg/encryption/EncryptionKeyMetadata.java index 7d3c04f8bde7..02ff8e212085 100644 --- a/api/src/main/java/com/netflix/iceberg/encryption/EncryptionKeyMetadata.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptionKeyMetadata.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; import java.nio.ByteBuffer; diff --git a/api/src/main/java/com/netflix/iceberg/encryption/EncryptionManager.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptionManager.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/encryption/EncryptionManager.java rename to api/src/main/java/org/apache/iceberg/encryption/EncryptionManager.java index 655598626d7e..ad712fcd45a7 100644 --- a/api/src/main/java/com/netflix/iceberg/encryption/EncryptionManager.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptionManager.java @@ -17,13 +17,12 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; import com.google.common.collect.Iterables; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; - import java.io.Serializable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; /** * Module for encrypting and decrypting table data files. diff --git a/api/src/main/java/com/netflix/iceberg/events/Listener.java b/api/src/main/java/org/apache/iceberg/events/Listener.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/events/Listener.java rename to api/src/main/java/org/apache/iceberg/events/Listener.java index c61a4218d430..843a1de91402 100644 --- a/api/src/main/java/com/netflix/iceberg/events/Listener.java +++ b/api/src/main/java/org/apache/iceberg/events/Listener.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.events; +package org.apache.iceberg.events; /** * A listener interface that can receive notifications. diff --git a/api/src/main/java/com/netflix/iceberg/events/Listeners.java b/api/src/main/java/org/apache/iceberg/events/Listeners.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/events/Listeners.java rename to api/src/main/java/org/apache/iceberg/events/Listeners.java index 4dc043bb5fb6..3b279c0f1a04 100644 --- a/api/src/main/java/com/netflix/iceberg/events/Listeners.java +++ b/api/src/main/java/org/apache/iceberg/events/Listeners.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.events; +package org.apache.iceberg.events; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; diff --git a/api/src/main/java/com/netflix/iceberg/events/ScanEvent.java b/api/src/main/java/org/apache/iceberg/events/ScanEvent.java similarity index 92% rename from api/src/main/java/com/netflix/iceberg/events/ScanEvent.java rename to api/src/main/java/org/apache/iceberg/events/ScanEvent.java index c1964df0bdf8..5c07de849bd7 100644 --- a/api/src/main/java/com/netflix/iceberg/events/ScanEvent.java +++ b/api/src/main/java/org/apache/iceberg/events/ScanEvent.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.events; +package org.apache.iceberg.events; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.Expression; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; /** * Event sent to listeners when a table scan is planned. diff --git a/api/src/main/java/com/netflix/iceberg/exceptions/AlreadyExistsException.java b/api/src/main/java/org/apache/iceberg/exceptions/AlreadyExistsException.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/exceptions/AlreadyExistsException.java rename to api/src/main/java/org/apache/iceberg/exceptions/AlreadyExistsException.java index d3f13c7faa32..9d73b93308c1 100644 --- a/api/src/main/java/com/netflix/iceberg/exceptions/AlreadyExistsException.java +++ b/api/src/main/java/org/apache/iceberg/exceptions/AlreadyExistsException.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.exceptions; +package org.apache.iceberg.exceptions; /** * Exception raised when attempting to create a table that already exists. diff --git a/api/src/main/java/com/netflix/iceberg/exceptions/CommitFailedException.java b/api/src/main/java/org/apache/iceberg/exceptions/CommitFailedException.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/exceptions/CommitFailedException.java rename to api/src/main/java/org/apache/iceberg/exceptions/CommitFailedException.java index e832f34d7874..c1485e9e1867 100644 --- a/api/src/main/java/com/netflix/iceberg/exceptions/CommitFailedException.java +++ b/api/src/main/java/org/apache/iceberg/exceptions/CommitFailedException.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.exceptions; +package org.apache.iceberg.exceptions; /** * Exception raised when a commit fails because of out of date metadata. diff --git a/api/src/main/java/com/netflix/iceberg/exceptions/NoSuchTableException.java b/api/src/main/java/org/apache/iceberg/exceptions/NoSuchTableException.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/exceptions/NoSuchTableException.java rename to api/src/main/java/org/apache/iceberg/exceptions/NoSuchTableException.java index 97580b9087e4..1fbb4d255e90 100644 --- a/api/src/main/java/com/netflix/iceberg/exceptions/NoSuchTableException.java +++ b/api/src/main/java/org/apache/iceberg/exceptions/NoSuchTableException.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.exceptions; +package org.apache.iceberg.exceptions; /** * Exception raised when attempting to load a table that does not exist. diff --git a/api/src/main/java/com/netflix/iceberg/exceptions/RuntimeIOException.java b/api/src/main/java/org/apache/iceberg/exceptions/RuntimeIOException.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/exceptions/RuntimeIOException.java rename to api/src/main/java/org/apache/iceberg/exceptions/RuntimeIOException.java index 837b018d1cd4..62fc8f18be05 100644 --- a/api/src/main/java/com/netflix/iceberg/exceptions/RuntimeIOException.java +++ b/api/src/main/java/org/apache/iceberg/exceptions/RuntimeIOException.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.exceptions; +package org.apache.iceberg.exceptions; import java.io.IOException; diff --git a/api/src/main/java/com/netflix/iceberg/exceptions/ValidationException.java b/api/src/main/java/org/apache/iceberg/exceptions/ValidationException.java similarity index 92% rename from api/src/main/java/com/netflix/iceberg/exceptions/ValidationException.java rename to api/src/main/java/org/apache/iceberg/exceptions/ValidationException.java index 7dfcffcfcb5b..83c485d541b2 100644 --- a/api/src/main/java/com/netflix/iceberg/exceptions/ValidationException.java +++ b/api/src/main/java/org/apache/iceberg/exceptions/ValidationException.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.exceptions; +package org.apache.iceberg.exceptions; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; /** * Exception raised when validation checks fail. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/And.java b/api/src/main/java/org/apache/iceberg/expressions/And.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/And.java rename to api/src/main/java/org/apache/iceberg/expressions/And.java index ebbac657ed2d..945a1c253640 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/And.java +++ b/api/src/main/java/org/apache/iceberg/expressions/And.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; public class And implements Expression { private final Expression left; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Binder.java b/api/src/main/java/org/apache/iceberg/expressions/Binder.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/expressions/Binder.java rename to api/src/main/java/org/apache/iceberg/expressions/Binder.java index 9fce899cba7d..1bc6563b0bc4 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Binder.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Binder.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types.StructType; import java.util.List; import java.util.Set; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types.StructType; /** * Rewrites {@link Expression expressions} by replacing unbound named references with references to diff --git a/api/src/main/java/com/netflix/iceberg/expressions/BoundPredicate.java b/api/src/main/java/org/apache/iceberg/expressions/BoundPredicate.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/expressions/BoundPredicate.java rename to api/src/main/java/org/apache/iceberg/expressions/BoundPredicate.java index 96f53b49502f..bf65eda3511e 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/BoundPredicate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundPredicate.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; public class BoundPredicate extends Predicate> { BoundPredicate(Operation op, BoundReference ref, Literal lit) { diff --git a/api/src/main/java/com/netflix/iceberg/expressions/BoundReference.java b/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java similarity index 90% rename from api/src/main/java/com/netflix/iceberg/expressions/BoundReference.java rename to api/src/main/java/org/apache/iceberg/expressions/BoundReference.java index 5a836504bcad..1e076d41fbb2 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/BoundReference.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.StructLike; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.util.List; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; public class BoundReference implements Reference { private final int fieldId; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Evaluator.java b/api/src/main/java/org/apache/iceberg/expressions/Evaluator.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/expressions/Evaluator.java rename to api/src/main/java/org/apache/iceberg/expressions/Evaluator.java index dacaba922d64..e96a5284670c 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Evaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Evaluator.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.StructLike; -import com.netflix.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import com.netflix.iceberg.types.Types; import java.io.Serializable; import java.util.Comparator; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; +import org.apache.iceberg.types.Types; /** * Evaluates an {@link Expression} for data described by a {@link Types.StructType}. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Expression.java b/api/src/main/java/org/apache/iceberg/expressions/Expression.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/expressions/Expression.java rename to api/src/main/java/org/apache/iceberg/expressions/Expression.java index d499b7651aa1..2c7534b6d20d 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Expression.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Expression.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import java.io.Serializable; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/ExpressionVisitors.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java similarity index 99% rename from api/src/main/java/com/netflix/iceberg/expressions/ExpressionVisitors.java rename to api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java index 4531b8cbe78d..9df28f3ea7ac 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/ExpressionVisitors.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; /** * Utils for traversing {@link Expression expressions}. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Expressions.java b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/Expressions.java rename to api/src/main/java/org/apache/iceberg/expressions/Expressions.java index 670789b97381..c42e9fa57a39 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Expressions.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import com.google.common.base.Preconditions; -import com.netflix.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expression.Operation; /** * Factory methods for creating {@link Expression expressions}. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/False.java b/api/src/main/java/org/apache/iceberg/expressions/False.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/False.java rename to api/src/main/java/org/apache/iceberg/expressions/False.java index 28b845cd798e..940750a67cca 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/False.java +++ b/api/src/main/java/org/apache/iceberg/expressions/False.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import java.io.ObjectStreamException; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveManifestEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/InclusiveManifestEvaluator.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/expressions/InclusiveManifestEvaluator.java rename to api/src/main/java/org/apache/iceberg/expressions/InclusiveManifestEvaluator.java index 203ae788b095..3d8fc687ff14 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveManifestEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/InclusiveManifestEvaluator.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg.expressions; - -import com.netflix.iceberg.ManifestFile; -import com.netflix.iceberg.ManifestFile.PartitionFieldSummary; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import com.netflix.iceberg.types.Conversions; -import com.netflix.iceberg.types.Types.StructType; +package org.apache.iceberg.expressions; + import java.nio.ByteBuffer; import java.util.List; - -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFile.PartitionFieldSummary; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types.StructType; + +import static org.apache.iceberg.expressions.Expressions.rewriteNot; /** * Evaluates an {@link Expression} on a {@link ManifestFile} to test whether the file contains diff --git a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveMetricsEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/expressions/InclusiveMetricsEvaluator.java rename to api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java index b8b0c48fc2bb..81e77561bc12 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/InclusiveMetricsEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import com.google.common.base.Preconditions; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import com.netflix.iceberg.types.Conversions; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; import java.nio.ByteBuffer; import java.util.Map; - -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.StructType; + +import static org.apache.iceberg.expressions.Expressions.rewriteNot; /** * Evaluates an {@link Expression} on a {@link DataFile} to test whether rows in the file may match. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Literal.java b/api/src/main/java/org/apache/iceberg/expressions/Literal.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/Literal.java rename to api/src/main/java/org/apache/iceberg/expressions/Literal.java index 69a2623303ae..b4642e31d2ff 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Literal.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literal.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.types.Type; import java.io.Serializable; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.Comparator; import java.util.UUID; +import org.apache.iceberg.types.Type; /** * Represents a literal fixed value in an expression predicate diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/expressions/Literals.java rename to api/src/main/java/org/apache/iceberg/expressions/Literals.java index f4e5d4e74446..4e2d4a4dcbae 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -17,12 +17,9 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import com.google.common.base.Preconditions; -import com.netflix.iceberg.types.Comparators; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.io.ObjectStreamException; import java.math.BigDecimal; import java.math.RoundingMode; @@ -37,6 +34,9 @@ import java.time.temporal.ChronoUnit; import java.util.Comparator; import java.util.UUID; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; class Literals { private Literals() { diff --git a/api/src/main/java/com/netflix/iceberg/expressions/NamedReference.java b/api/src/main/java/org/apache/iceberg/expressions/NamedReference.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/expressions/NamedReference.java rename to api/src/main/java/org/apache/iceberg/expressions/NamedReference.java index 5e4ec1e7a592..f5fcc8ebc471 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/NamedReference.java +++ b/api/src/main/java/org/apache/iceberg/expressions/NamedReference.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import com.google.common.base.Preconditions; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Not.java b/api/src/main/java/org/apache/iceberg/expressions/Not.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/expressions/Not.java rename to api/src/main/java/org/apache/iceberg/expressions/Not.java index f4edc9b4863c..ad71f41f0432 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Not.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Not.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; public class Not implements Expression { private final Expression child; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Or.java b/api/src/main/java/org/apache/iceberg/expressions/Or.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/Or.java rename to api/src/main/java/org/apache/iceberg/expressions/Or.java index 04a27df1b3a5..b41ef7f676be 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Or.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Or.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; public class Or implements Expression { private final Expression left; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Predicate.java b/api/src/main/java/org/apache/iceberg/expressions/Predicate.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/expressions/Predicate.java rename to api/src/main/java/org/apache/iceberg/expressions/Predicate.java index 2cf1d814a24a..12923ef2503f 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Predicate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Predicate.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; public abstract class Predicate implements Expression { private final Operation op; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Projections.java b/api/src/main/java/org/apache/iceberg/expressions/Projections.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/Projections.java rename to api/src/main/java/org/apache/iceberg/expressions/Projections.java index d54a33fdb251..9ebfae96090b 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Projections.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Projections.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; -import com.netflix.iceberg.PartitionField; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.transforms.Transform; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; +import org.apache.iceberg.transforms.Transform; /** * Utils to project expressions on rows to expressions on partitions. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/Reference.java b/api/src/main/java/org/apache/iceberg/expressions/Reference.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/expressions/Reference.java rename to api/src/main/java/org/apache/iceberg/expressions/Reference.java index be3144fcce39..f08867f7c102 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/Reference.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Reference.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import java.io.Serializable; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/ResidualEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/ResidualEvaluator.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/ResidualEvaluator.java rename to api/src/main/java/org/apache/iceberg/expressions/ResidualEvaluator.java index dff24e7241c6..459d20785fb6 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/ResidualEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ResidualEvaluator.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.StructLike; -import com.netflix.iceberg.PartitionField; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.transforms.Transform; import java.io.Serializable; import java.util.Comparator; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.transforms.Transform; /** * Finds the residuals for an {@link Expression} the partitions in the given {@link PartitionSpec}. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/RewriteNot.java b/api/src/main/java/org/apache/iceberg/expressions/RewriteNot.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/RewriteNot.java rename to api/src/main/java/org/apache/iceberg/expressions/RewriteNot.java index cc95d5f414fa..cc5b3c674de1 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/RewriteNot.java +++ b/api/src/main/java/org/apache/iceberg/expressions/RewriteNot.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; class RewriteNot extends ExpressionVisitors.ExpressionVisitor { private static final RewriteNot INSTANCE = new RewriteNot(); diff --git a/api/src/main/java/com/netflix/iceberg/expressions/SerializationProxies.java b/api/src/main/java/org/apache/iceberg/expressions/SerializationProxies.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/expressions/SerializationProxies.java rename to api/src/main/java/org/apache/iceberg/expressions/SerializationProxies.java index deac29ee170a..78da881608fd 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/SerializationProxies.java +++ b/api/src/main/java/org/apache/iceberg/expressions/SerializationProxies.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import java.io.ObjectStreamException; import java.io.Serializable; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/StrictMetricsEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/expressions/StrictMetricsEvaluator.java rename to api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java index 702c2557c908..6af5b2131c4c 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/StrictMetricsEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import com.google.common.base.Preconditions; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import com.netflix.iceberg.types.Conversions; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; import java.nio.ByteBuffer; import java.util.Map; - -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.StructType; + +import static org.apache.iceberg.expressions.Expressions.rewriteNot; /** * Evaluates an {@link Expression} on a {@link DataFile} to test whether all rows in the file match. diff --git a/api/src/main/java/com/netflix/iceberg/expressions/True.java b/api/src/main/java/org/apache/iceberg/expressions/True.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/expressions/True.java rename to api/src/main/java/org/apache/iceberg/expressions/True.java index b42adc03bcb4..8f43bc075407 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/True.java +++ b/api/src/main/java/org/apache/iceberg/expressions/True.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; import java.io.ObjectStreamException; diff --git a/api/src/main/java/com/netflix/iceberg/expressions/UnboundPredicate.java b/api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java similarity index 93% rename from api/src/main/java/com/netflix/iceberg/expressions/UnboundPredicate.java rename to api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java index da6f981f6679..e5dd33cb0cd2 100644 --- a/api/src/main/java/com/netflix/iceberg/expressions/UnboundPredicate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types; -import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; -import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.IS_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.NOT_NULL; public class UnboundPredicate extends Predicate { diff --git a/api/src/main/java/com/netflix/iceberg/io/CloseableGroup.java b/api/src/main/java/org/apache/iceberg/io/CloseableGroup.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/io/CloseableGroup.java rename to api/src/main/java/org/apache/iceberg/io/CloseableGroup.java index a3dc85cf8c69..8b5a977db7b7 100644 --- a/api/src/main/java/com/netflix/iceberg/io/CloseableGroup.java +++ b/api/src/main/java/org/apache/iceberg/io/CloseableGroup.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; import com.google.common.collect.Lists; import java.io.Closeable; diff --git a/api/src/main/java/com/netflix/iceberg/io/CloseableIterable.java b/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/io/CloseableIterable.java rename to api/src/main/java/org/apache/iceberg/io/CloseableIterable.java index 06259e681196..4baf2afc6977 100644 --- a/api/src/main/java/com/netflix/iceberg/io/CloseableIterable.java +++ b/api/src/main/java/org/apache/iceberg/io/CloseableIterable.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; import com.google.common.base.Preconditions; import java.io.Closeable; diff --git a/api/src/main/java/com/netflix/iceberg/io/DelegatingInputStream.java b/api/src/main/java/org/apache/iceberg/io/DelegatingInputStream.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/io/DelegatingInputStream.java rename to api/src/main/java/org/apache/iceberg/io/DelegatingInputStream.java index b9d3a785a332..701286250258 100644 --- a/api/src/main/java/com/netflix/iceberg/io/DelegatingInputStream.java +++ b/api/src/main/java/org/apache/iceberg/io/DelegatingInputStream.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; import java.io.InputStream; diff --git a/api/src/main/java/com/netflix/iceberg/io/DelegatingOutputStream.java b/api/src/main/java/org/apache/iceberg/io/DelegatingOutputStream.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/io/DelegatingOutputStream.java rename to api/src/main/java/org/apache/iceberg/io/DelegatingOutputStream.java index 3398ff386f1d..7228d0874154 100644 --- a/api/src/main/java/com/netflix/iceberg/io/DelegatingOutputStream.java +++ b/api/src/main/java/org/apache/iceberg/io/DelegatingOutputStream.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; import java.io.OutputStream; diff --git a/api/src/main/java/com/netflix/iceberg/io/FileAppender.java b/api/src/main/java/org/apache/iceberg/io/FileAppender.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/io/FileAppender.java rename to api/src/main/java/org/apache/iceberg/io/FileAppender.java index 535d7396db09..b98859a8b941 100644 --- a/api/src/main/java/com/netflix/iceberg/io/FileAppender.java +++ b/api/src/main/java/org/apache/iceberg/io/FileAppender.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; -import com.netflix.iceberg.Metrics; import java.io.Closeable; import java.util.Iterator; +import org.apache.iceberg.Metrics; public interface FileAppender extends Closeable { void add(D datum); diff --git a/api/src/main/java/com/netflix/iceberg/io/FileIO.java b/api/src/main/java/org/apache/iceberg/io/FileIO.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/io/FileIO.java rename to api/src/main/java/org/apache/iceberg/io/FileIO.java index 2c3558a01ba3..e466e2c43fdb 100644 --- a/api/src/main/java/com/netflix/iceberg/io/FileIO.java +++ b/api/src/main/java/org/apache/iceberg/io/FileIO.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; import java.io.Serializable; diff --git a/api/src/main/java/com/netflix/iceberg/io/InputFile.java b/api/src/main/java/org/apache/iceberg/io/InputFile.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/io/InputFile.java rename to api/src/main/java/org/apache/iceberg/io/InputFile.java index 91444f726b65..b576155920b6 100644 --- a/api/src/main/java/com/netflix/iceberg/io/InputFile.java +++ b/api/src/main/java/org/apache/iceberg/io/InputFile.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; -import com.netflix.iceberg.exceptions.RuntimeIOException; import java.io.IOException; +import org.apache.iceberg.exceptions.RuntimeIOException; /** * An interface used to read input files using {@link SeekableInputStream} instances. diff --git a/api/src/main/java/com/netflix/iceberg/io/LocationProvider.java b/api/src/main/java/org/apache/iceberg/io/LocationProvider.java similarity index 93% rename from api/src/main/java/com/netflix/iceberg/io/LocationProvider.java rename to api/src/main/java/org/apache/iceberg/io/LocationProvider.java index 3c37fd7475e3..58f70c1de883 100644 --- a/api/src/main/java/com/netflix/iceberg/io/LocationProvider.java +++ b/api/src/main/java/org/apache/iceberg/io/LocationProvider.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.StructLike; import java.io.Serializable; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; /** * Interface for providing data file locations to write tasks. diff --git a/api/src/main/java/com/netflix/iceberg/io/OutputFile.java b/api/src/main/java/org/apache/iceberg/io/OutputFile.java similarity index 93% rename from api/src/main/java/com/netflix/iceberg/io/OutputFile.java rename to api/src/main/java/org/apache/iceberg/io/OutputFile.java index f0f48ee7f73b..34b4e54abf62 100644 --- a/api/src/main/java/com/netflix/iceberg/io/OutputFile.java +++ b/api/src/main/java/org/apache/iceberg/io/OutputFile.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; -import com.netflix.iceberg.exceptions.AlreadyExistsException; -import com.netflix.iceberg.exceptions.RuntimeIOException; import java.io.IOException; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.RuntimeIOException; /** * An interface used to create output files using {@link PositionOutputStream} instances. diff --git a/api/src/main/java/com/netflix/iceberg/io/PositionOutputStream.java b/api/src/main/java/org/apache/iceberg/io/PositionOutputStream.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/io/PositionOutputStream.java rename to api/src/main/java/org/apache/iceberg/io/PositionOutputStream.java index 054ba9eb24c3..a6b34e092d8d 100644 --- a/api/src/main/java/com/netflix/iceberg/io/PositionOutputStream.java +++ b/api/src/main/java/org/apache/iceberg/io/PositionOutputStream.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; import java.io.IOException; import java.io.OutputStream; diff --git a/api/src/main/java/com/netflix/iceberg/io/SeekableInputStream.java b/api/src/main/java/org/apache/iceberg/io/SeekableInputStream.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/io/SeekableInputStream.java rename to api/src/main/java/org/apache/iceberg/io/SeekableInputStream.java index 807018af431f..f20d0c8bceee 100644 --- a/api/src/main/java/com/netflix/iceberg/io/SeekableInputStream.java +++ b/api/src/main/java/org/apache/iceberg/io/SeekableInputStream.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.io; +package org.apache.iceberg.io; import java.io.IOException; import java.io.InputStream; diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Bucket.java b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/transforms/Bucket.java rename to api/src/main/java/org/apache/iceberg/transforms/Bucket.java index 4f9c2fda5f49..b3dc4225768c 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Bucket.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Charsets; @@ -25,18 +25,18 @@ import com.google.common.collect.Sets; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Set; import java.util.UUID; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; -import static com.netflix.iceberg.types.Type.TypeID; +import static org.apache.iceberg.types.Type.TypeID; abstract class Bucket implements Transform { private static final HashFunction MURMUR3 = Hashing.murmur3_32(); diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Dates.java b/api/src/main/java/org/apache/iceberg/transforms/Dates.java similarity index 86% rename from api/src/main/java/com/netflix/iceberg/transforms/Dates.java rename to api/src/main/java/org/apache/iceberg/transforms/Dates.java index 0b45f0b0ebba..f8296374af10 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Dates.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Dates.java @@ -17,20 +17,20 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.time.Instant; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; -import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; -import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.IS_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.NOT_NULL; enum Dates implements Transform { YEAR(ChronoUnit.YEARS, "year"), diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Identity.java b/api/src/main/java/org/apache/iceberg/transforms/Identity.java similarity index 91% rename from api/src/main/java/com/netflix/iceberg/transforms/Identity.java rename to api/src/main/java/org/apache/iceberg/transforms/Identity.java index c1620ed17901..2f1ffa8e2e90 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Identity.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Identity.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; import com.google.common.base.Objects; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.nio.ByteBuffer; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; class Identity implements Transform { @SuppressWarnings("unchecked") diff --git a/api/src/main/java/com/netflix/iceberg/transforms/PartitionSpecVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/transforms/PartitionSpecVisitor.java rename to api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java index 0de992adfba2..e521909c893a 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/PartitionSpecVisitor.java +++ b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; import com.google.common.collect.Lists; -import com.netflix.iceberg.PartitionField; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; import java.util.List; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; public interface PartitionSpecVisitor { T identity(String sourceName, int sourceId); diff --git a/api/src/main/java/com/netflix/iceberg/transforms/ProjectionUtil.java b/api/src/main/java/org/apache/iceberg/transforms/ProjectionUtil.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/transforms/ProjectionUtil.java rename to api/src/main/java/org/apache/iceberg/transforms/ProjectionUtil.java index 06f0067fb49e..c5fd461f046c 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/ProjectionUtil.java +++ b/api/src/main/java/org/apache/iceberg/transforms/ProjectionUtil.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.UnboundPredicate; import java.math.BigDecimal; import java.math.BigInteger; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.UnboundPredicate; -import static com.netflix.iceberg.expressions.Expressions.predicate; +import static org.apache.iceberg.expressions.Expressions.predicate; class ProjectionUtil { static UnboundPredicate truncateInteger( diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Timestamps.java b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java similarity index 86% rename from api/src/main/java/com/netflix/iceberg/transforms/Timestamps.java rename to api/src/main/java/org/apache/iceberg/transforms/Timestamps.java index 7977e5bb97e0..b50b31369f9e 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Timestamps.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java @@ -17,20 +17,20 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.time.Instant; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; -import static com.netflix.iceberg.expressions.Expression.Operation.IS_NULL; -import static com.netflix.iceberg.expressions.Expression.Operation.NOT_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.IS_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.NOT_NULL; enum Timestamps implements Transform { YEAR(ChronoUnit.YEARS, "year"), diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Transform.java b/api/src/main/java/org/apache/iceberg/transforms/Transform.java similarity index 94% rename from api/src/main/java/com/netflix/iceberg/transforms/Transform.java rename to api/src/main/java/org/apache/iceberg/transforms/Transform.java index 3284774aef39..9bab11b5c49f 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Transform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transform.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.types.Type; import java.io.Serializable; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Type; /** * A transform function used for partitioning. diff --git a/api/src/main/java/com/netflix/iceberg/transforms/TransformUtil.java b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/transforms/TransformUtil.java rename to api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java index 8a71d44659b5..cd6f21c41d28 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/TransformUtil.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; import com.google.common.base.Charsets; import java.nio.ByteBuffer; diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Transforms.java b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/transforms/Transforms.java rename to api/src/main/java/org/apache/iceberg/transforms/Transforms.java index 044fe8928662..1881d255d304 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Transforms.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; import com.google.common.base.Preconditions; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.types.Type; import java.util.Locale; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; /** * Factory methods for transforms. diff --git a/api/src/main/java/com/netflix/iceberg/transforms/Truncate.java b/api/src/main/java/org/apache/iceberg/transforms/Truncate.java similarity index 95% rename from api/src/main/java/com/netflix/iceberg/transforms/Truncate.java rename to api/src/main/java/org/apache/iceberg/transforms/Truncate.java index d2542079f366..acf0df00a457 100644 --- a/api/src/main/java/com/netflix/iceberg/transforms/Truncate.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Truncate.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; import com.google.common.base.Objects; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.types.Type; import java.math.BigDecimal; 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; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Type; + +import static org.apache.iceberg.expressions.Expression.Operation.IS_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.LT; +import static org.apache.iceberg.expressions.Expression.Operation.LT_EQ; +import static org.apache.iceberg.expressions.Expression.Operation.NOT_NULL; abstract class Truncate implements Transform { @SuppressWarnings("unchecked") diff --git a/api/src/main/java/com/netflix/iceberg/types/AssignFreshIds.java b/api/src/main/java/org/apache/iceberg/types/AssignFreshIds.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/types/AssignFreshIds.java rename to api/src/main/java/org/apache/iceberg/types/AssignFreshIds.java index 0c5f65c067d0..c8006da3710f 100644 --- a/api/src/main/java/com/netflix/iceberg/types/AssignFreshIds.java +++ b/api/src/main/java/org/apache/iceberg/types/AssignFreshIds.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; import java.util.Iterator; import java.util.List; import java.util.function.Supplier; +import org.apache.iceberg.Schema; class AssignFreshIds extends TypeUtil.CustomOrderSchemaVisitor { private final TypeUtil.NextID nextId; diff --git a/api/src/main/java/com/netflix/iceberg/types/CheckCompatibility.java b/api/src/main/java/org/apache/iceberg/types/CheckCompatibility.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/types/CheckCompatibility.java rename to api/src/main/java/org/apache/iceberg/types/CheckCompatibility.java index 48ebb8b71b3c..9c65850dfa92 100644 --- a/api/src/main/java/com/netflix/iceberg/types/CheckCompatibility.java +++ b/api/src/main/java/org/apache/iceberg/types/CheckCompatibility.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.function.Supplier; +import org.apache.iceberg.Schema; -import static com.netflix.iceberg.types.TypeUtil.isPromotionAllowed; +import static org.apache.iceberg.types.TypeUtil.isPromotionAllowed; public class CheckCompatibility extends TypeUtil.CustomOrderSchemaVisitor> { /** diff --git a/api/src/main/java/com/netflix/iceberg/types/Comparators.java b/api/src/main/java/org/apache/iceberg/types/Comparators.java similarity index 99% rename from api/src/main/java/com/netflix/iceberg/types/Comparators.java rename to api/src/main/java/org/apache/iceberg/types/Comparators.java index 6680f7dd16ff..d885f9adcffa 100644 --- a/api/src/main/java/com/netflix/iceberg/types/Comparators.java +++ b/api/src/main/java/org/apache/iceberg/types/Comparators.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.collect.ImmutableMap; import java.nio.ByteBuffer; diff --git a/api/src/main/java/com/netflix/iceberg/types/Conversions.java b/api/src/main/java/org/apache/iceberg/types/Conversions.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/types/Conversions.java rename to api/src/main/java/org/apache/iceberg/types/Conversions.java index cfef71d133da..8fe8055d96cc 100644 --- a/api/src/main/java/com/netflix/iceberg/types/Conversions.java +++ b/api/src/main/java/org/apache/iceberg/types/Conversions.java @@ -17,10 +17,9 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.base.Charsets; -import com.netflix.iceberg.exceptions.RuntimeIOException; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -31,6 +30,7 @@ import java.nio.charset.CharsetEncoder; import java.util.Arrays; import java.util.UUID; +import org.apache.iceberg.exceptions.RuntimeIOException; public class Conversions { private static final String HIVE_NULL = "__HIVE_DEFAULT_PARTITION__"; diff --git a/api/src/main/java/com/netflix/iceberg/types/FindTypeVisitor.java b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/types/FindTypeVisitor.java rename to api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java index 7c11adf83c14..4febeb7e3685 100644 --- a/api/src/main/java/com/netflix/iceberg/types/FindTypeVisitor.java +++ b/api/src/main/java/org/apache/iceberg/types/FindTypeVisitor.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; -import com.netflix.iceberg.Schema; import java.util.List; import java.util.function.Predicate; +import org.apache.iceberg.Schema; class FindTypeVisitor extends TypeUtil.SchemaVisitor { private final Predicate predicate; diff --git a/api/src/main/java/com/netflix/iceberg/types/GetProjectedIds.java b/api/src/main/java/org/apache/iceberg/types/GetProjectedIds.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/types/GetProjectedIds.java rename to api/src/main/java/org/apache/iceberg/types/GetProjectedIds.java index 93809c5d5d58..d33b2f9de991 100644 --- a/api/src/main/java/com/netflix/iceberg/types/GetProjectedIds.java +++ b/api/src/main/java/org/apache/iceberg/types/GetProjectedIds.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; import java.util.List; import java.util.Set; +import org.apache.iceberg.Schema; class GetProjectedIds extends TypeUtil.SchemaVisitor> { private final Set fieldIds = Sets.newHashSet(); diff --git a/api/src/main/java/com/netflix/iceberg/types/IndexById.java b/api/src/main/java/org/apache/iceberg/types/IndexById.java similarity index 96% rename from api/src/main/java/com/netflix/iceberg/types/IndexById.java rename to api/src/main/java/org/apache/iceberg/types/IndexById.java index 84d8061e19ed..0f368362af38 100644 --- a/api/src/main/java/com/netflix/iceberg/types/IndexById.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexById.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; import java.util.List; import java.util.Map; +import org.apache.iceberg.Schema; class IndexById extends TypeUtil.SchemaVisitor> { private final Map index = Maps.newHashMap(); diff --git a/api/src/main/java/com/netflix/iceberg/types/IndexByName.java b/api/src/main/java/org/apache/iceberg/types/IndexByName.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/types/IndexByName.java rename to api/src/main/java/org/apache/iceberg/types/IndexByName.java index e98a3ba8e99b..4acbcaa29bfa 100644 --- a/api/src/main/java/com/netflix/iceberg/types/IndexByName.java +++ b/api/src/main/java/org/apache/iceberg/types/IndexByName.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.base.Joiner; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; import java.util.List; import java.util.Map; +import org.apache.iceberg.Schema; public class IndexByName extends TypeUtil.SchemaVisitor> { private static final Joiner DOT = Joiner.on("."); diff --git a/api/src/main/java/com/netflix/iceberg/types/PrimitiveHolder.java b/api/src/main/java/org/apache/iceberg/types/PrimitiveHolder.java similarity index 97% rename from api/src/main/java/com/netflix/iceberg/types/PrimitiveHolder.java rename to api/src/main/java/org/apache/iceberg/types/PrimitiveHolder.java index 244a99a94946..5571babdc1d5 100644 --- a/api/src/main/java/com/netflix/iceberg/types/PrimitiveHolder.java +++ b/api/src/main/java/org/apache/iceberg/types/PrimitiveHolder.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import java.io.ObjectStreamException; import java.io.Serializable; diff --git a/api/src/main/java/com/netflix/iceberg/types/PruneColumns.java b/api/src/main/java/org/apache/iceberg/types/PruneColumns.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/types/PruneColumns.java rename to api/src/main/java/org/apache/iceberg/types/PruneColumns.java index 7df09bb20f5f..b303282145e7 100644 --- a/api/src/main/java/com/netflix/iceberg/types/PruneColumns.java +++ b/api/src/main/java/org/apache/iceberg/types/PruneColumns.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; import java.util.List; import java.util.Set; +import org.apache.iceberg.Schema; class PruneColumns extends TypeUtil.SchemaVisitor { private final Set selected; diff --git a/api/src/main/java/com/netflix/iceberg/types/ReassignIds.java b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/types/ReassignIds.java rename to api/src/main/java/org/apache/iceberg/types/ReassignIds.java index 4dccb74ef8b4..98cced66c38c 100644 --- a/api/src/main/java/com/netflix/iceberg/types/ReassignIds.java +++ b/api/src/main/java/org/apache/iceberg/types/ReassignIds.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; import java.util.List; import java.util.function.Supplier; +import org.apache.iceberg.Schema; class ReassignIds extends TypeUtil.CustomOrderSchemaVisitor { private final Schema sourceSchema; diff --git a/api/src/main/java/com/netflix/iceberg/types/Type.java b/api/src/main/java/org/apache/iceberg/types/Type.java similarity index 98% rename from api/src/main/java/com/netflix/iceberg/types/Type.java rename to api/src/main/java/org/apache/iceberg/types/Type.java index b2df14f66e82..9eac853262b5 100644 --- a/api/src/main/java/com/netflix/iceberg/types/Type.java +++ b/api/src/main/java/org/apache/iceberg/types/Type.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import java.io.ObjectStreamException; import java.io.Serializable; diff --git a/api/src/main/java/com/netflix/iceberg/types/TypeUtil.java b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java similarity index 99% rename from api/src/main/java/com/netflix/iceberg/types/TypeUtil.java rename to api/src/main/java/org/apache/iceberg/types/TypeUtil.java index 81bb4227a600..83fbd742cd5e 100644 --- a/api/src/main/java/com/netflix/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -25,7 +25,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; import java.util.LinkedList; import java.util.List; import java.util.Locale; @@ -33,6 +32,7 @@ import java.util.Set; import java.util.function.Predicate; import java.util.function.Supplier; +import org.apache.iceberg.Schema; public class TypeUtil { public static Schema select(Schema schema, Set fieldIds) { diff --git a/api/src/main/java/com/netflix/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java similarity index 99% rename from api/src/main/java/com/netflix/iceberg/types/Types.java rename to api/src/main/java/org/apache/iceberg/types/Types.java index a4ef5ac2bd59..e924cd4d1811 100644 --- a/api/src/main/java/com/netflix/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -17,14 +17,12 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.types.Type.NestedType; -import com.netflix.iceberg.types.Type.PrimitiveType; import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -33,6 +31,8 @@ import java.util.Objects; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.iceberg.types.Type.NestedType; +import org.apache.iceberg.types.Type.PrimitiveType; public class Types { private static final ImmutableMap TYPES = ImmutableMap diff --git a/api/src/test/java/com/netflix/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java similarity index 97% rename from api/src/test/java/com/netflix/iceberg/TestHelpers.java rename to api/src/test/java/org/apache/iceberg/TestHelpers.java index 70dab6456303..bcf55533ecb7 100644 --- a/api/src/test/java/com/netflix/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -17,14 +17,8 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.encryption.EncryptionKeyMetadata; -import com.netflix.iceberg.expressions.BoundPredicate; -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; @@ -34,6 +28,11 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.junit.Assert; public class TestHelpers { public static T assertAndUnwrap(Expression expr, Class expected) { diff --git a/api/src/test/java/com/netflix/iceberg/TestPartitionPaths.java b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java similarity index 91% rename from api/src/test/java/com/netflix/iceberg/TestPartitionPaths.java rename to api/src/test/java/org/apache/iceberg/TestPartitionPaths.java index 1253f1e91017..8cb0e0f2aede 100644 --- a/api/src/test/java/com/netflix/iceberg/TestPartitionPaths.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.TestHelpers.Row; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.transforms.Transform; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; diff --git a/api/src/test/java/com/netflix/iceberg/events/TestListeners.java b/api/src/test/java/org/apache/iceberg/events/TestListeners.java similarity index 98% rename from api/src/test/java/com/netflix/iceberg/events/TestListeners.java rename to api/src/test/java/org/apache/iceberg/events/TestListeners.java index 2f2052796db4..5e1c1108a1b4 100644 --- a/api/src/test/java/com/netflix/iceberg/events/TestListeners.java +++ b/api/src/test/java/org/apache/iceberg/events/TestListeners.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.events; +package org.apache.iceberg.events; import org.junit.Assert; import org.junit.Test; diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestEvaluatior.java b/api/src/test/java/org/apache/iceberg/expressions/TestEvaluatior.java similarity index 83% rename from api/src/test/java/com/netflix/iceberg/expressions/TestEvaluatior.java rename to api/src/test/java/org/apache/iceberg/expressions/TestEvaluatior.java index 0c08903f0b80..9494226c664c 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestEvaluatior.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestEvaluatior.java @@ -17,31 +17,31 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.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.apache.avro.util.Utf8; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types; +import org.apache.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; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notEqual; -import static com.netflix.iceberg.expressions.Expressions.notNull; -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 static org.apache.iceberg.expressions.Expressions.alwaysFalse; +import static org.apache.iceberg.expressions.Expressions.alwaysTrue; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestEvaluatior { private static final StructType STRUCT = StructType.of( diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java similarity index 87% rename from api/src/test/java/com/netflix/iceberg/expressions/TestExpressionBinding.java rename to api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java index 266e863774db..3e846999a567 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java @@ -17,24 +17,24 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.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.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types; +import org.apache.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; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.or; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.expressions.Expressions.alwaysFalse; +import static org.apache.iceberg.expressions.Expressions.alwaysTrue; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestExpressionBinding { private static final StructType STRUCT = StructType.of( diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionHelpers.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java similarity index 81% rename from api/src/test/java/com/netflix/iceberg/expressions/TestExpressionHelpers.java rename to api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java index e67d31caa6ef..6ca11ad94f1c 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionHelpers.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.TestHelpers; +import org.apache.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; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.expressions.Expressions.alwaysFalse; +import static org.apache.iceberg.expressions.Expressions.alwaysTrue; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.or; 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/org/apache/iceberg/expressions/TestExpressionSerialization.java similarity index 95% rename from api/src/test/java/com/netflix/iceberg/expressions/TestExpressionSerialization.java rename to api/src/test/java/org/apache/iceberg/expressions/TestExpressionSerialization.java index 1c6c564e237a..d84ca548e1d0 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestExpressionSerialization.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionSerialization.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.expressions.Expression.Operation; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveManifestEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java similarity index 91% rename from api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveManifestEvaluator.java rename to api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java index b65a61c4a3df..c62d32c261a8 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveManifestEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java @@ -17,34 +17,33 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.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 java.nio.ByteBuffer; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.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; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notEqual; -import static com.netflix.iceberg.expressions.Expressions.notNull; -import static com.netflix.iceberg.expressions.Expressions.or; -import static com.netflix.iceberg.types.Conversions.toByteBuffer; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Conversions.toByteBuffer; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestInclusiveManifestEvaluator { private static final Schema SCHEMA = new Schema( diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java similarity index 91% rename from api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveMetricsEvaluator.java rename to api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java index 12b8d0d9df24..6014ce6ff9ac 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestInclusiveMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java @@ -17,35 +17,34 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.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.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.TestHelpers.TestDataFile; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types; +import org.apache.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; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notEqual; -import static com.netflix.iceberg.expressions.Expressions.notNull; -import static com.netflix.iceberg.expressions.Expressions.or; -import static com.netflix.iceberg.types.Conversions.toByteBuffer; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Conversions.toByteBuffer; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestInclusiveMetricsEvaluator { private static final Schema SCHEMA = new Schema( diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestLiteralSerialization.java b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java similarity index 94% rename from api/src/test/java/com/netflix/iceberg/expressions/TestLiteralSerialization.java rename to api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java index 5108c84cb3bb..1a3e6487a921 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestLiteralSerialization.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -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.apache.iceberg.TestHelpers; +import org.apache.iceberg.types.Types; +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/org/apache/iceberg/expressions/TestMiscLiteralConversions.java similarity index 98% rename from api/src/test/java/com/netflix/iceberg/expressions/TestMiscLiteralConversions.java rename to api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java index 3947b1197dbb..25c1c16ef98d 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestMiscLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -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.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; public class TestMiscLiteralConversions { @Test diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestNumericLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestNumericLiteralConversions.java similarity index 98% rename from api/src/test/java/com/netflix/iceberg/expressions/TestNumericLiteralConversions.java rename to api/src/test/java/org/apache/iceberg/expressions/TestNumericLiteralConversions.java index 4bf37951c638..8cbbe45aa08c 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestNumericLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestNumericLiteralConversions.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.iceberg.expressions; -import com.netflix.iceberg.types.Types; +import java.math.BigDecimal; +import org.apache.iceberg.types.Types; 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/org/apache/iceberg/expressions/TestPredicateBinding.java similarity index 92% rename from api/src/test/java/com/netflix/iceberg/expressions/TestPredicateBinding.java rename to api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java index c30986f5dd24..42d1157ac9fa 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestPredicateBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java @@ -17,29 +17,29 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.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 org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.StructType; +import org.junit.Assert; +import org.junit.Test; -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 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 static org.apache.iceberg.TestHelpers.assertAndUnwrap; +import static org.apache.iceberg.expressions.Expression.Operation.EQ; +import static org.apache.iceberg.expressions.Expression.Operation.GT; +import static org.apache.iceberg.expressions.Expression.Operation.GT_EQ; +import static org.apache.iceberg.expressions.Expression.Operation.IS_NULL; +import static org.apache.iceberg.expressions.Expression.Operation.LT; +import static org.apache.iceberg.expressions.Expression.Operation.LT_EQ; +import static org.apache.iceberg.expressions.Expression.Operation.NOT_EQ; +import static org.apache.iceberg.expressions.Expression.Operation.NOT_NULL; +import static org.apache.iceberg.expressions.Expressions.ref; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestPredicateBinding { private static List COMPARISONS = Arrays.asList( diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestStrictMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java similarity index 90% rename from api/src/test/java/com/netflix/iceberg/expressions/TestStrictMetricsEvaluator.java rename to api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java index da785f8a0e7c..2269c07bc02c 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestStrictMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java @@ -17,36 +17,34 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.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.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.TestHelpers.TestDataFile; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.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; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notEqual; -import static com.netflix.iceberg.expressions.Expressions.notNull; -import static com.netflix.iceberg.expressions.Expressions.or; -import static com.netflix.iceberg.types.Conversions.toByteBuffer; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Conversions.toByteBuffer; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestStrictMetricsEvaluator { private static final Schema SCHEMA = new Schema( diff --git a/api/src/test/java/com/netflix/iceberg/expressions/TestStringLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java similarity index 98% rename from api/src/test/java/com/netflix/iceberg/expressions/TestStringLiteralConversions.java rename to api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java index bb1b7ce10960..0c71dde758ed 100644 --- a/api/src/test/java/com/netflix/iceberg/expressions/TestStringLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg.expressions; +package org.apache.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; +import org.apache.iceberg.types.Types; import org.joda.time.DateTimeZone; import org.joda.time.LocalDate; import org.joda.time.LocalDateTime; 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/org/apache/iceberg/transforms/TestBucketing.java similarity index 98% rename from api/src/test/java/com/netflix/iceberg/transforms/TestBucketing.java rename to api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java index 04db9e9060e3..6b1ddd7c4ee1 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java @@ -17,18 +17,11 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.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; @@ -36,6 +29,13 @@ import java.nio.ByteOrder; import java.util.Random; import java.util.UUID; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +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(); diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestDates.java b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java similarity index 94% rename from api/src/test/java/com/netflix/iceberg/transforms/TestDates.java rename to api/src/test/java/org/apache/iceberg/transforms/TestDates.java index 9dc3cf1a86dc..f8229ccb4dd7 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestDates.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestIdentity.java b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java similarity index 97% rename from api/src/test/java/com/netflix/iceberg/transforms/TestIdentity.java rename to api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java index 93afd4bceabd..b3396513f909 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestIdentity.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -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.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; public class TestIdentity { @Test diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestProjection.java b/api/src/test/java/org/apache/iceberg/transforms/TestProjection.java similarity index 89% rename from api/src/test/java/com/netflix/iceberg/transforms/TestProjection.java rename to api/src/test/java/org/apache/iceberg/transforms/TestProjection.java index 546cb853ac14..f17e6d9e0720 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestProjection.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestProjection.java @@ -17,33 +17,33 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expressions; -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.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Or; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Types; 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.TestHelpers.assertThrows; -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 static org.apache.iceberg.TestHelpers.assertAndUnwrap; +import static org.apache.iceberg.TestHelpers.assertAndUnwrapUnbound; +import static org.apache.iceberg.TestHelpers.assertThrows; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.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/org/apache/iceberg/transforms/TestResiduals.java similarity index 81% rename from api/src/test/java/com/netflix/iceberg/transforms/TestResiduals.java rename to api/src/test/java/org/apache/iceberg/transforms/TestResiduals.java index ec0cbe7e143f..3b60f744daa4 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestResiduals.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestResiduals.java @@ -17,29 +17,29 @@ * under the License. */ -package com.netflix.iceberg.transforms; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers.Row; -import com.netflix.iceberg.exceptions.ValidationException; -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; +package org.apache.iceberg.transforms; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.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; -import static com.netflix.iceberg.expressions.Expressions.alwaysFalse; -import static com.netflix.iceberg.expressions.Expressions.alwaysTrue; -import static com.netflix.iceberg.expressions.Expressions.and; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.TestHelpers.assertAndUnwrapUnbound; +import static org.apache.iceberg.expressions.Expression.Operation.GT; +import static org.apache.iceberg.expressions.Expression.Operation.LT; +import static org.apache.iceberg.expressions.Expressions.alwaysFalse; +import static org.apache.iceberg.expressions.Expressions.alwaysTrue; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.or; public class TestResiduals { @Test diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestTimestamps.java b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java similarity index 96% rename from api/src/test/java/com/netflix/iceberg/transforms/TestTimestamps.java rename to api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java index c05aff33b83f..25e22a519913 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestTimestamps.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestTransformSerialization.java b/api/src/test/java/org/apache/iceberg/transforms/TestTransformSerialization.java similarity index 93% rename from api/src/test/java/com/netflix/iceberg/transforms/TestTransformSerialization.java rename to api/src/test/java/org/apache/iceberg/transforms/TestTransformSerialization.java index e194e704bc2b..c5561a3fb402 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestTransformSerialization.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTransformSerialization.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.iceberg.transforms; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; diff --git a/api/src/test/java/com/netflix/iceberg/transforms/TestTruncate.java b/api/src/test/java/org/apache/iceberg/transforms/TestTruncate.java similarity index 97% rename from api/src/test/java/com/netflix/iceberg/transforms/TestTruncate.java rename to api/src/test/java/org/apache/iceberg/transforms/TestTruncate.java index d42d43e15633..3d0c8e3f4a0e 100644 --- a/api/src/test/java/com/netflix/iceberg/transforms/TestTruncate.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTruncate.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.transforms; +package org.apache.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.apache.iceberg.types.Types; +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/org/apache/iceberg/types/TestBinaryComparator.java similarity index 96% rename from api/src/test/java/com/netflix/iceberg/types/TestBinaryComparator.java rename to api/src/test/java/org/apache/iceberg/types/TestBinaryComparator.java index 9efa2d0536be..39655cabcd75 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestBinaryComparator.java +++ b/api/src/test/java/org/apache/iceberg/types/TestBinaryComparator.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.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.apache.iceberg.expressions.Literal; +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/org/apache/iceberg/types/TestCharSeqComparator.java similarity index 97% rename from api/src/test/java/com/netflix/iceberg/types/TestCharSeqComparator.java rename to api/src/test/java/org/apache/iceberg/types/TestCharSeqComparator.java index 4e3b4a3565cb..a2edb69c4d9e 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestCharSeqComparator.java +++ b/api/src/test/java/org/apache/iceberg/types/TestCharSeqComparator.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; -import com.netflix.iceberg.expressions.Literal; +import java.util.Comparator; import org.apache.avro.util.Utf8; +import org.apache.iceberg.expressions.Literal; 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/org/apache/iceberg/types/TestComparableComparator.java similarity index 95% rename from api/src/test/java/com/netflix/iceberg/types/TestComparableComparator.java rename to api/src/test/java/org/apache/iceberg/types/TestComparableComparator.java index 1f3f6e39995a..309d885b8fae 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestComparableComparator.java +++ b/api/src/test/java/org/apache/iceberg/types/TestComparableComparator.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; -import com.netflix.iceberg.expressions.Literal; +import java.util.Comparator; +import org.apache.iceberg.expressions.Literal; 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/org/apache/iceberg/types/TestReadabilityChecks.java similarity index 98% rename from api/src/test/java/com/netflix/iceberg/types/TestReadabilityChecks.java rename to api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java index 90ccbed44ec0..e905df3f6488 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestReadabilityChecks.java +++ b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; -import com.netflix.iceberg.Schema; +import java.util.List; +import org.apache.iceberg.Schema; 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; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestReadabilityChecks { private static final Type.PrimitiveType[] PRIMITIVES = new Type.PrimitiveType[] { diff --git a/api/src/test/java/com/netflix/iceberg/types/TestSerializableTypes.java b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java similarity index 95% rename from api/src/test/java/com/netflix/iceberg/types/TestSerializableTypes.java rename to api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java index 757bd72b728b..82e0636e4158 100644 --- a/api/src/test/java/com/netflix/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.types; +package org.apache.iceberg.types; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; +import org.apache.iceberg.Schema; +import org.apache.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; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestSerializableTypes { @Test diff --git a/build.gradle b/build.gradle index f6be71179d2f..e715274b5d45 100644 --- a/build.gradle +++ b/build.gradle @@ -36,7 +36,7 @@ if (JavaVersion.current() != JavaVersion.VERSION_1_8) { } allprojects { - group = "com.netflix.iceberg" + group = "org.apache.iceberg" apply plugin: 'idea' version = gitVersion() } @@ -278,9 +278,9 @@ project(':iceberg-runtime') { // Relocate dependencies to avoid conflicts // relocate 'com.google.common', 'com.netflix.bdp.shaded.com.google.common' - relocate 'com.fasterxml', 'com.netflix.iceberg.shaded.com.fasterxml' - relocate 'org.apache.avro', 'com.netflix.iceberg.shaded.org.apache.avro' - relocate 'org.apache.parquet.avro', 'com.netflix.iceberg.shaded.org.apache.parquet.avro' + relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'org.apache.parquet.avro', 'org.apache.iceberg.shaded.org.apache.parquet.avro' archiveName = "iceberg-runtime-${version}.${extension}" } @@ -344,10 +344,10 @@ project(':iceberg-presto-runtime') { configurations = [project.configurations.shadow] from(project.sourceSets.main.output) - relocate 'org.apache.avro', 'com.netflix.iceberg.shaded.org.apache.avro' - relocate 'org.apache.parquet.avro', 'com.netflix.iceberg.shaded.org.apache.parquet.avro' - relocate 'org.apache.hadoop.hive', 'com.netflix.iceberg.shaded.org.apache.hadoop.hive' - relocate 'org.apache.hive', 'com.netflix.iceberg.shaded.org.apache.hive' + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'org.apache.parquet.avro', 'org.apache.iceberg.shaded.org.apache.parquet.avro' + relocate 'org.apache.hadoop.hive', 'org.apache.iceberg.shaded.org.apache.hadoop.hive' + relocate 'org.apache.hive', 'org.apache.iceberg.shaded.org.apache.hive' archiveName = "iceberg-presto-runtime-${version}.${extension}" } } diff --git a/common/src/main/java/com/netflix/iceberg/common/DynClasses.java b/common/src/main/java/org/apache/iceberg/common/DynClasses.java similarity index 99% rename from common/src/main/java/com/netflix/iceberg/common/DynClasses.java rename to common/src/main/java/org/apache/iceberg/common/DynClasses.java index 7acbe3512e16..2b03a2983ce5 100644 --- a/common/src/main/java/com/netflix/iceberg/common/DynClasses.java +++ b/common/src/main/java/org/apache/iceberg/common/DynClasses.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.common; +package org.apache.iceberg.common; import com.google.common.base.Joiner; import java.util.LinkedHashSet; diff --git a/common/src/main/java/com/netflix/iceberg/common/DynConstructors.java b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java similarity index 99% rename from common/src/main/java/com/netflix/iceberg/common/DynConstructors.java rename to common/src/main/java/org/apache/iceberg/common/DynConstructors.java index 08acc3ef67c5..660105c8a716 100644 --- a/common/src/main/java/com/netflix/iceberg/common/DynConstructors.java +++ b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.common; +package org.apache.iceberg.common; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; diff --git a/common/src/main/java/com/netflix/iceberg/common/DynFields.java b/common/src/main/java/org/apache/iceberg/common/DynFields.java similarity index 99% rename from common/src/main/java/com/netflix/iceberg/common/DynFields.java rename to common/src/main/java/org/apache/iceberg/common/DynFields.java index 4144b28d7777..19cdc9224f7a 100644 --- a/common/src/main/java/com/netflix/iceberg/common/DynFields.java +++ b/common/src/main/java/org/apache/iceberg/common/DynFields.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.common; +package org.apache.iceberg.common; import com.google.common.base.Joiner; import com.google.common.base.Objects; diff --git a/common/src/main/java/com/netflix/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java similarity index 99% rename from common/src/main/java/com/netflix/iceberg/common/DynMethods.java rename to common/src/main/java/org/apache/iceberg/common/DynMethods.java index f7d62960f013..6c5242a7d022 100644 --- a/common/src/main/java/com/netflix/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.common; +package org.apache.iceberg.common; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; diff --git a/core/src/main/java/com/netflix/iceberg/BaseCombinedScanTask.java b/core/src/main/java/org/apache/iceberg/BaseCombinedScanTask.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/BaseCombinedScanTask.java rename to core/src/main/java/org/apache/iceberg/BaseCombinedScanTask.java index 71fb25046f9c..e8513ab9bc78 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseCombinedScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseCombinedScanTask.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableList; import java.util.Collection; diff --git a/core/src/main/java/com/netflix/iceberg/BaseFileScanTask.java b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/BaseFileScanTask.java rename to core/src/main/java/org/apache/iceberg/BaseFileScanTask.java index 0467f90a7841..6f179e4a1ed5 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseFileScanTask.java +++ b/core/src/main/java/org/apache/iceberg/BaseFileScanTask.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.ResidualEvaluator; import java.util.Iterator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ResidualEvaluator; class BaseFileScanTask implements FileScanTask { private final DataFile file; diff --git a/core/src/main/java/com/netflix/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java similarity index 90% rename from core/src/main/java/com/netflix/iceberg/BaseMetastoreTableOperations.java rename to core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 4cda7f8f53fc..f420e70559db 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -17,28 +17,26 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.hadoop.HadoopFileIO; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.io.LocationProvider; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.util.PropertyUtil; -import com.netflix.iceberg.util.Tasks; +import java.io.IOException; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.UUID; - -import static com.netflix.iceberg.TableMetadataParser.getFileExtension; -import static com.netflix.iceberg.TableMetadataParser.read; -import static com.netflix.iceberg.hadoop.HadoopInputFile.fromLocation; +import static org.apache.iceberg.TableMetadataParser.getFileExtension; +import static org.apache.iceberg.TableMetadataParser.read; +import static org.apache.iceberg.hadoop.HadoopInputFile.fromLocation; public abstract class BaseMetastoreTableOperations implements TableOperations { diff --git a/core/src/main/java/com/netflix/iceberg/BaseMetastoreTables.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTables.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/BaseMetastoreTables.java rename to core/src/main/java/org/apache/iceberg/BaseMetastoreTables.java index 6c125fecaeb0..121b5940296f 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseMetastoreTables.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTables.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.exceptions.AlreadyExistsException; -import com.netflix.iceberg.exceptions.NoSuchTableException; -import org.apache.hadoop.conf.Configuration; import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; -import static com.netflix.iceberg.TableMetadata.newTableMetadata; +import static org.apache.iceberg.TableMetadata.newTableMetadata; public abstract class BaseMetastoreTables implements Tables { private final Configuration conf; diff --git a/core/src/main/java/com/netflix/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/BaseSnapshot.java rename to core/src/main/java/org/apache/iceberg/BaseSnapshot.java index 8289ea2fc0f2..6b430323f05f 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.InputFile; import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.Map; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; class BaseSnapshot implements Snapshot { private final TableOperations ops; diff --git a/core/src/main/java/com/netflix/iceberg/BaseTable.java b/core/src/main/java/org/apache/iceberg/BaseTable.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/BaseTable.java rename to core/src/main/java/org/apache/iceberg/BaseTable.java index 16c7655a0e8d..4fb69c972fa4 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseTable.java @@ -17,12 +17,12 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.encryption.EncryptionManager; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.io.LocationProvider; import java.util.Map; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; /** * Base {@link Table} implementation. diff --git a/core/src/main/java/com/netflix/iceberg/BaseTableScan.java b/core/src/main/java/org/apache/iceberg/BaseTableScan.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/BaseTableScan.java rename to core/src/main/java/org/apache/iceberg/BaseTableScan.java index 4a39409431e2..e4c8de6d8d0e 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseTableScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseTableScan.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; import com.google.common.base.Preconditions; @@ -28,20 +28,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import com.netflix.iceberg.TableMetadata.SnapshotLogEntry; -import com.netflix.iceberg.events.Listeners; -import com.netflix.iceberg.events.ScanEvent; -import com.netflix.iceberg.expressions.Binder; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.InclusiveManifestEvaluator; -import com.netflix.iceberg.expressions.ResidualEvaluator; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.util.BinPacking; -import com.netflix.iceberg.util.ParallelIterable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.Closeable; import java.text.SimpleDateFormat; import java.util.Collection; @@ -51,8 +37,22 @@ import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.Function; +import org.apache.iceberg.TableMetadata.SnapshotLogEntry; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.InclusiveManifestEvaluator; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.BinPacking; +import org.apache.iceberg.util.ParallelIterable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static com.netflix.iceberg.util.ThreadPools.getWorkerPool; +import static org.apache.iceberg.util.ThreadPools.getWorkerPool; /** * Base class for {@link TableScan} implementations. diff --git a/core/src/main/java/com/netflix/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/BaseTransaction.java rename to core/src/main/java/org/apache/iceberg/BaseTransaction.java index 5403f75a2345..472c10792cce 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -17,28 +17,28 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.encryption.EncryptionManager; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.io.LocationProvider; -import com.netflix.iceberg.util.Tasks; import java.util.List; import java.util.Map; import java.util.Set; - -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.util.Tasks; + +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; class BaseTransaction implements Transaction { private enum TransactionType { diff --git a/core/src/main/java/com/netflix/iceberg/ConfigProperties.java b/core/src/main/java/org/apache/iceberg/ConfigProperties.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/ConfigProperties.java rename to core/src/main/java/org/apache/iceberg/ConfigProperties.java index 1b6858edeca6..7a9465f703f1 100644 --- a/core/src/main/java/com/netflix/iceberg/ConfigProperties.java +++ b/core/src/main/java/org/apache/iceberg/ConfigProperties.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import org.apache.hadoop.conf.Configuration; diff --git a/core/src/main/java/com/netflix/iceberg/DataFiles.java b/core/src/main/java/org/apache/iceberg/DataFiles.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/DataFiles.java rename to core/src/main/java/org/apache/iceberg/DataFiles.java index d26980b50cad..90b483cc54e6 100644 --- a/core/src/main/java/com/netflix/iceberg/DataFiles.java +++ b/core/src/main/java/org/apache/iceberg/DataFiles.java @@ -17,21 +17,20 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; -import com.netflix.iceberg.encryption.EncryptedInputFile; -import com.netflix.iceberg.encryption.EncryptedOutputFile; -import com.netflix.iceberg.encryption.EncryptionKeyMetadata; -import com.netflix.iceberg.hadoop.HadoopInputFile; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.types.Conversions; -import com.netflix.iceberg.util.ByteBuffers; -import org.apache.hadoop.fs.FileStatus; import java.nio.ByteBuffer; import java.util.List; import java.util.Locale; import java.util.Map; +import org.apache.hadoop.fs.FileStatus; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.util.ByteBuffers; public class DataFiles { diff --git a/core/src/main/java/com/netflix/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java similarity index 94% rename from core/src/main/java/com/netflix/iceberg/FastAppend.java rename to core/src/main/java/org/apache/iceberg/FastAppend.java index 63bfcfcf69ba..2bb90387737e 100644 --- a/core/src/main/java/com/netflix/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.OutputFile; import java.io.IOException; import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; /** * {@link AppendFiles Append} implementation that adds a new manifest file for the write. diff --git a/core/src/main/java/com/netflix/iceberg/FileHistory.java b/core/src/main/java/org/apache/iceberg/FileHistory.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/FileHistory.java rename to core/src/main/java/org/apache/iceberg/FileHistory.java index 60146b0c8b9a..37d6511741af 100644 --- a/core/src/main/java/com/netflix/iceberg/FileHistory.java +++ b/core/src/main/java/org/apache/iceberg/FileHistory.java @@ -17,20 +17,20 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.CharSequenceWrapper; import java.io.IOException; import java.util.List; import java.util.Set; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceWrapper; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.transform; diff --git a/core/src/main/java/com/netflix/iceberg/FilteredManifest.java b/core/src/main/java/org/apache/iceberg/FilteredManifest.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/FilteredManifest.java rename to core/src/main/java/org/apache/iceberg/FilteredManifest.java index 10c68cb0ec9b..d1addc032b3b 100644 --- a/core/src/main/java/com/netflix/iceberg/FilteredManifest.java +++ b/core/src/main/java/org/apache/iceberg/FilteredManifest.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; -import com.netflix.iceberg.ManifestEntry.Status; -import com.netflix.iceberg.expressions.Evaluator; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.InclusiveMetricsEvaluator; -import com.netflix.iceberg.expressions.Projections; import java.util.Collection; import java.util.Iterator; +import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.InclusiveMetricsEvaluator; +import org.apache.iceberg.expressions.Projections; public class FilteredManifest implements Filterable { private final ManifestReader reader; diff --git a/core/src/main/java/com/netflix/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/GenericDataFile.java rename to core/src/main/java/org/apache/iceberg/GenericDataFile.java index d1000b664184..b0b9299ecdc1 100644 --- a/core/src/main/java/com/netflix/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.ByteBuffers; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.specific.SpecificData; import java.io.Serializable; import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.specific.SpecificData; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; class GenericDataFile implements DataFile, IndexedRecord, StructLike, SpecificData.SchemaConstructable, Serializable { diff --git a/core/src/main/java/com/netflix/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/GenericManifestFile.java rename to core/src/main/java/org/apache/iceberg/GenericManifestFile.java index 628515bc98f7..1f75a261f6e2 100644 --- a/core/src/main/java/com/netflix/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.types.Types; +import java.io.Serializable; +import java.util.List; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData.SchemaConstructable; -import java.io.Serializable; -import java.util.List; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.types.Types; import static com.google.common.collect.ImmutableList.copyOf; import static com.google.common.collect.Iterables.transform; diff --git a/core/src/main/java/com/netflix/iceberg/GenericPartitionFieldSummary.java b/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/GenericPartitionFieldSummary.java rename to core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java index 0c57cb3cd7c4..d615abff4e03 100644 --- a/core/src/main/java/com/netflix/iceberg/GenericPartitionFieldSummary.java +++ b/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; -import com.netflix.iceberg.ManifestFile.PartitionFieldSummary; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.types.Types; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.specific.SpecificData.SchemaConstructable; import java.io.Serializable; import java.nio.ByteBuffer; import java.util.List; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.specific.SpecificData.SchemaConstructable; +import org.apache.iceberg.ManifestFile.PartitionFieldSummary; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Types; public class GenericPartitionFieldSummary implements PartitionFieldSummary, StructLike, IndexedRecord, SchemaConstructable, Serializable { diff --git a/core/src/main/java/com/netflix/iceberg/HasTableOperations.java b/core/src/main/java/org/apache/iceberg/HasTableOperations.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/HasTableOperations.java rename to core/src/main/java/org/apache/iceberg/HasTableOperations.java index 181049ca6613..d15cbcb93ab9 100644 --- a/core/src/main/java/com/netflix/iceberg/HasTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/HasTableOperations.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; /** * Used to expose a table's TableOperations. diff --git a/core/src/main/java/com/netflix/iceberg/LocationProviders.java b/core/src/main/java/org/apache/iceberg/LocationProviders.java similarity index 91% rename from core/src/main/java/com/netflix/iceberg/LocationProviders.java rename to core/src/main/java/org/apache/iceberg/LocationProviders.java index c20e8f53545f..ed6dcfc6f3b6 100644 --- a/core/src/main/java/com/netflix/iceberg/LocationProviders.java +++ b/core/src/main/java/org/apache/iceberg/LocationProviders.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; -import com.netflix.iceberg.io.LocationProvider; -import com.netflix.iceberg.transforms.Transform; -import com.netflix.iceberg.transforms.Transforms; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.PropertyUtil; -import org.apache.hadoop.fs.Path; import java.util.Map; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; -import static com.netflix.iceberg.TableProperties.OBJECT_STORE_PATH; +import static org.apache.iceberg.TableProperties.OBJECT_STORE_PATH; public class LocationProviders { diff --git a/core/src/main/java/com/netflix/iceberg/ManifestEntry.java b/core/src/main/java/org/apache/iceberg/ManifestEntry.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/ManifestEntry.java rename to core/src/main/java/org/apache/iceberg/ManifestEntry.java index 8a5d3dd62a03..d98e4bedf8ac 100644 --- a/core/src/main/java/com/netflix/iceberg/ManifestEntry.java +++ b/core/src/main/java/org/apache/iceberg/ManifestEntry.java @@ -17,19 +17,18 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.StructType; +import java.util.Collection; import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.StructType; -import java.util.Collection; - -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.required; class ManifestEntry implements IndexedRecord, SpecificData.SchemaConstructable{ enum Status { diff --git a/core/src/main/java/com/netflix/iceberg/ManifestGroup.java b/core/src/main/java/org/apache/iceberg/ManifestGroup.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/ManifestGroup.java rename to core/src/main/java/org/apache/iceberg/ManifestGroup.java index 2f86fdf69a24..5bc2bab66c52 100644 --- a/core/src/main/java/com/netflix/iceberg/ManifestGroup.java +++ b/core/src/main/java/org/apache/iceberg/ManifestGroup.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; @@ -26,16 +26,16 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.expressions.Evaluator; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.InclusiveManifestEvaluator; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.types.Types; import java.io.Closeable; import java.util.Arrays; import java.util.List; import java.util.Set; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.InclusiveManifestEvaluator; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; class ManifestGroup { private static final Types.StructType EMPTY_STRUCT = Types.StructType.of(); diff --git a/core/src/main/java/com/netflix/iceberg/ManifestListWriter.java b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java similarity index 91% rename from core/src/main/java/com/netflix/iceberg/ManifestListWriter.java rename to core/src/main/java/org/apache/iceberg/ManifestListWriter.java index d7ef491b5508..2d578e7420d9 100644 --- a/core/src/main/java/com/netflix/iceberg/ManifestListWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.OutputFile; import java.io.IOException; import java.util.Iterator; import java.util.Map; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.OutputFile; class ManifestListWriter implements FileAppender { private final FileAppender writer; diff --git a/core/src/main/java/com/netflix/iceberg/ManifestReader.java b/core/src/main/java/org/apache/iceberg/ManifestReader.java similarity index 92% rename from core/src/main/java/com/netflix/iceberg/ManifestReader.java rename to core/src/main/java/org/apache/iceberg/ManifestReader.java index 9baaf0c3e21b..6b428f0e2327 100644 --- a/core/src/main/java/com/netflix/iceberg/ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/ManifestReader.java @@ -17,30 +17,30 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.avro.AvroIterable; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Projections; -import com.netflix.iceberg.io.CloseableGroup; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.types.Types; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static com.netflix.iceberg.ManifestEntry.Status.DELETED; -import static com.netflix.iceberg.expressions.Expressions.alwaysTrue; +import static org.apache.iceberg.ManifestEntry.Status.DELETED; +import static org.apache.iceberg.expressions.Expressions.alwaysTrue; /** * Reader for manifest files. diff --git a/core/src/main/java/com/netflix/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java similarity index 94% rename from core/src/main/java/com/netflix/iceberg/ManifestWriter.java rename to core/src/main/java/org/apache/iceberg/ManifestWriter.java index 9ecaa81a7dc4..bad18cc90dd1 100644 --- a/core/src/main/java/com/netflix/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.OutputFile; +import java.io.IOException; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.OutputFile; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import static com.netflix.iceberg.ManifestEntry.Status.DELETED; +import static org.apache.iceberg.ManifestEntry.Status.DELETED; /** * Writer for manifest files. diff --git a/core/src/main/java/com/netflix/iceberg/MergeAppend.java b/core/src/main/java/org/apache/iceberg/MergeAppend.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/MergeAppend.java rename to core/src/main/java/org/apache/iceberg/MergeAppend.java index 8268afa521e5..5d992bb07c71 100644 --- a/core/src/main/java/com/netflix/iceberg/MergeAppend.java +++ b/core/src/main/java/org/apache/iceberg/MergeAppend.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitFailedException; /** * Append implementation that produces a minimal number of manifest files. diff --git a/core/src/main/java/com/netflix/iceberg/MergingSnapshotUpdate.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotUpdate.java similarity index 94% rename from core/src/main/java/com/netflix/iceberg/MergingSnapshotUpdate.java rename to core/src/main/java/org/apache/iceberg/MergingSnapshotUpdate.java index eb374665c2bc..3a4d4935a172 100644 --- a/core/src/main/java/com/netflix/iceberg/MergingSnapshotUpdate.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotUpdate.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; @@ -26,21 +26,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.ManifestEntry.Status; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.Evaluator; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.Projections; -import com.netflix.iceberg.expressions.StrictMetricsEvaluator; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.util.BinPacking.ListPacker; -import com.netflix.iceberg.util.CharSequenceWrapper; -import com.netflix.iceberg.util.StructLikeWrapper; -import com.netflix.iceberg.util.Tasks; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.lang.reflect.Array; import java.util.Comparator; @@ -48,14 +33,29 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.expressions.StrictMetricsEvaluator; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.util.BinPacking.ListPacker; +import org.apache.iceberg.util.CharSequenceWrapper; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.google.common.collect.Iterables.filter; import static com.google.common.collect.Iterables.transform; -import static com.netflix.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; -import static com.netflix.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT_DEFAULT; -import static com.netflix.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES; -import static com.netflix.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; -import static com.netflix.iceberg.util.ThreadPools.getWorkerPool; +import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; +import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT_DEFAULT; +import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; +import static org.apache.iceberg.util.ThreadPools.getWorkerPool; abstract class MergingSnapshotUpdate extends SnapshotUpdate { private final Logger LOG = LoggerFactory.getLogger(getClass()); diff --git a/core/src/main/java/com/netflix/iceberg/OverwriteData.java b/core/src/main/java/org/apache/iceberg/OverwriteData.java similarity index 89% rename from core/src/main/java/com/netflix/iceberg/OverwriteData.java rename to core/src/main/java/org/apache/iceberg/OverwriteData.java index f9a6abe651ee..f6ba7f3c6a2f 100644 --- a/core/src/main/java/com/netflix/iceberg/OverwriteData.java +++ b/core/src/main/java/org/apache/iceberg/OverwriteData.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.Evaluator; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Projections; -import com.netflix.iceberg.expressions.StrictMetricsEvaluator; import java.util.List; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.expressions.StrictMetricsEvaluator; public class OverwriteData extends MergingSnapshotUpdate implements OverwriteFiles { private boolean validateAddedFiles = false; diff --git a/core/src/main/java/com/netflix/iceberg/PartitionData.java b/core/src/main/java/org/apache/iceberg/PartitionData.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/PartitionData.java rename to core/src/main/java/org/apache/iceberg/PartitionData.java index f4a6cff93266..e638c7cf7bba 100644 --- a/core/src/main/java/com/netflix/iceberg/PartitionData.java +++ b/core/src/main/java/org/apache/iceberg/PartitionData.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; import com.google.common.base.Preconditions; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.io.Serializable; +import java.util.Arrays; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData; import org.apache.avro.util.Utf8; -import java.io.Serializable; -import java.util.Arrays; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; class PartitionData implements IndexedRecord, StructLike, SpecificData.SchemaConstructable, Serializable { diff --git a/core/src/main/java/com/netflix/iceberg/PartitionSpecParser.java b/core/src/main/java/org/apache/iceberg/PartitionSpecParser.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/PartitionSpecParser.java rename to core/src/main/java/org/apache/iceberg/PartitionSpecParser.java index 4df7c5516a80..e55103fac89d 100644 --- a/core/src/main/java/com/netflix/iceberg/PartitionSpecParser.java +++ b/core/src/main/java/org/apache/iceberg/PartitionSpecParser.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import com.google.common.base.Preconditions; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.JsonUtil; -import com.netflix.iceberg.util.Pair; import java.io.IOException; import java.io.StringWriter; import java.util.Iterator; import java.util.concurrent.ExecutionException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.Pair; public class PartitionSpecParser { private PartitionSpecParser() { diff --git a/core/src/main/java/com/netflix/iceberg/PartitionSummary.java b/core/src/main/java/org/apache/iceberg/PartitionSummary.java similarity index 91% rename from core/src/main/java/com/netflix/iceberg/PartitionSummary.java rename to core/src/main/java/org/apache/iceberg/PartitionSummary.java index 52a2b4a66de2..cc2f182928ee 100644 --- a/core/src/main/java/com/netflix/iceberg/PartitionSummary.java +++ b/core/src/main/java/org/apache/iceberg/PartitionSummary.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; -import com.netflix.iceberg.ManifestFile.PartitionFieldSummary; -import com.netflix.iceberg.types.Comparators; -import com.netflix.iceberg.types.Conversions; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; import java.util.Arrays; import java.util.Comparator; import java.util.List; +import org.apache.iceberg.ManifestFile.PartitionFieldSummary; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; class PartitionSummary { private final PartitionFieldStats[] fields; diff --git a/core/src/main/java/com/netflix/iceberg/PropertiesUpdate.java b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java similarity index 81% rename from core/src/main/java/com/netflix/iceberg/PropertiesUpdate.java rename to core/src/main/java/org/apache/iceberg/PropertiesUpdate.java index 2e74b2995feb..70a5ffcb9c90 100644 --- a/core/src/main/java/com/netflix/iceberg/PropertiesUpdate.java +++ b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java @@ -17,24 +17,24 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.util.Tasks; import java.util.Map; import java.util.Set; - -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.util.Tasks; + +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; class PropertiesUpdate implements UpdateProperties { private final TableOperations ops; diff --git a/core/src/main/java/com/netflix/iceberg/RemoveSnapshots.java b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java similarity index 89% rename from core/src/main/java/com/netflix/iceberg/RemoveSnapshots.java rename to core/src/main/java/org/apache/iceberg/RemoveSnapshots.java index 5c16e49146d0..e4ca10e76109 100644 --- a/core/src/main/java/com/netflix/iceberg/RemoveSnapshots.java +++ b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java @@ -17,32 +17,32 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Joiner; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.util.Tasks; -import com.netflix.iceberg.util.ThreadPools; import io.netty.util.internal.ConcurrentSet; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Date; import java.util.List; import java.util.Set; import java.util.function.Consumer; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; class RemoveSnapshots implements ExpireSnapshots { private static final Logger LOG = LoggerFactory.getLogger(RemoveSnapshots.class); diff --git a/core/src/main/java/com/netflix/iceberg/ReplaceFiles.java b/core/src/main/java/org/apache/iceberg/ReplaceFiles.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/ReplaceFiles.java rename to core/src/main/java/org/apache/iceberg/ReplaceFiles.java index 71ac06626f43..71779f21745b 100644 --- a/core/src/main/java/com/netflix/iceberg/ReplaceFiles.java +++ b/core/src/main/java/org/apache/iceberg/ReplaceFiles.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import java.util.Set; diff --git a/core/src/main/java/com/netflix/iceberg/ReplacePartitionsOperation.java b/core/src/main/java/org/apache/iceberg/ReplacePartitionsOperation.java similarity index 92% rename from core/src/main/java/com/netflix/iceberg/ReplacePartitionsOperation.java rename to core/src/main/java/org/apache/iceberg/ReplacePartitionsOperation.java index 360487edf3c9..fb21231e29cc 100644 --- a/core/src/main/java/com/netflix/iceberg/ReplacePartitionsOperation.java +++ b/core/src/main/java/org/apache/iceberg/ReplacePartitionsOperation.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.Expressions; import java.util.List; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; public class ReplacePartitionsOperation extends MergingSnapshotUpdate implements ReplacePartitions { ReplacePartitionsOperation(TableOperations ops) { diff --git a/core/src/main/java/com/netflix/iceberg/RollbackToSnapshot.java b/core/src/main/java/org/apache/iceberg/RollbackToSnapshot.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/RollbackToSnapshot.java rename to core/src/main/java/org/apache/iceberg/RollbackToSnapshot.java index b883c78c340d..191f113500c7 100644 --- a/core/src/main/java/com/netflix/iceberg/RollbackToSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/RollbackToSnapshot.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; -import com.netflix.iceberg.exceptions.ValidationException; +import org.apache.iceberg.exceptions.ValidationException; class RollbackToSnapshot implements Rollback { private final TableOperations ops; diff --git a/core/src/main/java/com/netflix/iceberg/ScanSummary.java b/core/src/main/java/org/apache/iceberg/ScanSummary.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/ScanSummary.java rename to core/src/main/java/org/apache/iceberg/ScanSummary.java index 909495c5902f..60e1f4567dee 100644 --- a/core/src/main/java/com/netflix/iceberg/ScanSummary.java +++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; @@ -26,18 +26,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.And; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expression.Operation; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.expressions.NamedReference; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.types.Comparators; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.Pair; import java.io.IOException; import java.util.Comparator; import java.util.Date; @@ -46,6 +34,18 @@ import java.util.Set; import java.util.TreeMap; import java.util.function.Function; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.And; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.expressions.NamedReference; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; public class ScanSummary { private ScanSummary() { diff --git a/core/src/main/java/com/netflix/iceberg/SchemaParser.java b/core/src/main/java/org/apache/iceberg/SchemaParser.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/SchemaParser.java rename to core/src/main/java/org/apache/iceberg/SchemaParser.java index 476459bbe7f6..4ae158a49e94 100644 --- a/core/src/main/java/com/netflix/iceberg/SchemaParser.java +++ b/core/src/main/java/org/apache/iceberg/SchemaParser.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; @@ -25,15 +25,15 @@ import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import com.google.common.collect.Lists; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.JsonUtil; import java.io.IOException; import java.io.StringWriter; import java.util.Iterator; import java.util.List; import java.util.concurrent.ExecutionException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; public class SchemaParser { private static final String TYPE = "type"; diff --git a/core/src/main/java/com/netflix/iceberg/SchemaUpdate.java b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/SchemaUpdate.java rename to core/src/main/java/org/apache/iceberg/SchemaUpdate.java index 4e7f290a5896..a5df2c9884d8 100644 --- a/core/src/main/java/com/netflix/iceberg/SchemaUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java @@ -17,23 +17,23 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; import com.google.common.collect.Multimaps; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; /** * Schema evolution API implementation. diff --git a/core/src/main/java/com/netflix/iceberg/SerializableByteBufferMap.java b/core/src/main/java/org/apache/iceberg/SerializableByteBufferMap.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/SerializableByteBufferMap.java rename to core/src/main/java/org/apache/iceberg/SerializableByteBufferMap.java index f67834ef429e..5adf57cd5c90 100644 --- a/core/src/main/java/com/netflix/iceberg/SerializableByteBufferMap.java +++ b/core/src/main/java/org/apache/iceberg/SerializableByteBufferMap.java @@ -17,17 +17,16 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Maps; -import com.netflix.iceberg.util.ByteBuffers; - import java.io.ObjectStreamException; import java.io.Serializable; import java.nio.ByteBuffer; import java.util.Collection; import java.util.Map; import java.util.Set; +import org.apache.iceberg.util.ByteBuffers; class SerializableByteBufferMap implements Map, Serializable { private final Map wrapped; diff --git a/core/src/main/java/com/netflix/iceberg/SetLocation.java b/core/src/main/java/org/apache/iceberg/SetLocation.java similarity index 71% rename from core/src/main/java/com/netflix/iceberg/SetLocation.java rename to core/src/main/java/org/apache/iceberg/SetLocation.java index be649715f23f..69b5b88bf35e 100644 --- a/core/src/main/java/com/netflix/iceberg/SetLocation.java +++ b/core/src/main/java/org/apache/iceberg/SetLocation.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.util.Tasks; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.util.Tasks; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; public class SetLocation implements UpdateLocation { private final TableOperations ops; diff --git a/core/src/main/java/com/netflix/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/SnapshotParser.java rename to core/src/main/java/org/apache/iceberg/SnapshotParser.java index 647f7f01ac62..7d1b0a146c9d 100644 --- a/core/src/main/java/com/netflix/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -17,20 +17,20 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.util.JsonUtil; import java.io.IOException; import java.io.StringWriter; import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.util.JsonUtil; public class SnapshotParser { diff --git a/core/src/main/java/com/netflix/iceberg/SnapshotSummary.java b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/SnapshotSummary.java rename to core/src/main/java/org/apache/iceberg/SnapshotSummary.java index 6dd223f7de3e..dee9f0206764 100644 --- a/core/src/main/java/com/netflix/iceberg/SnapshotSummary.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; diff --git a/core/src/main/java/com/netflix/iceberg/SnapshotUpdate.java b/core/src/main/java/org/apache/iceberg/SnapshotUpdate.java similarity index 91% rename from core/src/main/java/com/netflix/iceberg/SnapshotUpdate.java rename to core/src/main/java/org/apache/iceberg/SnapshotUpdate.java index 39e7a58039de..df5105b5d171 100644 --- a/core/src/main/java/com/netflix/iceberg/SnapshotUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotUpdate.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; @@ -25,13 +25,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.util.Exceptions; -import com.netflix.iceberg.util.Tasks; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Arrays; import java.util.List; @@ -40,18 +33,25 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.util.Exceptions; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.MANIFEST_LISTS_ENABLED; -import static com.netflix.iceberg.TableProperties.MANIFEST_LISTS_ENABLED_DEFAULT; -import static com.netflix.iceberg.util.ThreadPools.getWorkerPool; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.MANIFEST_LISTS_ENABLED; +import static org.apache.iceberg.TableProperties.MANIFEST_LISTS_ENABLED_DEFAULT; +import static org.apache.iceberg.util.ThreadPools.getWorkerPool; abstract class SnapshotUpdate implements PendingUpdate { private static final Logger LOG = LoggerFactory.getLogger(SnapshotUpdate.class); diff --git a/core/src/main/java/com/netflix/iceberg/StreamingDelete.java b/core/src/main/java/org/apache/iceberg/StreamingDelete.java similarity index 91% rename from core/src/main/java/com/netflix/iceberg/StreamingDelete.java rename to core/src/main/java/org/apache/iceberg/StreamingDelete.java index 8e3fab9fa296..3237d7b3f273 100644 --- a/core/src/main/java/com/netflix/iceberg/StreamingDelete.java +++ b/core/src/main/java/org/apache/iceberg/StreamingDelete.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.expressions.Expression; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.expressions.Expression; /** * {@link DeleteFiles Delete} implementation that avoids loading full manifests in memory. diff --git a/core/src/main/java/com/netflix/iceberg/SystemProperties.java b/core/src/main/java/org/apache/iceberg/SystemProperties.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/SystemProperties.java rename to core/src/main/java/org/apache/iceberg/SystemProperties.java index 8f2652b466f8..8ff3998f360c 100644 --- a/core/src/main/java/com/netflix/iceberg/SystemProperties.java +++ b/core/src/main/java/org/apache/iceberg/SystemProperties.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; /** * Configuration properties that are controlled by Java system properties. diff --git a/core/src/main/java/com/netflix/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/TableMetadata.java rename to core/src/main/java/org/apache/iceberg/TableMetadata.java index 42239735ee51..efae379ba19b 100644 --- a/core/src/main/java/com/netflix/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Objects; import com.google.common.base.Preconditions; @@ -27,15 +27,15 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.util.PropertyUtil; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PropertyUtil; /** * Metadata for a table. diff --git a/core/src/main/java/com/netflix/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/TableMetadataParser.java rename to core/src/main/java/org/apache/iceberg/TableMetadataParser.java index 0961d8c32456..fb4e37a4aee7 100644 --- a/core/src/main/java/com/netflix/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; @@ -25,15 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.TableMetadata.SnapshotLogEntry; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.util.JsonUtil; -import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; -import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; -import org.apache.hadoop.conf.Configuration; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStreamWriter; @@ -43,6 +34,14 @@ import java.util.List; import java.util.Map; import java.util.SortedSet; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.TableMetadata.SnapshotLogEntry; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.util.JsonUtil; public class TableMetadataParser { diff --git a/core/src/main/java/com/netflix/iceberg/TableOperations.java b/core/src/main/java/org/apache/iceberg/TableOperations.java similarity index 90% rename from core/src/main/java/com/netflix/iceberg/TableOperations.java rename to core/src/main/java/org/apache/iceberg/TableOperations.java index 03e275e4d8bb..8539059f6fcb 100644 --- a/core/src/main/java/com/netflix/iceberg/TableOperations.java +++ b/core/src/main/java/org/apache/iceberg/TableOperations.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.encryption.EncryptionManager; -import com.netflix.iceberg.encryption.PlaintextEncryptionManager; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.io.LocationProvider; import java.util.UUID; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; /** * SPI interface to abstract table metadata access and updates. @@ -64,7 +64,7 @@ public interface TableOperations { FileIO io(); /** - * @return a {@link com.netflix.iceberg.encryption.EncryptionManager} to encrypt and decrypt + * @return a {@link org.apache.iceberg.encryption.EncryptionManager} to encrypt and decrypt * data files. */ default EncryptionManager encryption() { diff --git a/core/src/main/java/com/netflix/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/TableProperties.java rename to core/src/main/java/org/apache/iceberg/TableProperties.java index 26128ee005bd..c35063b47c1e 100644 --- a/core/src/main/java/com/netflix/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; public class TableProperties { public static final String COMMIT_NUM_RETRIES = "commit.retry.num-retries"; diff --git a/core/src/main/java/com/netflix/iceberg/avro/Avro.java b/core/src/main/java/org/apache/iceberg/avro/Avro.java similarity index 91% rename from core/src/main/java/com/netflix/iceberg/avro/Avro.java rename to core/src/main/java/org/apache/iceberg/avro/Avro.java index 921321cccf9a..fae769eebb94 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/Avro.java +++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; -import com.netflix.iceberg.SchemaParser; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; +import java.io.IOException; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; import org.apache.avro.Conversions; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -33,13 +33,13 @@ import org.apache.avro.io.DatumReader; import org.apache.avro.io.DatumWriter; import org.apache.avro.specific.SpecificData; -import java.io.IOException; -import java.util.Locale; -import java.util.Map; -import java.util.function.Function; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; -import static com.netflix.iceberg.TableProperties.AVRO_COMPRESSION; -import static com.netflix.iceberg.TableProperties.AVRO_COMPRESSION_DEFAULT; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_DEFAULT; public class Avro { private Avro() { @@ -78,7 +78,7 @@ public static WriteBuilder write(OutputFile file) { public static class WriteBuilder { private final OutputFile file; - private com.netflix.iceberg.Schema schema = null; + private org.apache.iceberg.Schema schema = null; private String name = "table"; private Map config = Maps.newHashMap(); private Map metadata = Maps.newLinkedHashMap(); @@ -88,7 +88,7 @@ private WriteBuilder(OutputFile file) { this.file = file; } - public WriteBuilder schema(com.netflix.iceberg.Schema schema) { + public WriteBuilder schema(org.apache.iceberg.Schema schema) { this.schema = schema; return this; } @@ -153,7 +153,7 @@ public static class ReadBuilder { private final InputFile file; private final Map renames = Maps.newLinkedHashMap(); private boolean reuseContainers = false; - private com.netflix.iceberg.Schema schema = null; + private org.apache.iceberg.Schema schema = null; private Function> createReaderFunc = schema -> { GenericAvroReader reader = new GenericAvroReader<>(schema); reader.setClassLoader(defaultLoader); @@ -185,7 +185,7 @@ public ReadBuilder split(long start, long length) { return this; } - public ReadBuilder project(com.netflix.iceberg.Schema schema) { + public ReadBuilder project(org.apache.iceberg.Schema schema) { this.schema = schema; return this; } diff --git a/core/src/main/java/com/netflix/iceberg/avro/AvroCustomOrderSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/avro/AvroCustomOrderSchemaVisitor.java rename to core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java index 3896562c276a..3d6779614111 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/AvroCustomOrderSchemaVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import org.apache.avro.Schema; import java.util.LinkedList; import java.util.List; import java.util.function.Supplier; +import org.apache.avro.Schema; abstract class AvroCustomOrderSchemaVisitor { public static T visit(Schema schema, AvroCustomOrderSchemaVisitor visitor) { diff --git a/core/src/main/java/com/netflix/iceberg/avro/AvroFileAppender.java b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java similarity index 92% rename from core/src/main/java/com/netflix/iceberg/avro/AvroFileAppender.java rename to core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java index b510180214df..91d12ff6c90c 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/AvroFileAppender.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.io.PositionOutputStream; +import java.io.IOException; +import java.util.Map; +import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileWriter; import org.apache.avro.io.DatumWriter; -import java.io.IOException; -import java.util.Map; -import java.util.function.Function; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; class AvroFileAppender implements FileAppender { private PositionOutputStream stream = null; diff --git a/core/src/main/java/com/netflix/iceberg/avro/AvroIO.java b/core/src/main/java/org/apache/iceberg/avro/AvroIO.java similarity index 93% rename from core/src/main/java/com/netflix/iceberg/avro/AvroIO.java rename to core/src/main/java/org/apache/iceberg/avro/AvroIO.java index 79962ebfc82d..3fc7652d0253 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/AvroIO.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroIO.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; -import com.netflix.iceberg.common.DynClasses; -import com.netflix.iceberg.common.DynConstructors; -import com.netflix.iceberg.io.DelegatingInputStream; -import com.netflix.iceberg.io.SeekableInputStream; -import org.apache.avro.file.SeekableInput; import java.io.IOException; import java.io.InputStream; +import org.apache.avro.file.SeekableInput; +import org.apache.iceberg.common.DynClasses; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.io.DelegatingInputStream; +import org.apache.iceberg.io.SeekableInputStream; class AvroIO { private AvroIO() { diff --git a/core/src/main/java/com/netflix/iceberg/avro/AvroIterable.java b/core/src/main/java/org/apache/iceberg/avro/AvroIterable.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/avro/AvroIterable.java rename to core/src/main/java/org/apache/iceberg/avro/AvroIterable.java index 4324fd740359..6a388067c11c 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/AvroIterable.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroIterable.java @@ -17,22 +17,22 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Maps; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.CloseableGroup; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.InputFile; -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.file.FileReader; -import org.apache.avro.io.DatumReader; import java.io.Closeable; import java.io.IOException; import java.util.Iterator; import java.util.Map; import java.util.NoSuchElementException; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.FileReader; +import org.apache.avro.io.DatumReader; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; public class AvroIterable extends CloseableGroup implements CloseableIterable { private final InputFile file; diff --git a/core/src/main/java/com/netflix/iceberg/avro/AvroSchemaUtil.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/avro/AvroSchemaUtil.java rename to core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java index 68d4549af803..7d94f1c1f0c4 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/AvroSchemaUtil.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.avro.JsonProperties; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; -import java.util.List; -import java.util.Map; -import java.util.Set; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import static org.apache.avro.Schema.Type.ARRAY; import static org.apache.avro.Schema.Type.MAP; @@ -47,12 +47,12 @@ public class AvroSchemaUtil { private static final Schema NULL = Schema.create(Schema.Type.NULL); - public static Schema convert(com.netflix.iceberg.Schema schema, + public static Schema convert(org.apache.iceberg.Schema schema, String tableName) { return convert(schema, ImmutableMap.of(schema.asStruct(), tableName)); } - public static Schema convert(com.netflix.iceberg.Schema schema, + public static Schema convert(org.apache.iceberg.Schema schema, Map names) { return TypeUtil.visit(schema, new TypeToSchema(names)); } @@ -83,7 +83,7 @@ public static Schema pruneColumns(Schema schema, Set selectedIds) { return new PruneColumns(selectedIds).rootSchema(schema); } - public static Schema buildAvroProjection(Schema schema, com.netflix.iceberg.Schema expected, + public static Schema buildAvroProjection(Schema schema, org.apache.iceberg.Schema expected, Map renames) { return AvroCustomOrderSchemaVisitor.visit(schema, new BuildAvroProjection(expected, renames)); } diff --git a/core/src/main/java/com/netflix/iceberg/avro/AvroSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/avro/AvroSchemaVisitor.java rename to core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java index b4a8a3aa2d25..c2011b5cae7f 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/AvroSchemaVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import org.apache.avro.Schema; import java.util.LinkedList; import java.util.List; +import org.apache.avro.Schema; public abstract class AvroSchemaVisitor { public static T visit(Schema schema, AvroSchemaVisitor visitor) { diff --git a/core/src/main/java/com/netflix/iceberg/avro/BuildAvroProjection.java b/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java similarity index 90% rename from core/src/main/java/com/netflix/iceberg/avro/BuildAvroProjection.java rename to core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java index 8c90d3fdf113..b3d7f9fbfb3a 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/BuildAvroProjection.java +++ b/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java @@ -17,30 +17,28 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; -import org.apache.avro.JsonProperties; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import java.util.List; import java.util.Map; import java.util.function.Supplier; - -import static com.netflix.iceberg.avro.AvroSchemaUtil.convert; -import static com.netflix.iceberg.avro.AvroSchemaUtil.copyField; -import static com.netflix.iceberg.avro.AvroSchemaUtil.copyRecord; -import static com.netflix.iceberg.avro.AvroSchemaUtil.fromOption; -import static com.netflix.iceberg.avro.AvroSchemaUtil.fromOptions; -import static com.netflix.iceberg.avro.AvroSchemaUtil.getFieldId; -import static com.netflix.iceberg.avro.AvroSchemaUtil.isKeyValueSchema; -import static com.netflix.iceberg.avro.AvroSchemaUtil.isOptionSchema; -import static com.netflix.iceberg.avro.AvroSchemaUtil.toOption; +import org.apache.avro.JsonProperties; +import org.apache.avro.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +import static org.apache.iceberg.avro.AvroSchemaUtil.convert; +import static org.apache.iceberg.avro.AvroSchemaUtil.copyField; +import static org.apache.iceberg.avro.AvroSchemaUtil.copyRecord; +import static org.apache.iceberg.avro.AvroSchemaUtil.fromOption; +import static org.apache.iceberg.avro.AvroSchemaUtil.fromOptions; +import static org.apache.iceberg.avro.AvroSchemaUtil.getFieldId; +import static org.apache.iceberg.avro.AvroSchemaUtil.isKeyValueSchema; +import static org.apache.iceberg.avro.AvroSchemaUtil.isOptionSchema; +import static org.apache.iceberg.avro.AvroSchemaUtil.toOption; /** * Renames and aliases fields in an Avro schema based on the current table schema. @@ -54,7 +52,7 @@ class BuildAvroProjection extends AvroCustomOrderSchemaVisitor renames; private Type current = null; - BuildAvroProjection(com.netflix.iceberg.Schema expectedSchema, Map renames) { + BuildAvroProjection(org.apache.iceberg.Schema expectedSchema, Map renames) { this.renames = renames; this.current = expectedSchema.asStruct(); } diff --git a/core/src/main/java/com/netflix/iceberg/avro/GenericAvroReader.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/avro/GenericAvroReader.java rename to core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java index 399f5b63a7e4..468af47ebbee 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/GenericAvroReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java @@ -17,11 +17,13 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.MapMaker; -import com.netflix.iceberg.common.DynClasses; -import com.netflix.iceberg.exceptions.RuntimeIOException; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -30,10 +32,8 @@ import org.apache.avro.io.Decoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.ResolvingDecoder; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import org.apache.iceberg.common.DynClasses; +import org.apache.iceberg.exceptions.RuntimeIOException; class GenericAvroReader implements DatumReader { diff --git a/core/src/main/java/com/netflix/iceberg/avro/GenericAvroWriter.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroWriter.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/avro/GenericAvroWriter.java rename to core/src/main/java/org/apache/iceberg/avro/GenericAvroWriter.java index c70f5aed3667..c99220dbeac7 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/GenericAvroWriter.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroWriter.java @@ -17,19 +17,18 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; +import java.io.IOException; +import java.util.List; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.io.DatumWriter; import org.apache.avro.io.Encoder; -import java.io.IOException; -import java.util.List; - -import static com.netflix.iceberg.avro.AvroSchemaVisitor.visit; +import static org.apache.iceberg.avro.AvroSchemaVisitor.visit; class GenericAvroWriter implements DatumWriter { private ValueWriter writer = null; diff --git a/core/src/main/java/com/netflix/iceberg/avro/LogicalMap.java b/core/src/main/java/org/apache/iceberg/avro/LogicalMap.java similarity index 90% rename from core/src/main/java/com/netflix/iceberg/avro/LogicalMap.java rename to core/src/main/java/org/apache/iceberg/avro/LogicalMap.java index b0da080da2df..f650dacde294 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/LogicalMap.java +++ b/core/src/main/java/org/apache/iceberg/avro/LogicalMap.java @@ -17,17 +17,12 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; -import com.google.common.collect.Maps; -import org.apache.avro.Conversion; import org.apache.avro.LogicalType; import org.apache.avro.Schema; -import java.util.Collection; -import java.util.Map; - import static org.apache.avro.Schema.Type.ARRAY; public class LogicalMap extends LogicalType { diff --git a/core/src/main/java/com/netflix/iceberg/avro/ProjectionDatumReader.java b/core/src/main/java/org/apache/iceberg/avro/ProjectionDatumReader.java similarity index 90% rename from core/src/main/java/com/netflix/iceberg/avro/ProjectionDatumReader.java rename to core/src/main/java/org/apache/iceberg/avro/ProjectionDatumReader.java index 267f24c4f382..a68ebcdda6d1 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/ProjectionDatumReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/ProjectionDatumReader.java @@ -17,28 +17,28 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; -import org.apache.avro.Schema; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.Decoder; import java.io.IOException; import java.util.Map; import java.util.Set; import java.util.function.Function; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; -import static com.netflix.iceberg.types.TypeUtil.getProjectedIds; +import static org.apache.iceberg.types.TypeUtil.getProjectedIds; public class ProjectionDatumReader implements DatumReader { private final Function> getReader; - private final com.netflix.iceberg.Schema expectedSchema; + private final org.apache.iceberg.Schema expectedSchema; private final Map renames; private Schema readSchema = null; private Schema fileSchema = null; private DatumReader wrapped = null; public ProjectionDatumReader(Function> getReader, - com.netflix.iceberg.Schema expectedSchema, + org.apache.iceberg.Schema expectedSchema, Map renames) { this.getReader = getReader; this.expectedSchema = expectedSchema; diff --git a/core/src/main/java/com/netflix/iceberg/avro/PruneColumns.java b/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/avro/PruneColumns.java rename to core/src/main/java/org/apache/iceberg/avro/PruneColumns.java index d612aa0acda5..f5c7e251b860 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/PruneColumns.java +++ b/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java @@ -17,20 +17,20 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.avro.Schema; import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.avro.Schema; -import static com.netflix.iceberg.avro.AvroSchemaUtil.getElementId; -import static com.netflix.iceberg.avro.AvroSchemaUtil.getFieldId; -import static com.netflix.iceberg.avro.AvroSchemaUtil.getKeyId; -import static com.netflix.iceberg.avro.AvroSchemaUtil.getValueId; +import static org.apache.iceberg.avro.AvroSchemaUtil.getElementId; +import static org.apache.iceberg.avro.AvroSchemaUtil.getFieldId; +import static org.apache.iceberg.avro.AvroSchemaUtil.getKeyId; +import static org.apache.iceberg.avro.AvroSchemaUtil.getValueId; class PruneColumns extends AvroSchemaVisitor { private final Set selectedIds; diff --git a/core/src/main/java/com/netflix/iceberg/avro/SchemaToType.java b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/avro/SchemaToType.java rename to core/src/main/java/org/apache/iceberg/avro/SchemaToType.java index 3804c970d2db..9856132e419b 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/SchemaToType.java +++ b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.util.List; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; -import java.util.List; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; class SchemaToType extends AvroSchemaVisitor { private final Schema root; diff --git a/core/src/main/java/com/netflix/iceberg/avro/TypeToSchema.java b/core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/avro/TypeToSchema.java rename to core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java index 085caa8479d1..109f45620b7b 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/TypeToSchema.java +++ b/core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java @@ -17,20 +17,20 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; import java.util.List; import java.util.Map; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; -import static com.netflix.iceberg.avro.AvroSchemaUtil.toOption; import static org.apache.avro.JsonProperties.NULL_VALUE; +import static org.apache.iceberg.avro.AvroSchemaUtil.toOption; class TypeToSchema extends TypeUtil.SchemaVisitor { private static final Schema BOOLEAN_SCHEMA = Schema.create(Schema.Type.BOOLEAN); @@ -68,7 +68,7 @@ Map getConversionMap() { } @Override - public Schema schema(com.netflix.iceberg.Schema schema, Schema structSchema) { + public Schema schema(org.apache.iceberg.Schema schema, Schema structSchema) { return structSchema; } diff --git a/core/src/main/java/com/netflix/iceberg/avro/UUIDConversion.java b/core/src/main/java/org/apache/iceberg/avro/UUIDConversion.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/avro/UUIDConversion.java rename to core/src/main/java/org/apache/iceberg/avro/UUIDConversion.java index 4fcb16d4c538..b772effd1508 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/UUIDConversion.java +++ b/core/src/main/java/org/apache/iceberg/avro/UUIDConversion.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.UUID; import org.apache.avro.Conversion; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.UUID; public class UUIDConversion extends Conversion { @Override diff --git a/core/src/main/java/com/netflix/iceberg/avro/ValueReader.java b/core/src/main/java/org/apache/iceberg/avro/ValueReader.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/avro/ValueReader.java rename to core/src/main/java/org/apache/iceberg/avro/ValueReader.java index c8b5e34d650d..2264dc896ca6 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/ValueReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueReader.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; -import org.apache.avro.io.Decoder; import java.io.IOException; +import org.apache.avro.io.Decoder; public interface ValueReader { T read(Decoder decoder, Object reuse) throws IOException; diff --git a/core/src/main/java/com/netflix/iceberg/avro/ValueReaders.java b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/avro/ValueReaders.java rename to core/src/main/java/org/apache/iceberg/avro/ValueReaders.java index 15fc9cb2ff52..20672964e3c2 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/ValueReaders.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java @@ -17,17 +17,10 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.common.DynConstructors; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.io.Decoder; -import org.apache.avro.io.ResolvingDecoder; -import org.apache.avro.util.Utf8; import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; @@ -39,6 +32,13 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.ResolvingDecoder; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.common.DynConstructors; import static java.util.Collections.emptyIterator; diff --git a/core/src/main/java/com/netflix/iceberg/avro/ValueWriter.java b/core/src/main/java/org/apache/iceberg/avro/ValueWriter.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/avro/ValueWriter.java rename to core/src/main/java/org/apache/iceberg/avro/ValueWriter.java index 62cf9ca8277a..1753adfdae1f 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/ValueWriter.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueWriter.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; -import org.apache.avro.io.Encoder; import java.io.IOException; +import org.apache.avro.io.Encoder; public interface ValueWriter { void write(D datum, Encoder encoder) throws IOException; diff --git a/core/src/main/java/com/netflix/iceberg/avro/ValueWriters.java b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/avro/ValueWriters.java rename to core/src/main/java/org/apache/iceberg/avro/ValueWriters.java index 576425f54b8a..b2416e68200b 100644 --- a/core/src/main/java/com/netflix/iceberg/avro/ValueWriters.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java @@ -17,14 +17,9 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.base.Preconditions; -import com.netflix.iceberg.types.TypeUtil; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.io.Encoder; -import org.apache.avro.util.Utf8; import java.io.IOException; import java.lang.reflect.Array; import java.math.BigDecimal; @@ -35,6 +30,11 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.Encoder; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.types.TypeUtil; public class ValueWriters { private ValueWriters() { diff --git a/core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptedInputFile.java b/core/src/main/java/org/apache/iceberg/encryption/BaseEncryptedInputFile.java similarity index 94% rename from core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptedInputFile.java rename to core/src/main/java/org/apache/iceberg/encryption/BaseEncryptedInputFile.java index 540abe3a6ac0..d905347efa3f 100644 --- a/core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptedInputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/BaseEncryptedInputFile.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; -import com.netflix.iceberg.io.InputFile; +import org.apache.iceberg.io.InputFile; class BaseEncryptedInputFile implements EncryptedInputFile { diff --git a/core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptedOutputFile.java b/core/src/main/java/org/apache/iceberg/encryption/BaseEncryptedOutputFile.java similarity index 94% rename from core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptedOutputFile.java rename to core/src/main/java/org/apache/iceberg/encryption/BaseEncryptedOutputFile.java index 9cb8d063f991..ab758b2c7171 100644 --- a/core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptedOutputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/BaseEncryptedOutputFile.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; -import com.netflix.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFile; class BaseEncryptedOutputFile implements EncryptedOutputFile { diff --git a/core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptionKeyMetadata.java b/core/src/main/java/org/apache/iceberg/encryption/BaseEncryptionKeyMetadata.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptionKeyMetadata.java rename to core/src/main/java/org/apache/iceberg/encryption/BaseEncryptionKeyMetadata.java index 00e581c39c55..598edf274f60 100644 --- a/core/src/main/java/com/netflix/iceberg/encryption/BaseEncryptionKeyMetadata.java +++ b/core/src/main/java/org/apache/iceberg/encryption/BaseEncryptionKeyMetadata.java @@ -17,11 +17,10 @@ * under the License. */ -package com.netflix.iceberg.encryption; - -import com.netflix.iceberg.util.ByteBuffers; +package org.apache.iceberg.encryption; import java.nio.ByteBuffer; +import org.apache.iceberg.util.ByteBuffers; class BaseEncryptionKeyMetadata implements EncryptionKeyMetadata { diff --git a/core/src/main/java/com/netflix/iceberg/encryption/EncryptedFiles.java b/core/src/main/java/org/apache/iceberg/encryption/EncryptedFiles.java similarity index 94% rename from core/src/main/java/com/netflix/iceberg/encryption/EncryptedFiles.java rename to core/src/main/java/org/apache/iceberg/encryption/EncryptedFiles.java index 84a01597d474..1e4915453e14 100644 --- a/core/src/main/java/com/netflix/iceberg/encryption/EncryptedFiles.java +++ b/core/src/main/java/org/apache/iceberg/encryption/EncryptedFiles.java @@ -17,12 +17,11 @@ * under the License. */ -package com.netflix.iceberg.encryption; - -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; +package org.apache.iceberg.encryption; import java.nio.ByteBuffer; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; public class EncryptedFiles { diff --git a/core/src/main/java/com/netflix/iceberg/encryption/EncryptionKeyMetadatas.java b/core/src/main/java/org/apache/iceberg/encryption/EncryptionKeyMetadatas.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/encryption/EncryptionKeyMetadatas.java rename to core/src/main/java/org/apache/iceberg/encryption/EncryptionKeyMetadatas.java index 07554e6eac04..201546eca0ed 100644 --- a/core/src/main/java/com/netflix/iceberg/encryption/EncryptionKeyMetadatas.java +++ b/core/src/main/java/org/apache/iceberg/encryption/EncryptionKeyMetadatas.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; import java.nio.ByteBuffer; diff --git a/core/src/main/java/com/netflix/iceberg/encryption/PlaintextEncryptionManager.java b/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java similarity index 92% rename from core/src/main/java/com/netflix/iceberg/encryption/PlaintextEncryptionManager.java rename to core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java index 6b1a78b7938c..59ac871ca878 100644 --- a/core/src/main/java/com/netflix/iceberg/encryption/PlaintextEncryptionManager.java +++ b/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java @@ -17,15 +17,14 @@ * under the License. */ -package com.netflix.iceberg.encryption; +package org.apache.iceberg.encryption; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; +import java.nio.ByteBuffer; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; - public class PlaintextEncryptionManager implements EncryptionManager { private static final Logger LOG = LoggerFactory.getLogger(PlaintextEncryptionManager.class); diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopFileIO.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java similarity index 81% rename from core/src/main/java/com/netflix/iceberg/hadoop/HadoopFileIO.java rename to core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java index 7e1d004813c8..26c54e135a8f 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopFileIO.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java @@ -1,14 +1,13 @@ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; - -import java.io.IOException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; public class HadoopFileIO implements FileIO { diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopInputFile.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/hadoop/HadoopInputFile.java rename to core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java index 7f5b975f850b..9f18bd724f30 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopInputFile.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.SeekableInputStream; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import java.io.IOException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.SeekableInputStream; /** * {@link InputFile} implementation using the Hadoop {@link FileSystem} API. diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopOutputFile.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopOutputFile.java similarity index 89% rename from core/src/main/java/com/netflix/iceberg/hadoop/HadoopOutputFile.java rename to core/src/main/java/org/apache/iceberg/hadoop/HadoopOutputFile.java index 2cf23ce22331..2952ddb63932 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopOutputFile.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopOutputFile.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; -import com.netflix.iceberg.exceptions.AlreadyExistsException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.io.PositionOutputStream; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import java.io.IOException; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; /** * {@link OutputFile} implementation using the Hadoop {@link FileSystem} API. diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopStreams.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/hadoop/HadoopStreams.java rename to core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java index d05345d40646..c8436ae30050 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopStreams.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java @@ -17,23 +17,22 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; import com.google.common.base.Joiner; -import com.netflix.iceberg.io.DelegatingInputStream; -import com.netflix.iceberg.io.DelegatingOutputStream; -import com.netflix.iceberg.io.PositionOutputStream; -import com.netflix.iceberg.io.SeekableInputStream; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.Arrays; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.iceberg.io.DelegatingInputStream; +import org.apache.iceberg.io.DelegatingOutputStream; +import org.apache.iceberg.io.PositionOutputStream; +import org.apache.iceberg.io.SeekableInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Convenience methods to get Parquet abstractions for Hadoop data streams. diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopTableOperations.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java similarity index 91% rename from core/src/main/java/com/netflix/iceberg/hadoop/HadoopTableOperations.java rename to core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java index 1f784a7cabca..9cd5dff0b9de 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -17,32 +17,32 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; import com.google.common.base.Preconditions; -import com.netflix.iceberg.LocationProviders; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.TableMetadata; -import com.netflix.iceberg.TableMetadataParser; -import com.netflix.iceberg.TableOperations; -import com.netflix.iceberg.TableProperties; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.io.LocationProvider; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.LocationProviders; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.util.UUID; -import static com.netflix.iceberg.TableMetadataParser.getFileExtension; +import static org.apache.iceberg.TableMetadataParser.getFileExtension; /** * TableOperations implementation for file systems that support atomic rename. diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopTables.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java similarity index 85% rename from core/src/main/java/com/netflix/iceberg/hadoop/HadoopTables.java rename to core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java index 8c142c0866ce..69f9e5132a70 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/HadoopTables.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java @@ -17,23 +17,23 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; -import com.netflix.iceberg.BaseTable; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableMetadata; -import com.netflix.iceberg.TableOperations; -import com.netflix.iceberg.Tables; -import com.netflix.iceberg.exceptions.AlreadyExistsException; -import com.netflix.iceberg.exceptions.NoSuchTableException; +import java.util.Map; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import java.util.Map; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.Tables; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; -import static com.netflix.iceberg.TableMetadata.newTableMetadata; +import static org.apache.iceberg.TableMetadata.newTableMetadata; /** * Implementation of Iceberg tables that uses the Hadoop FileSystem diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/SerializableConfiguration.java b/core/src/main/java/org/apache/iceberg/hadoop/SerializableConfiguration.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/hadoop/SerializableConfiguration.java rename to core/src/main/java/org/apache/iceberg/hadoop/SerializableConfiguration.java index 30c756378d4e..37b9601474ce 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/SerializableConfiguration.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/SerializableConfiguration.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; import java.io.IOException; import java.io.ObjectInputStream; diff --git a/core/src/main/java/com/netflix/iceberg/hadoop/Util.java b/core/src/main/java/org/apache/iceberg/hadoop/Util.java similarity index 92% rename from core/src/main/java/com/netflix/iceberg/hadoop/Util.java rename to core/src/main/java/org/apache/iceberg/hadoop/Util.java index 70aee4f2d4db..d482ae635ca6 100644 --- a/core/src/main/java/com/netflix/iceberg/hadoop/Util.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/Util.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; -import com.netflix.iceberg.exceptions.RuntimeIOException; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import java.io.IOException; +import org.apache.iceberg.exceptions.RuntimeIOException; class Util { private Util() { diff --git a/core/src/main/java/com/netflix/iceberg/util/BinPacking.java b/core/src/main/java/org/apache/iceberg/util/BinPacking.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/util/BinPacking.java rename to core/src/main/java/org/apache/iceberg/util/BinPacking.java index a99cea7ea935..d99674f289ea 100644 --- a/core/src/main/java/com/netflix/iceberg/util/BinPacking.java +++ b/core/src/main/java/org/apache/iceberg/util/BinPacking.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; diff --git a/core/src/main/java/com/netflix/iceberg/util/ByteBuffers.java b/core/src/main/java/org/apache/iceberg/util/ByteBuffers.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/util/ByteBuffers.java rename to core/src/main/java/org/apache/iceberg/util/ByteBuffers.java index 9b0bbe4f9f18..a56f988d26b2 100644 --- a/core/src/main/java/com/netflix/iceberg/util/ByteBuffers.java +++ b/core/src/main/java/org/apache/iceberg/util/ByteBuffers.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/core/src/main/java/com/netflix/iceberg/util/CharSequenceWrapper.java b/core/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java similarity index 95% rename from core/src/main/java/com/netflix/iceberg/util/CharSequenceWrapper.java rename to core/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java index 79016b70d54e..26e5b2897e96 100644 --- a/core/src/main/java/com/netflix/iceberg/util/CharSequenceWrapper.java +++ b/core/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; -import com.netflix.iceberg.types.Comparators; +import org.apache.iceberg.types.Comparators; /** * Wrapper class to adapt CharSequence for use in maps and sets. diff --git a/core/src/main/java/com/netflix/iceberg/util/ExceptionUtil.java b/core/src/main/java/org/apache/iceberg/util/ExceptionUtil.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/util/ExceptionUtil.java rename to core/src/main/java/org/apache/iceberg/util/ExceptionUtil.java index ccdf61a4685b..7f89dd69ac18 100644 --- a/core/src/main/java/com/netflix/iceberg/util/ExceptionUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/ExceptionUtil.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; public class ExceptionUtil { @SuppressWarnings("unchecked") diff --git a/core/src/main/java/com/netflix/iceberg/util/Exceptions.java b/core/src/main/java/org/apache/iceberg/util/Exceptions.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/util/Exceptions.java rename to core/src/main/java/org/apache/iceberg/util/Exceptions.java index 7eb49d88f927..2e223247cd61 100644 --- a/core/src/main/java/com/netflix/iceberg/util/Exceptions.java +++ b/core/src/main/java/org/apache/iceberg/util/Exceptions.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; public class Exceptions { private Exceptions() { diff --git a/core/src/main/java/com/netflix/iceberg/util/JsonUtil.java b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/util/JsonUtil.java rename to core/src/main/java/org/apache/iceberg/util/JsonUtil.java index 68115f9ee568..11cb6913a13b 100644 --- a/core/src/main/java/com/netflix/iceberg/util/JsonUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.databind.JsonNode; diff --git a/core/src/main/java/com/netflix/iceberg/util/Pair.java b/core/src/main/java/org/apache/iceberg/util/Pair.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/util/Pair.java rename to core/src/main/java/org/apache/iceberg/util/Pair.java index 25497aee6774..10e670bbb6f9 100644 --- a/core/src/main/java/com/netflix/iceberg/util/Pair.java +++ b/core/src/main/java/org/apache/iceberg/util/Pair.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import com.google.common.base.Objects; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.Lists; +import java.io.Serializable; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.avro.reflect.ReflectData; import org.apache.avro.specific.SpecificData; -import java.io.Serializable; public class Pair implements IndexedRecord, SpecificData.SchemaConstructable, Serializable { public static Pair of(X x, Y y) { diff --git a/core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java rename to core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index f2c7d3fd014d..a2fb875eab60 100644 --- a/core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import com.netflix.iceberg.io.CloseableGroup; import java.io.Closeable; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import org.apache.iceberg.io.CloseableGroup; public class ParallelIterable extends CloseableGroup implements Iterable { private final Iterable> iterables; diff --git a/core/src/main/java/com/netflix/iceberg/util/PropertyUtil.java b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java similarity index 98% rename from core/src/main/java/com/netflix/iceberg/util/PropertyUtil.java rename to core/src/main/java/org/apache/iceberg/util/PropertyUtil.java index bca16c437f30..becfb71b8c3e 100644 --- a/core/src/main/java/com/netflix/iceberg/util/PropertyUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import java.util.Map; diff --git a/core/src/main/java/com/netflix/iceberg/util/StructLikeWrapper.java b/core/src/main/java/org/apache/iceberg/util/StructLikeWrapper.java similarity index 96% rename from core/src/main/java/com/netflix/iceberg/util/StructLikeWrapper.java rename to core/src/main/java/org/apache/iceberg/util/StructLikeWrapper.java index 983a8f168a58..20b051857cf6 100644 --- a/core/src/main/java/com/netflix/iceberg/util/StructLikeWrapper.java +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeWrapper.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; -import com.netflix.iceberg.StructLike; import java.util.Objects; +import org.apache.iceberg.StructLike; /** * Wrapper to adapt StructLike for use in maps and sets by implementing equals and hashCode. diff --git a/core/src/main/java/com/netflix/iceberg/util/Tasks.java b/core/src/main/java/org/apache/iceberg/util/Tasks.java similarity index 99% rename from core/src/main/java/com/netflix/iceberg/util/Tasks.java rename to core/src/main/java/org/apache/iceberg/util/Tasks.java index c63f4d5beb68..6a47e5239661 100644 --- a/core/src/main/java/com/netflix/iceberg/util/Tasks.java +++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java @@ -17,11 +17,9 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -37,6 +35,8 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class Tasks { diff --git a/core/src/main/java/com/netflix/iceberg/util/ThreadPools.java b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java similarity index 97% rename from core/src/main/java/com/netflix/iceberg/util/ThreadPools.java rename to core/src/main/java/org/apache/iceberg/util/ThreadPools.java index cc977d335e7e..f0404822dd68 100644 --- a/core/src/main/java/com/netflix/iceberg/util/ThreadPools.java +++ b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.netflix.iceberg.SystemProperties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadPoolExecutor; +import org.apache.iceberg.SystemProperties; public class ThreadPools { public static final String WORKER_THREAD_POOL_SIZE_PROP = diff --git a/core/src/test/java/com/netflix/iceberg/AssertHelpers.java b/core/src/test/java/org/apache/iceberg/AssertHelpers.java similarity index 99% rename from core/src/test/java/com/netflix/iceberg/AssertHelpers.java rename to core/src/test/java/org/apache/iceberg/AssertHelpers.java index dce4ba110e45..e9754328e61d 100644 --- a/core/src/test/java/com/netflix/iceberg/AssertHelpers.java +++ b/core/src/test/java/org/apache/iceberg/AssertHelpers.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import org.junit.Assert; import java.util.concurrent.Callable; +import org.junit.Assert; public class AssertHelpers { /** diff --git a/core/src/test/java/com/netflix/iceberg/LocalTableOperations.java b/core/src/test/java/org/apache/iceberg/LocalTableOperations.java similarity index 92% rename from core/src/test/java/com/netflix/iceberg/LocalTableOperations.java rename to core/src/test/java/org/apache/iceberg/LocalTableOperations.java index e60bbaaa7269..2986e505ac84 100644 --- a/core/src/test/java/com/netflix/iceberg/LocalTableOperations.java +++ b/core/src/test/java/org/apache/iceberg/LocalTableOperations.java @@ -17,17 +17,16 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Maps; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileIO; +import java.io.IOException; import java.util.Map; -import com.netflix.iceberg.io.LocationProvider; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; import org.junit.rules.TemporaryFolder; -import java.io.IOException; - class LocalTableOperations implements TableOperations { private final TemporaryFolder temp; private final FileIO io; diff --git a/core/src/test/java/com/netflix/iceberg/TableMetadataParserTest.java b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java similarity index 87% rename from core/src/test/java/com/netflix/iceberg/TableMetadataParserTest.java rename to core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java index 8d3c2eea3112..2f709b119211 100644 --- a/core/src/test/java/com/netflix/iceberg/TableMetadataParserTest.java +++ b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java @@ -16,28 +16,27 @@ * specific language governing permissions and limitations * under the License. */ -package com.netflix.iceberg; - -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.types.Types.BooleanType; -import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; -import org.apache.hadoop.conf.Configuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +package org.apache.iceberg; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.file.Paths; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.types.Types.BooleanType; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; import static avro.shaded.com.google.common.collect.Lists.newArrayList; -import static com.netflix.iceberg.ConfigProperties.COMPRESS_METADATA; -import static com.netflix.iceberg.PartitionSpec.unpartitioned; -import static com.netflix.iceberg.TableMetadata.newTableMetadata; -import static com.netflix.iceberg.TableMetadataParser.getFileExtension; -import static com.netflix.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.ConfigProperties.COMPRESS_METADATA; +import static org.apache.iceberg.PartitionSpec.unpartitioned; +import static org.apache.iceberg.TableMetadata.newTableMetadata; +import static org.apache.iceberg.TableMetadataParser.getFileExtension; +import static org.apache.iceberg.types.Types.NestedField.optional; public class TableMetadataParserTest { diff --git a/core/src/test/java/com/netflix/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java similarity index 97% rename from core/src/test/java/com/netflix/iceberg/TableTestBase.java rename to core/src/test/java/org/apache/iceberg/TableTestBase.java index 010896c8ad5f..30d93b422a87 100644 --- a/core/src/test/java/com/netflix/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -17,24 +17,24 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.io.Files; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.types.Types; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.util.Iterator; -import java.util.List; -import static com.netflix.iceberg.Files.localInput; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.types.Types.NestedField.required; public class TableTestBase { // Schema passed to create tables diff --git a/core/src/test/java/com/netflix/iceberg/TestBaseTableScan.java b/core/src/test/java/org/apache/iceberg/TestBaseTableScan.java similarity index 95% rename from core/src/test/java/com/netflix/iceberg/TestBaseTableScan.java rename to core/src/test/java/org/apache/iceberg/TestBaseTableScan.java index ae0a841e74d1..034866e84782 100644 --- a/core/src/test/java/com/netflix/iceberg/TestBaseTableScan.java +++ b/core/src/test/java/org/apache/iceberg/TestBaseTableScan.java @@ -17,19 +17,18 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import org.apache.iceberg.types.Types; import org.junit.After; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; - -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.required; import static org.junit.Assert.assertEquals; public class TestBaseTableScan { diff --git a/core/src/test/java/com/netflix/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java similarity index 98% rename from core/src/test/java/com/netflix/iceberg/TestCreateTransaction.java rename to core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index ffdec59f5b38..fff40a334bcb 100644 --- a/core/src/test/java/com/netflix/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.types.TypeUtil; -import org.junit.Assert; -import org.junit.Test; import java.io.File; import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.types.TypeUtil; +import org.junit.Assert; +import org.junit.Test; -import static com.netflix.iceberg.PartitionSpec.unpartitioned; +import static org.apache.iceberg.PartitionSpec.unpartitioned; public class TestCreateTransaction extends TableTestBase { @Test diff --git a/core/src/test/java/com/netflix/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java similarity index 96% rename from core/src/test/java/com/netflix/iceberg/TestDeleteFiles.java rename to core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index 7bdc7c53c840..cb95d02357c4 100644 --- a/core/src/test/java/com/netflix/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -17,9 +17,9 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.ManifestEntry.Status; +import org.apache.iceberg.ManifestEntry.Status; import org.junit.Assert; import org.junit.Test; diff --git a/core/src/test/java/com/netflix/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java similarity index 98% rename from core/src/test/java/com/netflix/iceberg/TestFastAppend.java rename to core/src/test/java/org/apache/iceberg/TestFastAppend.java index d9ca6f26356c..f036a8543a12 100644 --- a/core/src/test/java/com/netflix/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.exceptions.CommitFailedException; -import org.junit.Assert; -import org.junit.Test; import java.io.File; import java.util.List; import java.util.Set; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.junit.Assert; +import org.junit.Test; public class TestFastAppend extends TableTestBase { diff --git a/core/src/test/java/com/netflix/iceberg/TestFilterFiles.java b/core/src/test/java/org/apache/iceberg/TestFilterFiles.java similarity index 95% rename from core/src/test/java/com/netflix/iceberg/TestFilterFiles.java rename to core/src/test/java/org/apache/iceberg/TestFilterFiles.java index 22b0ae511db2..be01515eb485 100644 --- a/core/src/test/java/com/netflix/iceberg/TestFilterFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestFilterFiles.java @@ -17,26 +17,25 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.types.Conversions; -import com.netflix.iceberg.types.Types; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; import org.junit.After; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.required; import static org.junit.Assert.assertEquals; public class TestFilterFiles { diff --git a/core/src/test/java/com/netflix/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java similarity index 98% rename from core/src/test/java/com/netflix/iceberg/TestMergeAppend.java rename to core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 3446c9750674..33b2b09cd9bd 100644 --- a/core/src/test/java/com/netflix/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.ManifestEntry.Status; -import com.netflix.iceberg.exceptions.CommitFailedException; -import org.junit.Assert; -import org.junit.Test; import java.io.File; import java.util.Set; +import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.junit.Assert; +import org.junit.Test; import static com.google.common.collect.Iterators.concat; diff --git a/core/src/test/java/com/netflix/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java similarity index 94% rename from core/src/test/java/com/netflix/iceberg/TestOverwrite.java rename to core/src/test/java/org/apache/iceberg/TestOverwrite.java index be21e75d2710..ff2f1e1dd973 100644 --- a/core/src/test/java/com/netflix/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -17,26 +17,25 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.ManifestEntry.Status; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Before; +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.lessThan; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestOverwrite extends TableTestBase { private static final Schema DATE_SCHEMA = new Schema( diff --git a/core/src/test/java/com/netflix/iceberg/TestReplaceFiles.java b/core/src/test/java/org/apache/iceberg/TestReplaceFiles.java similarity index 96% rename from core/src/test/java/com/netflix/iceberg/TestReplaceFiles.java rename to core/src/test/java/org/apache/iceberg/TestReplaceFiles.java index 3cc1d50400a6..17a5ff9bfc90 100644 --- a/core/src/test/java/com/netflix/iceberg/TestReplaceFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceFiles.java @@ -17,20 +17,19 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.ValidationException; +import java.io.File; +import java.util.Collections; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; import org.junit.Assert; import org.junit.Test; import org.mockito.internal.util.collections.Sets; -import java.io.File; -import java.util.Collections; - -import static com.netflix.iceberg.ManifestEntry.Status.ADDED; -import static com.netflix.iceberg.ManifestEntry.Status.DELETED; -import static com.netflix.iceberg.ManifestEntry.Status.EXISTING; +import static org.apache.iceberg.ManifestEntry.Status.ADDED; +import static org.apache.iceberg.ManifestEntry.Status.DELETED; +import static org.apache.iceberg.ManifestEntry.Status.EXISTING; public class TestReplaceFiles extends TableTestBase { diff --git a/core/src/test/java/com/netflix/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java similarity index 98% rename from core/src/test/java/com/netflix/iceberg/TestReplacePartitions.java rename to core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index 50a6a31fb522..1dc4d2519c53 100644 --- a/core/src/test/java/com/netflix/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import com.netflix.iceberg.ManifestEntry.Status; -import com.netflix.iceberg.exceptions.ValidationException; -import org.junit.Assert; -import org.junit.Test; import java.io.File; import java.io.IOException; +import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.exceptions.ValidationException; +import org.junit.Assert; +import org.junit.Test; public class TestReplacePartitions extends TableTestBase { diff --git a/core/src/test/java/com/netflix/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java similarity index 97% rename from core/src/test/java/com/netflix/iceberg/TestReplaceTransaction.java rename to core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java index 2c4e7d92e2ac..f038d36909a2 100644 --- a/core/src/test/java/com/netflix/iceberg/TestReplaceTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java @@ -17,20 +17,19 @@ * under the License. */ -package com.netflix.iceberg; - -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +package org.apache.iceberg; import java.io.File; import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; -import static com.netflix.iceberg.PartitionSpec.unpartitioned; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.PartitionSpec.unpartitioned; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestReplaceTransaction extends TableTestBase { @Test diff --git a/core/src/test/java/com/netflix/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java similarity index 85% rename from core/src/test/java/com/netflix/iceberg/TestScanSummary.java rename to core/src/test/java/org/apache/iceberg/TestScanSummary.java index b1b297345cff..2339187d45ab 100644 --- a/core/src/test/java/com/netflix/iceberg/TestScanSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java @@ -17,20 +17,20 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableList; -import com.netflix.iceberg.util.Pair; +import org.apache.iceberg.util.Pair; import org.junit.Assert; import org.junit.Test; -import static com.netflix.iceberg.ScanSummary.timestampRange; -import static com.netflix.iceberg.ScanSummary.toMillis; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.ScanSummary.timestampRange; +import static org.apache.iceberg.ScanSummary.toMillis; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; public class TestScanSummary { @Test diff --git a/core/src/test/java/com/netflix/iceberg/TestSchemaUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java similarity index 98% rename from core/src/test/java/com/netflix/iceberg/TestSchemaUpdate.java rename to core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java index d548bd559cf0..b1b9fc456944 100644 --- a/core/src/test/java/com/netflix/iceberg/TestSchemaUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java @@ -17,22 +17,22 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Test; import java.util.List; import java.util.Set; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +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; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestSchemaUpdate { private static final Schema SCHEMA = new Schema( diff --git a/core/src/test/java/com/netflix/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java similarity index 98% rename from core/src/test/java/com/netflix/iceberg/TestSnapshotJson.java rename to core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index d60247b9b09e..f4f99ef2ddb1 100644 --- a/core/src/test/java/com/netflix/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import java.io.File; +import java.io.IOException; +import java.util.List; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.List; -import static com.netflix.iceberg.Files.localInput; +import static org.apache.iceberg.Files.localInput; public class TestSnapshotJson { @Rule diff --git a/core/src/test/java/com/netflix/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java similarity index 96% rename from core/src/test/java/com/netflix/iceberg/TestSplitPlanning.java rename to core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index 11df14de6d35..2813becf5410 100644 --- a/core/src/test/java/com/netflix/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -17,24 +17,24 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.UUID; -import static com.netflix.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.optional; public class TestSplitPlanning { diff --git a/core/src/test/java/com/netflix/iceberg/TestSplitScanTaskIterator.java b/core/src/test/java/org/apache/iceberg/TestSplitScanTaskIterator.java similarity index 95% rename from core/src/test/java/com/netflix/iceberg/TestSplitScanTaskIterator.java rename to core/src/test/java/org/apache/iceberg/TestSplitScanTaskIterator.java index cee02b5c87bf..9964058f4e2f 100644 --- a/core/src/test/java/com/netflix/iceberg/TestSplitScanTaskIterator.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitScanTaskIterator.java @@ -17,13 +17,14 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; +import java.util.List; import org.junit.Assert; import org.junit.Test; -import java.util.List; -import static com.netflix.iceberg.BaseFileScanTask.SplitScanTaskIterator; + +import static org.apache.iceberg.BaseFileScanTask.SplitScanTaskIterator; public class TestSplitScanTaskIterator { @Test diff --git a/core/src/test/java/com/netflix/iceberg/TestTableMetadataJson.java b/core/src/test/java/org/apache/iceberg/TestTableMetadataJson.java similarity index 93% rename from core/src/test/java/com/netflix/iceberg/TestTableMetadataJson.java rename to core/src/test/java/org/apache/iceberg/TestTableMetadataJson.java index 1b54390aa870..00f24bd3509d 100644 --- a/core/src/test/java/com/netflix/iceberg/TestTableMetadataJson.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadataJson.java @@ -17,38 +17,38 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.netflix.iceberg.TableMetadata.SnapshotLogEntry; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.JsonUtil; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; import java.io.IOException; import java.io.StringWriter; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Random; +import org.apache.iceberg.TableMetadata.SnapshotLogEntry; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; -import static com.netflix.iceberg.Files.localInput; -import static com.netflix.iceberg.TableMetadataParser.CURRENT_SNAPSHOT_ID; -import static com.netflix.iceberg.TableMetadataParser.FORMAT_VERSION; -import static com.netflix.iceberg.TableMetadataParser.LAST_COLUMN_ID; -import static com.netflix.iceberg.TableMetadataParser.LAST_UPDATED_MILLIS; -import static com.netflix.iceberg.TableMetadataParser.LOCATION; -import static com.netflix.iceberg.TableMetadataParser.PARTITION_SPEC; -import static com.netflix.iceberg.TableMetadataParser.PROPERTIES; -import static com.netflix.iceberg.TableMetadataParser.SCHEMA; -import static com.netflix.iceberg.TableMetadataParser.SNAPSHOTS; +import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.TableMetadataParser.CURRENT_SNAPSHOT_ID; +import static org.apache.iceberg.TableMetadataParser.FORMAT_VERSION; +import static org.apache.iceberg.TableMetadataParser.LAST_COLUMN_ID; +import static org.apache.iceberg.TableMetadataParser.LAST_UPDATED_MILLIS; +import static org.apache.iceberg.TableMetadataParser.LOCATION; +import static org.apache.iceberg.TableMetadataParser.PARTITION_SPEC; +import static org.apache.iceberg.TableMetadataParser.PROPERTIES; +import static org.apache.iceberg.TableMetadataParser.SCHEMA; +import static org.apache.iceberg.TableMetadataParser.SNAPSHOTS; public class TestTableMetadataJson { @Rule diff --git a/core/src/test/java/com/netflix/iceberg/TestTables.java b/core/src/test/java/org/apache/iceberg/TestTables.java similarity index 93% rename from core/src/test/java/com/netflix/iceberg/TestTables.java rename to core/src/test/java/org/apache/iceberg/TestTables.java index a0048e384bed..96a6ced2adbc 100644 --- a/core/src/test/java/com/netflix/iceberg/TestTables.java +++ b/core/src/test/java/org/apache/iceberg/TestTables.java @@ -17,22 +17,22 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; -import com.netflix.iceberg.exceptions.AlreadyExistsException; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.LocationProvider; -import com.netflix.iceberg.io.OutputFile; import java.io.File; import java.util.Map; - -import static com.netflix.iceberg.TableMetadata.newTableMetadata; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; + +import static org.apache.iceberg.TableMetadata.newTableMetadata; public class TestTables { static TestTable create(File temp, String name, Schema schema, PartitionSpec spec) { diff --git a/core/src/test/java/com/netflix/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java similarity index 99% rename from core/src/test/java/com/netflix/iceberg/TestTransaction.java rename to core/src/test/java/org/apache/iceberg/TestTransaction.java index bdc3ddd9d3de..5f2fbd9fa1ba 100644 --- a/core/src/test/java/com/netflix/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Sets; -import com.netflix.iceberg.ManifestEntry.Status; -import com.netflix.iceberg.exceptions.CommitFailedException; -import org.junit.Assert; -import org.junit.Test; import java.io.File; import java.util.Set; +import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.junit.Assert; +import org.junit.Test; public class TestTransaction extends TableTestBase { @Test diff --git a/core/src/test/java/com/netflix/iceberg/avro/AvroDataTest.java b/core/src/test/java/org/apache/iceberg/avro/AvroDataTest.java similarity index 92% rename from core/src/test/java/com/netflix/iceberg/avro/AvroDataTest.java rename to core/src/test/java/org/apache/iceberg/avro/AvroDataTest.java index d2c2d7ffe68e..e4513238ce89 100644 --- a/core/src/test/java/com/netflix/iceberg/avro/AvroDataTest.java +++ b/core/src/test/java/org/apache/iceberg/avro/AvroDataTest.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg.avro; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.ListType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.MapType; -import com.netflix.iceberg.types.Types.StructType; +package org.apache.iceberg.avro; + +import java.io.IOException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StructType; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.IOException; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public abstract class AvroDataTest { diff --git a/core/src/test/java/com/netflix/iceberg/avro/AvroTestHelpers.java b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java similarity index 96% rename from core/src/test/java/com/netflix/iceberg/avro/AvroTestHelpers.java rename to core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java index c12912fef3af..d4e60bdec4fd 100644 --- a/core/src/test/java/com/netflix/iceberg/avro/AvroTestHelpers.java +++ b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java @@ -17,19 +17,19 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.avro.JsonProperties; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.junit.Assert; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import static com.netflix.iceberg.avro.AvroSchemaUtil.toOption; +import static org.apache.iceberg.avro.AvroSchemaUtil.toOption; class AvroTestHelpers { static Schema.Field optionalField(int id, String name, Schema schema) { diff --git a/core/src/test/java/com/netflix/iceberg/avro/RandomAvroData.java b/core/src/test/java/org/apache/iceberg/avro/RandomAvroData.java similarity index 97% rename from core/src/test/java/com/netflix/iceberg/avro/RandomAvroData.java rename to core/src/test/java/org/apache/iceberg/avro/RandomAvroData.java index 1b0a2be612b5..a686cfa5e323 100644 --- a/core/src/test/java/com/netflix/iceberg/avro/RandomAvroData.java +++ b/core/src/test/java/org/apache/iceberg/avro/RandomAvroData.java @@ -17,18 +17,11 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericData.Record; -import org.apache.avro.util.Utf8; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -38,6 +31,13 @@ import java.util.Set; import java.util.UUID; import java.util.function.Supplier; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; public class RandomAvroData { public static List generate(Schema schema, int numRecords, long seed) { diff --git a/core/src/test/java/com/netflix/iceberg/avro/TestAvroReadProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroReadProjection.java similarity index 92% rename from core/src/test/java/com/netflix/iceberg/avro/TestAvroReadProjection.java rename to core/src/test/java/org/apache/iceberg/avro/TestAvroReadProjection.java index b3dde55d6bae..94b3e58cd67f 100644 --- a/core/src/test/java/com/netflix/iceberg/avro/TestAvroReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroReadProjection.java @@ -17,15 +17,15 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Iterables; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.io.FileAppender; -import org.apache.avro.generic.GenericData; import java.io.File; import java.io.IOException; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; public class TestAvroReadProjection extends TestReadProjection { protected GenericData.Record writeAndRead(String desc, diff --git a/core/src/test/java/com/netflix/iceberg/avro/TestGenericAvro.java b/core/src/test/java/org/apache/iceberg/avro/TestGenericAvro.java similarity index 92% rename from core/src/test/java/com/netflix/iceberg/avro/TestGenericAvro.java rename to core/src/test/java/org/apache/iceberg/avro/TestGenericAvro.java index a416a32b938b..bea553a43c9e 100644 --- a/core/src/test/java/com/netflix/iceberg/avro/TestGenericAvro.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestGenericAvro.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Lists; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.io.FileAppender; -import org.apache.avro.generic.GenericData.Record; -import org.junit.Assert; import java.io.File; import java.io.IOException; import java.util.List; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; +import org.junit.Assert; public class TestGenericAvro extends AvroDataTest { protected void writeAndValidate(Schema schema) throws IOException { diff --git a/core/src/test/java/com/netflix/iceberg/avro/TestReadProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java similarity index 99% rename from core/src/test/java/com/netflix/iceberg/avro/TestReadProjection.java rename to core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java index 7ef7fe0bde08..b0d18d0ce910 100644 --- a/core/src/test/java/com/netflix/iceberg/avro/TestReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -17,23 +17,23 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Comparators; -import com.netflix.iceberg.types.Types; +import java.io.IOException; +import java.util.List; +import java.util.Map; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.IOException; -import java.util.List; -import java.util.Map; public abstract class TestReadProjection { protected abstract Record writeAndRead(String desc, diff --git a/core/src/test/java/com/netflix/iceberg/avro/TestSchemaConversions.java b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java similarity index 93% rename from core/src/test/java/com/netflix/iceberg/avro/TestSchemaConversions.java rename to core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java index 341a0c910bad..4462e0152ea8 100644 --- a/core/src/test/java/com/netflix/iceberg/avro/TestSchemaConversions.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestSchemaConversions.java @@ -17,26 +17,26 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Lists; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.util.List; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; -import java.util.List; -import static com.netflix.iceberg.avro.AvroTestHelpers.addElementId; -import static com.netflix.iceberg.avro.AvroTestHelpers.addKeyId; -import static com.netflix.iceberg.avro.AvroTestHelpers.addValueId; -import static com.netflix.iceberg.avro.AvroTestHelpers.optionalField; -import static com.netflix.iceberg.avro.AvroTestHelpers.record; -import static com.netflix.iceberg.avro.AvroTestHelpers.requiredField; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.avro.AvroTestHelpers.addElementId; +import static org.apache.iceberg.avro.AvroTestHelpers.addKeyId; +import static org.apache.iceberg.avro.AvroTestHelpers.addValueId; +import static org.apache.iceberg.avro.AvroTestHelpers.optionalField; +import static org.apache.iceberg.avro.AvroTestHelpers.record; +import static org.apache.iceberg.avro.AvroTestHelpers.requiredField; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestSchemaConversions { @Test @@ -227,7 +227,7 @@ public void testMapOfStringToStructs() { @Test public void testComplexSchema() { - com.netflix.iceberg.Schema schema = new com.netflix.iceberg.Schema( + org.apache.iceberg.Schema schema = new org.apache.iceberg.Schema( required(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get()), optional( diff --git a/core/src/test/java/com/netflix/iceberg/hadoop/HadoopTableTestBase.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java similarity index 88% rename from core/src/test/java/com/netflix/iceberg/hadoop/HadoopTableTestBase.java rename to core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java index 4d854d7651b0..70d740176bc3 100644 --- a/core/src/test/java/com/netflix/iceberg/hadoop/HadoopTableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java @@ -17,32 +17,32 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; import com.google.common.base.Charsets; import com.google.common.collect.Lists; import com.google.common.io.Files; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableMetadata; -import com.netflix.iceberg.TableMetadataParser; -import com.netflix.iceberg.TestTables; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.types.Types; import org.junit.Before; import org.junit.Rule; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.List; -import static com.netflix.iceberg.Files.localInput; -import static com.netflix.iceberg.TableMetadataParser.getFileExtension; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.TableMetadataParser.getFileExtension; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class HadoopTableTestBase { // Schema passed to create tables diff --git a/core/src/test/java/com/netflix/iceberg/hadoop/TestHadoopCommits.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java similarity index 95% rename from core/src/test/java/com/netflix/iceberg/hadoop/TestHadoopCommits.java rename to core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java index bc24f4bf55e7..f3d1b624f88b 100644 --- a/core/src/test/java/com/netflix/iceberg/hadoop/TestHadoopCommits.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java @@ -17,25 +17,25 @@ * under the License. */ -package com.netflix.iceberg.hadoop; +package org.apache.iceberg.hadoop; import com.google.common.collect.Lists; -import com.netflix.iceberg.AssertHelpers; -import com.netflix.iceberg.FileScanTask; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableMetadata; -import com.netflix.iceberg.UpdateSchema; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; import java.io.File; import java.util.List; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.types.Types; +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; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestHadoopCommits extends HadoopTableTestBase { diff --git a/core/src/test/java/com/netflix/iceberg/util/TestBinPacking.java b/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java similarity index 98% rename from core/src/test/java/com/netflix/iceberg/util/TestBinPacking.java rename to core/src/test/java/org/apache/iceberg/util/TestBinPacking.java index c283afb9d2ec..6877167927f3 100644 --- a/core/src/test/java/com/netflix/iceberg/util/TestBinPacking.java +++ b/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.util; +package org.apache.iceberg.util; import com.google.common.collect.Lists; -import com.netflix.iceberg.util.BinPacking.ListPacker; +import java.util.List; +import org.apache.iceberg.util.BinPacking.ListPacker; import org.junit.Assert; import org.junit.Test; -import java.util.List; public class TestBinPacking { @Test diff --git a/data/src/main/java/com/netflix/iceberg/data/GenericRecord.java b/data/src/main/java/org/apache/iceberg/data/GenericRecord.java similarity index 96% rename from data/src/main/java/com/netflix/iceberg/data/GenericRecord.java rename to data/src/main/java/org/apache/iceberg/data/GenericRecord.java index 60f2b0657c30..14d5fe86e249 100644 --- a/data/src/main/java/com/netflix/iceberg/data/GenericRecord.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericRecord.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; import com.google.common.base.Objects; import com.google.common.base.Preconditions; @@ -25,13 +25,13 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.StructLike; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; import java.util.Arrays; import java.util.List; import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.StructType; public class GenericRecord implements Record, StructLike { private static final LoadingCache> NAME_MAP_CACHE = diff --git a/data/src/main/java/com/netflix/iceberg/data/IcebergGenerics.java b/data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java similarity index 93% rename from data/src/main/java/com/netflix/iceberg/data/IcebergGenerics.java rename to data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java index 941da26f4c79..9e784afe0d6d 100644 --- a/data/src/main/java/com/netflix/iceberg/data/IcebergGenerics.java +++ b/data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; import com.google.common.collect.ImmutableList; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expressions; import java.util.List; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; public class IcebergGenerics { private IcebergGenerics() { diff --git a/data/src/main/java/com/netflix/iceberg/data/Record.java b/data/src/main/java/org/apache/iceberg/data/Record.java similarity index 89% rename from data/src/main/java/com/netflix/iceberg/data/Record.java rename to data/src/main/java/org/apache/iceberg/data/Record.java index 0ad5e4d6b795..2ee57780bb08 100644 --- a/data/src/main/java/com/netflix/iceberg/data/Record.java +++ b/data/src/main/java/org/apache/iceberg/data/Record.java @@ -17,11 +17,11 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; -import com.netflix.iceberg.StructLike; -import com.netflix.iceberg.types.Types.StructType; import java.util.Map; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types.StructType; public interface Record extends StructLike { StructType struct(); diff --git a/data/src/main/java/com/netflix/iceberg/data/TableScanIterable.java b/data/src/main/java/org/apache/iceberg/data/TableScanIterable.java similarity index 78% rename from data/src/main/java/com/netflix/iceberg/data/TableScanIterable.java rename to data/src/main/java/org/apache/iceberg/data/TableScanIterable.java index b48b057dfc98..4f749357b7d9 100644 --- a/data/src/main/java/com/netflix/iceberg/data/TableScanIterable.java +++ b/data/src/main/java/org/apache/iceberg/data/TableScanIterable.java @@ -17,42 +17,34 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.netflix.iceberg.CombinedScanTask; -import com.netflix.iceberg.FileScanTask; -import com.netflix.iceberg.HasTableOperations; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TableOperations; -import com.netflix.iceberg.TableScan; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.data.avro.DataReader; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Binder; -import com.netflix.iceberg.expressions.Evaluator; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.io.CloseableGroup; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.types.TypeUtil; - import java.io.Closeable; import java.io.IOException; -import java.util.Collections; import java.util.Iterator; -import java.util.List; import java.util.NoSuchElementException; -import java.util.Set; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.parquet.Parquet; -import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.filter; -import static com.google.common.collect.Iterables.transform; -import static com.netflix.iceberg.data.parquet.GenericParquetReaders.buildReader; import static java.util.Collections.emptyIterator; +import static org.apache.iceberg.data.parquet.GenericParquetReaders.buildReader; class TableScanIterable extends CloseableGroup implements CloseableIterable { private final TableOperations ops; @@ -127,8 +119,9 @@ private class ScanIterator implements Iterator, Closeable { private Closeable currentCloseable = null; private Iterator currentIterator = emptyIterator(); - private ScanIterator(Iterable tasks, boolean caseSensitive) { - this.tasks = Lists.newArrayList(concat(transform(tasks, CombinedScanTask::files))).iterator(); + private ScanIterator(CloseableIterable tasks, boolean caseSensitive) { + this.tasks = Lists.newArrayList(Iterables.concat( + CloseableIterable.transform(tasks, CombinedScanTask::files))).iterator(); this.caseSensitive = caseSensitive; } diff --git a/data/src/main/java/com/netflix/iceberg/data/avro/DataReader.java b/data/src/main/java/org/apache/iceberg/data/avro/DataReader.java similarity index 94% rename from data/src/main/java/com/netflix/iceberg/data/avro/DataReader.java rename to data/src/main/java/org/apache/iceberg/data/avro/DataReader.java index fee76883954d..d340ded0254f 100644 --- a/data/src/main/java/com/netflix/iceberg/data/avro/DataReader.java +++ b/data/src/main/java/org/apache/iceberg/data/avro/DataReader.java @@ -17,15 +17,13 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; import com.google.common.collect.MapMaker; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.avro.AvroSchemaVisitor; -import com.netflix.iceberg.avro.LogicalMap; -import com.netflix.iceberg.avro.ValueReader; -import com.netflix.iceberg.avro.ValueReaders; -import com.netflix.iceberg.exceptions.RuntimeIOException; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -33,10 +31,12 @@ import org.apache.avro.io.Decoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.ResolvingDecoder; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.AvroSchemaVisitor; +import org.apache.iceberg.avro.LogicalMap; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.exceptions.RuntimeIOException; public class DataReader implements DatumReader { diff --git a/data/src/main/java/com/netflix/iceberg/data/avro/DataWriter.java b/data/src/main/java/org/apache/iceberg/data/avro/DataWriter.java similarity index 93% rename from data/src/main/java/com/netflix/iceberg/data/avro/DataWriter.java rename to data/src/main/java/org/apache/iceberg/data/avro/DataWriter.java index cfb47793f183..543e4b4d14b2 100644 --- a/data/src/main/java/com/netflix/iceberg/data/avro/DataWriter.java +++ b/data/src/main/java/org/apache/iceberg/data/avro/DataWriter.java @@ -17,23 +17,23 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; import com.google.common.base.Preconditions; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.avro.AvroSchemaVisitor; -import com.netflix.iceberg.avro.LogicalMap; -import com.netflix.iceberg.avro.ValueWriter; -import com.netflix.iceberg.avro.ValueWriters; +import java.io.IOException; +import java.util.List; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.io.DatumWriter; import org.apache.avro.io.Encoder; -import java.io.IOException; -import java.util.List; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.AvroSchemaVisitor; +import org.apache.iceberg.avro.LogicalMap; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; -import static com.netflix.iceberg.avro.AvroSchemaVisitor.visit; +import static org.apache.iceberg.avro.AvroSchemaVisitor.visit; public class DataWriter implements DatumWriter { private ValueWriter writer = null; diff --git a/data/src/main/java/com/netflix/iceberg/data/avro/GenericReaders.java b/data/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java similarity index 93% rename from data/src/main/java/com/netflix/iceberg/data/avro/GenericReaders.java rename to data/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java index c875cc501fbe..1eceeda6c12e 100644 --- a/data/src/main/java/com/netflix/iceberg/data/avro/GenericReaders.java +++ b/data/src/main/java/org/apache/iceberg/data/avro/GenericReaders.java @@ -17,14 +17,8 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; -import com.netflix.iceberg.avro.ValueReader; -import com.netflix.iceberg.avro.ValueReaders; -import com.netflix.iceberg.data.GenericRecord; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.types.Types.StructType; -import org.apache.avro.io.Decoder; import java.io.IOException; import java.time.Instant; import java.time.LocalDate; @@ -34,6 +28,12 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.List; +import org.apache.avro.io.Decoder; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types.StructType; class GenericReaders { private GenericReaders() { diff --git a/data/src/main/java/com/netflix/iceberg/data/avro/GenericWriters.java b/data/src/main/java/org/apache/iceberg/data/avro/GenericWriters.java similarity index 95% rename from data/src/main/java/com/netflix/iceberg/data/avro/GenericWriters.java rename to data/src/main/java/org/apache/iceberg/data/avro/GenericWriters.java index 4805ba261ca1..f8638946ccaf 100644 --- a/data/src/main/java/com/netflix/iceberg/data/avro/GenericWriters.java +++ b/data/src/main/java/org/apache/iceberg/data/avro/GenericWriters.java @@ -17,12 +17,8 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; -import com.netflix.iceberg.avro.ValueWriter; -import com.netflix.iceberg.avro.ValueWriters; -import com.netflix.iceberg.data.Record; -import org.apache.avro.io.Encoder; import java.io.IOException; import java.time.Instant; import java.time.LocalDate; @@ -32,6 +28,10 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.List; +import org.apache.avro.io.Encoder; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; +import org.apache.iceberg.data.Record; class GenericWriters { private GenericWriters() { diff --git a/data/src/main/java/com/netflix/iceberg/data/avro/IcebergDecoder.java b/data/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java similarity index 89% rename from data/src/main/java/com/netflix/iceberg/data/avro/IcebergDecoder.java rename to data/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java index a002030d7a60..94285d8bd619 100644 --- a/data/src/main/java/com/netflix/iceberg/data/avro/IcebergDecoder.java +++ b/data/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java @@ -17,12 +17,15 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; import com.google.common.collect.ImmutableMap; import com.google.common.collect.MapMaker; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.avro.ProjectionDatumReader; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; import org.apache.avro.SchemaNormalization; @@ -33,11 +36,8 @@ import org.apache.avro.message.MessageDecoder; import org.apache.avro.message.MissingSchemaException; import org.apache.avro.message.SchemaStore; -import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Map; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.ProjectionDatumReader; public class IcebergDecoder extends MessageDecoder.BaseDecoder { private static final ThreadLocal HEADER_BUFFER = @@ -49,13 +49,13 @@ public class IcebergDecoder extends MessageDecoder.BaseDecoder { return ByteBuffer.wrap(header).order(ByteOrder.LITTLE_ENDIAN); }); - private final com.netflix.iceberg.Schema readSchema; + private final org.apache.iceberg.Schema readSchema; private final SchemaStore resolver; private final Map> decoders = new MapMaker().makeMap(); /** * Creates a new decoder that constructs datum instances described by an - * {@link com.netflix.iceberg.Schema Iceberg schema}. + * {@link org.apache.iceberg.Schema Iceberg schema}. *

* The {@code readSchema} is as used the expected schema (read schema). Datum instances created * by this class will are described by the expected schema. @@ -63,17 +63,17 @@ public class IcebergDecoder extends MessageDecoder.BaseDecoder { * The schema used to decode incoming buffers is determined by the schema fingerprint encoded in * the message header. This class can decode messages that were encoded using the * {@code readSchema} and other schemas that are added using - * {@link #addSchema(com.netflix.iceberg.Schema)}. + * {@link #addSchema(org.apache.iceberg.Schema)}. * * @param readSchema the schema used to construct datum instances */ - public IcebergDecoder(com.netflix.iceberg.Schema readSchema) { + public IcebergDecoder(org.apache.iceberg.Schema readSchema) { this(readSchema, null); } /** * Creates a new decoder that constructs datum instances described by an - * {@link com.netflix.iceberg.Schema Iceberg schema}. + * {@link org.apache.iceberg.Schema Iceberg schema}. *

* The {@code readSchema} is as used the expected schema (read schema). Datum instances created * by this class will are described by the expected schema. @@ -81,7 +81,7 @@ public IcebergDecoder(com.netflix.iceberg.Schema readSchema) { * The schema used to decode incoming buffers is determined by the schema fingerprint encoded in * the message header. This class can decode messages that were encoded using the * {@code readSchema} and other schemas that are added using - * {@link #addSchema(com.netflix.iceberg.Schema)}. + * {@link #addSchema(org.apache.iceberg.Schema)}. *

* Schemas may also be returned from an Avro {@link SchemaStore}. Avro Schemas from the store * must be compatible with Iceberg and should contain id properties and use only Iceberg types. @@ -89,18 +89,18 @@ public IcebergDecoder(com.netflix.iceberg.Schema readSchema) { * @param readSchema the {@link Schema} used to construct datum instances * @param resolver a {@link SchemaStore} used to find schemas by fingerprint */ - public IcebergDecoder(com.netflix.iceberg.Schema readSchema, SchemaStore resolver) { + public IcebergDecoder(org.apache.iceberg.Schema readSchema, SchemaStore resolver) { this.readSchema = readSchema; this.resolver = resolver; addSchema(this.readSchema); } /** - * Adds an {@link com.netflix.iceberg.Schema Iceberg schema} that can be used to decode buffers. + * Adds an {@link org.apache.iceberg.Schema Iceberg schema} that can be used to decode buffers. * * @param writeSchema a schema to use when decoding buffers */ - public void addSchema(com.netflix.iceberg.Schema writeSchema) { + public void addSchema(org.apache.iceberg.Schema writeSchema) { addSchema(AvroSchemaUtil.convert(writeSchema, "table")); } @@ -165,7 +165,7 @@ private static class RawDecoder extends MessageDecoder.BaseDecoder { * @param readSchema the schema used to construct datum instances * @param writeSchema the schema used to decode buffers */ - private RawDecoder(com.netflix.iceberg.Schema readSchema, org.apache.avro.Schema writeSchema) { + private RawDecoder(org.apache.iceberg.Schema readSchema, org.apache.avro.Schema writeSchema) { this.reader = new ProjectionDatumReader<>(DataReader::create, readSchema, ImmutableMap.of()); this.reader.setSchema(writeSchema); } diff --git a/data/src/main/java/com/netflix/iceberg/data/avro/IcebergEncoder.java b/data/src/main/java/org/apache/iceberg/data/avro/IcebergEncoder.java similarity index 97% rename from data/src/main/java/com/netflix/iceberg/data/avro/IcebergEncoder.java rename to data/src/main/java/org/apache/iceberg/data/avro/IcebergEncoder.java index 369cd8d3ce21..eb5ff1e6a90f 100644 --- a/data/src/main/java/com/netflix/iceberg/data/avro/IcebergEncoder.java +++ b/data/src/main/java/org/apache/iceberg/data/avro/IcebergEncoder.java @@ -16,22 +16,22 @@ * specific language governing permissions and limitations * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; import com.google.common.primitives.Bytes; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.AvroSchemaUtil; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.security.NoSuchAlgorithmException; import org.apache.avro.AvroRuntimeException; import org.apache.avro.SchemaNormalization; import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.DatumWriter; import org.apache.avro.io.EncoderFactory; import org.apache.avro.message.MessageEncoder; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.security.NoSuchAlgorithmException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; public class IcebergEncoder implements MessageEncoder { diff --git a/data/src/main/java/com/netflix/iceberg/data/parquet/GenericParquetReaders.java b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java similarity index 90% rename from data/src/main/java/com/netflix/iceberg/data/parquet/GenericParquetReaders.java rename to data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java index aa07a74c0c03..6257694282cb 100644 --- a/data/src/main/java/com/netflix/iceberg/data/parquet/GenericParquetReaders.java +++ b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetReaders.java @@ -17,39 +17,11 @@ * under the License. */ -package com.netflix.iceberg.data.parquet; +package org.apache.iceberg.data.parquet; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.data.GenericRecord; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.parquet.ParquetValueReader; -import com.netflix.iceberg.parquet.ParquetValueReaders; -import com.netflix.iceberg.parquet.ParquetValueReaders.BinaryAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.BytesReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.IntAsLongReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.IntegerAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.ListReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.LongAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.MapReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.PrimitiveReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.StringReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.StructReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.UnboxedReader; -import com.netflix.iceberg.parquet.TypeWithSchemaVisitor; -import com.netflix.iceberg.types.Type.TypeID; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; -import com.netflix.iceberg.types.Types.TimestampType; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.schema.DecimalMetadata; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -59,9 +31,36 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetValueReaders.BinaryAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.BytesReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntAsLongReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntegerAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.ListReader; +import org.apache.iceberg.parquet.ParquetValueReaders.LongAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.MapReader; +import org.apache.iceberg.parquet.ParquetValueReaders.PrimitiveReader; +import org.apache.iceberg.parquet.ParquetValueReaders.StringReader; +import org.apache.iceberg.parquet.ParquetValueReaders.StructReader; +import org.apache.iceberg.parquet.ParquetValueReaders.UnboxedReader; +import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.types.Types.TimestampType; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.DecimalMetadata; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.hasIds; -import static com.netflix.iceberg.parquet.ParquetValueReaders.option; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.hasIds; +import static org.apache.iceberg.parquet.ParquetValueReaders.option; public class GenericParquetReaders { private GenericParquetReaders() { @@ -195,7 +194,7 @@ public ParquetValueReader map(Types.MapType expectedMap, GroupType map, } @Override - public ParquetValueReader primitive(com.netflix.iceberg.types.Type.PrimitiveType expected, + public ParquetValueReader primitive(org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); diff --git a/data/src/main/java/com/netflix/iceberg/data/parquet/GenericParquetWriter.java b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java similarity index 90% rename from data/src/main/java/com/netflix/iceberg/data/parquet/GenericParquetWriter.java rename to data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java index 646b3bc28cdb..f0e061a69f74 100644 --- a/data/src/main/java/com/netflix/iceberg/data/parquet/GenericParquetWriter.java +++ b/data/src/main/java/org/apache/iceberg/data/parquet/GenericParquetWriter.java @@ -17,21 +17,9 @@ * under the License. */ -package com.netflix.iceberg.data.parquet; +package org.apache.iceberg.data.parquet; import com.google.common.collect.Lists; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.parquet.ParquetTypeVisitor; -import com.netflix.iceberg.parquet.ParquetValueWriter; -import com.netflix.iceberg.parquet.ParquetValueWriters.PrimitiveWriter; -import com.netflix.iceberg.parquet.ParquetValueWriters.StructWriter; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.DecimalMetadata; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -41,16 +29,28 @@ import java.time.temporal.ChronoUnit; import java.util.Iterator; import java.util.List; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.parquet.ParquetTypeVisitor; +import org.apache.iceberg.parquet.ParquetValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters.PrimitiveWriter; +import org.apache.iceberg.parquet.ParquetValueWriters.StructWriter; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.DecimalMetadata; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; -import static com.netflix.iceberg.parquet.ParquetValueWriters.byteBuffers; -import static com.netflix.iceberg.parquet.ParquetValueWriters.collections; -import static com.netflix.iceberg.parquet.ParquetValueWriters.decimalAsFixed; -import static com.netflix.iceberg.parquet.ParquetValueWriters.decimalAsInteger; -import static com.netflix.iceberg.parquet.ParquetValueWriters.decimalAsLong; -import static com.netflix.iceberg.parquet.ParquetValueWriters.maps; -import static com.netflix.iceberg.parquet.ParquetValueWriters.option; -import static com.netflix.iceberg.parquet.ParquetValueWriters.strings; -import static com.netflix.iceberg.parquet.ParquetValueWriters.unboxed; +import static org.apache.iceberg.parquet.ParquetValueWriters.byteBuffers; +import static org.apache.iceberg.parquet.ParquetValueWriters.collections; +import static org.apache.iceberg.parquet.ParquetValueWriters.decimalAsFixed; +import static org.apache.iceberg.parquet.ParquetValueWriters.decimalAsInteger; +import static org.apache.iceberg.parquet.ParquetValueWriters.decimalAsLong; +import static org.apache.iceberg.parquet.ParquetValueWriters.maps; +import static org.apache.iceberg.parquet.ParquetValueWriters.option; +import static org.apache.iceberg.parquet.ParquetValueWriters.strings; +import static org.apache.iceberg.parquet.ParquetValueWriters.unboxed; public class GenericParquetWriter { private GenericParquetWriter() { diff --git a/data/src/test/java/com/netflix/iceberg/TestSplitScan.java b/data/src/test/java/org/apache/iceberg/TestSplitScan.java similarity index 87% rename from data/src/test/java/com/netflix/iceberg/TestSplitScan.java rename to data/src/test/java/org/apache/iceberg/TestSplitScan.java index 022c8ccf92e7..c0e35722819d 100644 --- a/data/src/test/java/com/netflix/iceberg/TestSplitScan.java +++ b/data/src/test/java/org/apache/iceberg/TestSplitScan.java @@ -1,17 +1,21 @@ -package com.netflix.iceberg; +package org.apache.iceberg; import com.google.common.collect.Lists; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.data.IcebergGenerics; -import com.netflix.iceberg.data.RandomGenericData; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.data.avro.DataWriter; -import com.netflix.iceberg.data.parquet.GenericParquetWriter; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Locale; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -19,12 +23,8 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Locale; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.required; @RunWith(Parameterized.class) public class TestSplitScan { diff --git a/data/src/test/java/com/netflix/iceberg/data/DataTest.java b/data/src/test/java/org/apache/iceberg/data/DataTest.java similarity index 92% rename from data/src/test/java/com/netflix/iceberg/data/DataTest.java rename to data/src/test/java/org/apache/iceberg/data/DataTest.java index a029bb6119f7..60058ae645d2 100644 --- a/data/src/test/java/com/netflix/iceberg/data/DataTest.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTest.java @@ -17,21 +17,21 @@ * under the License. */ -package com.netflix.iceberg.data; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.ListType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.MapType; -import com.netflix.iceberg.types.Types.StructType; +package org.apache.iceberg.data; + +import java.io.IOException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StructType; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.IOException; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public abstract class DataTest { diff --git a/data/src/test/java/com/netflix/iceberg/data/DataTestHelpers.java b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java similarity index 97% rename from data/src/test/java/com/netflix/iceberg/data/DataTestHelpers.java rename to data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java index 08c07611c5b5..bc92cfb1c7cd 100644 --- a/data/src/test/java/com/netflix/iceberg/data/DataTestHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; -import org.junit.Assert; import java.util.List; import java.util.Map; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.Assert; public class DataTestHelpers { public static void assertEquals(Types.StructType struct, Record expected, Record actual) { diff --git a/data/src/test/java/com/netflix/iceberg/data/RandomGenericData.java b/data/src/test/java/org/apache/iceberg/data/RandomGenericData.java similarity index 97% rename from data/src/test/java/com/netflix/iceberg/data/RandomGenericData.java rename to data/src/test/java/org/apache/iceberg/data/RandomGenericData.java index be4ec0cda05e..8c6668a27b1c 100644 --- a/data/src/test/java/com/netflix/iceberg/data/RandomGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/RandomGenericData.java @@ -17,16 +17,12 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; import com.google.common.base.Charsets; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -41,6 +37,10 @@ import java.util.Set; import java.util.UUID; import java.util.function.Supplier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import static java.time.temporal.ChronoUnit.MICROS; diff --git a/data/src/test/java/com/netflix/iceberg/data/TestLocalScan.java b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java similarity index 89% rename from data/src/test/java/com/netflix/iceberg/data/TestLocalScan.java rename to data/src/test/java/org/apache/iceberg/data/TestLocalScan.java index 2b7376d244cd..388e1cfc308d 100644 --- a/data/src/test/java/com/netflix/iceberg/data/TestLocalScan.java +++ b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java @@ -17,33 +17,41 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.AppendFiles; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableProperties; -import com.netflix.iceberg.Tables; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.data.avro.DataWriter; -import com.netflix.iceberg.data.parquet.GenericParquetWriter; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.hadoop.HadoopInputFile; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.Tables; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -51,24 +59,16 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Iterator; -import java.util.List; -import java.util.Locale; -import java.util.Set; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.filter; import static com.google.common.collect.Iterables.transform; -import static com.netflix.iceberg.DataFiles.fromInputFile; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.hadoop.HadoopOutputFile.fromPath; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.DataFiles.fromInputFile; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; @RunWith(Parameterized.class) public class TestLocalScan { diff --git a/data/src/test/java/com/netflix/iceberg/data/TestReadProjection.java b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java similarity index 99% rename from data/src/test/java/com/netflix/iceberg/data/TestReadProjection.java rename to data/src/test/java/org/apache/iceberg/data/TestReadProjection.java index c042ad408aab..54b76d8c8647 100644 --- a/data/src/test/java/com/netflix/iceberg/data/TestReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java @@ -17,22 +17,22 @@ * under the License. */ -package com.netflix.iceberg.data; +package org.apache.iceberg.data; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Comparators; -import com.netflix.iceberg.types.Types; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.IOException; -import java.util.List; -import java.util.Map; public abstract class TestReadProjection { protected abstract Record writeAndRead(String desc, diff --git a/data/src/test/java/com/netflix/iceberg/data/avro/TestGenericData.java b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java similarity index 81% rename from data/src/test/java/com/netflix/iceberg/data/avro/TestGenericData.java rename to data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java index e08ed969f0d7..a752b8c206a4 100644 --- a/data/src/test/java/com/netflix/iceberg/data/avro/TestGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericData.java @@ -17,22 +17,22 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; import com.google.common.collect.Lists; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.avro.AvroIterable; -import com.netflix.iceberg.data.DataTest; -import com.netflix.iceberg.data.DataTestHelpers; -import com.netflix.iceberg.data.RandomGenericData; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.io.FileAppender; -import org.junit.Assert; import java.io.File; import java.io.IOException; import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.DataTestHelpers; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.FileAppender; +import org.junit.Assert; public class TestGenericData extends DataTest { protected void writeAndValidate(Schema schema) throws IOException { diff --git a/data/src/test/java/com/netflix/iceberg/data/avro/TestGenericReadProjection.java b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java similarity index 84% rename from data/src/test/java/com/netflix/iceberg/data/avro/TestGenericReadProjection.java rename to data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java index e9d179b6fcde..362af0a37144 100644 --- a/data/src/test/java/com/netflix/iceberg/data/avro/TestGenericReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestGenericReadProjection.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; import com.google.common.collect.Iterables; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.data.TestReadProjection; -import com.netflix.iceberg.io.FileAppender; import java.io.File; import java.io.IOException; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.TestReadProjection; +import org.apache.iceberg.io.FileAppender; public class TestGenericReadProjection extends TestReadProjection { protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) diff --git a/data/src/test/java/com/netflix/iceberg/data/avro/TestSingleMessageEncoding.java b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java similarity index 95% rename from data/src/test/java/com/netflix/iceberg/data/avro/TestSingleMessageEncoding.java rename to data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java index 90b054df6b72..37931818751e 100644 --- a/data/src/test/java/com/netflix/iceberg/data/avro/TestSingleMessageEncoding.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java @@ -17,32 +17,32 @@ * under the License. */ -package com.netflix.iceberg.data.avro; +package org.apache.iceberg.data.avro; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.data.GenericRecord; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.types.Types; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import org.apache.avro.AvroRuntimeException; import org.apache.avro.message.BadHeaderException; import org.apache.avro.message.MessageDecoder; import org.apache.avro.message.MessageEncoder; import org.apache.avro.message.MissingSchemaException; import org.apache.avro.message.SchemaStore; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.List; -import java.util.Set; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestSingleMessageEncoding { private static final Schema SCHEMA_V1 = new Schema( diff --git a/data/src/test/java/com/netflix/iceberg/data/parquet/TestGenericData.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java similarity index 81% rename from data/src/test/java/com/netflix/iceberg/data/parquet/TestGenericData.java rename to data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java index b4eda81e0f97..a54b7c8212cc 100644 --- a/data/src/test/java/com/netflix/iceberg/data/parquet/TestGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericData.java @@ -17,22 +17,22 @@ * under the License. */ -package com.netflix.iceberg.data.parquet; +package org.apache.iceberg.data.parquet; import com.google.common.collect.Lists; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.data.DataTest; -import com.netflix.iceberg.data.DataTestHelpers; -import com.netflix.iceberg.data.RandomGenericData; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import org.junit.Assert; import java.io.File; import java.io.IOException; import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.DataTestHelpers; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.junit.Assert; public class TestGenericData extends DataTest { protected void writeAndValidate(Schema schema) throws IOException { diff --git a/data/src/test/java/com/netflix/iceberg/data/parquet/TestGenericReadProjection.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericReadProjection.java similarity index 85% rename from data/src/test/java/com/netflix/iceberg/data/parquet/TestGenericReadProjection.java rename to data/src/test/java/org/apache/iceberg/data/parquet/TestGenericReadProjection.java index 6bfd64e14882..a6b323892994 100644 --- a/data/src/test/java/com/netflix/iceberg/data/parquet/TestGenericReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericReadProjection.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.data.parquet; +package org.apache.iceberg.data.parquet; import com.google.common.collect.Iterables; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.data.Record; -import com.netflix.iceberg.data.TestReadProjection; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; import java.io.File; import java.io.IOException; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.TestReadProjection; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; public class TestGenericReadProjection extends TestReadProjection { protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) diff --git a/examples/Convert table to Iceberg.ipynb b/examples/Convert table to Iceberg.ipynb index 867510cb5e5b..9891ec0c6ef7 100644 --- a/examples/Convert table to Iceberg.ipynb +++ b/examples/Convert table to Iceberg.ipynb @@ -135,8 +135,8 @@ ], "source": [ "import org.apache.hadoop.fs.Path\n", - "import com.netflix.iceberg.hadoop.HadoopTables\n", - "import com.netflix.iceberg.spark.SparkSchemaUtil\n", + "import org.apache.iceberg.hadoop.HadoopTables\n", + "import org.apache.iceberg.spark.SparkSchemaUtil\n", "\n", "val path = \"hdfs:/tmp/tables/job_metrics_tmp\"\n", "\n", @@ -229,7 +229,7 @@ } ], "source": [ - "import com.netflix.iceberg.spark.SparkTableUtil\n", + "import org.apache.iceberg.spark.SparkTableUtil\n", "\n", "// get a data frame with the table's partitions\n", "val partitions = SparkTableUtil.partitionDF(spark, \"default.job_metrics\")\n", @@ -272,7 +272,7 @@ } ], "source": [ - "import com.netflix.iceberg.hadoop.HadoopTables\n", + "import org.apache.iceberg.hadoop.HadoopTables\n", "import org.apache.hadoop.conf.Configuration\n", "\n", "partitions.repartition(100).flatMap { row =>\n", @@ -319,7 +319,7 @@ { "data": { "text/plain": [ - "tables = com.netflix.iceberg.hadoop.HadoopTables@1782cb95\n", + "tables = org.apache.iceberg.hadoop.HadoopTables@1782cb95\n", "table = hdfs:/tmp/tables/job_metrics_tmp\n" ] }, diff --git a/hive/src/main/java/com/netflix/iceberg/hive/HiveTableOperations.java b/hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java similarity index 96% rename from hive/src/main/java/com/netflix/iceberg/hive/HiveTableOperations.java rename to hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index e0e2e19211a5..c15775670991 100644 --- a/hive/src/main/java/com/netflix/iceberg/hive/HiveTableOperations.java +++ b/hive/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -13,14 +13,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.netflix.iceberg.hive; +package org.apache.iceberg.hive; import com.google.common.collect.Lists; -import com.netflix.iceberg.BaseMetastoreTableOperations; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TableMetadata; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.NoSuchTableException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.CheckLockRequest; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -37,21 +42,15 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; import org.apache.hadoop.hive.metastore.api.UnlockRequest; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; - import static java.lang.String.format; /** diff --git a/hive/src/main/java/com/netflix/iceberg/hive/HiveTables.java b/hive/src/main/java/org/apache/iceberg/hive/HiveTables.java similarity index 92% rename from hive/src/main/java/com/netflix/iceberg/hive/HiveTables.java rename to hive/src/main/java/org/apache/iceberg/hive/HiveTables.java index a3ab915d5ede..ea3d32f5c6dd 100644 --- a/hive/src/main/java/com/netflix/iceberg/hive/HiveTables.java +++ b/hive/src/main/java/org/apache/iceberg/hive/HiveTables.java @@ -13,27 +13,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.netflix.iceberg.hive; +package org.apache.iceberg.hive; import com.google.common.base.Splitter; -import com.netflix.iceberg.BaseMetastoreTableOperations; -import com.netflix.iceberg.BaseMetastoreTables; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.BaseMetastoreTables; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.CLIENT_SOCKET_TIMEOUT; import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.THRIFT_URIS; diff --git a/hive/src/main/java/com/netflix/iceberg/hive/HiveTypeConverter.java b/hive/src/main/java/org/apache/iceberg/hive/HiveTypeConverter.java similarity index 95% rename from hive/src/main/java/com/netflix/iceberg/hive/HiveTypeConverter.java rename to hive/src/main/java/org/apache/iceberg/hive/HiveTypeConverter.java index 040b74f2cc91..02ed796e65eb 100644 --- a/hive/src/main/java/com/netflix/iceberg/hive/HiveTypeConverter.java +++ b/hive/src/main/java/org/apache/iceberg/hive/HiveTypeConverter.java @@ -13,12 +13,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.netflix.iceberg.hive; - -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +package org.apache.iceberg.hive; import java.util.stream.Collectors; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import static java.lang.String.format; diff --git a/hive/src/test/java/com/netflix/iceberg/hive/HiveTableBaseTest.java b/hive/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java similarity index 95% rename from hive/src/test/java/com/netflix/iceberg/hive/HiveTableBaseTest.java rename to hive/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java index 41f36bc0ce2e..4b6fc1544d2f 100644 --- a/hive/src/test/java/com/netflix/iceberg/hive/HiveTableBaseTest.java +++ b/hive/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java @@ -13,11 +13,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.netflix.iceberg.hive; +package org.apache.iceberg.hive; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Types; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.lang.reflect.InvocationTargetException; +import java.net.URL; +import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; @@ -31,6 +45,9 @@ import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; import org.apache.thrift.TException; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.server.TServer; @@ -41,34 +58,16 @@ import org.junit.After; import org.junit.Before; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileReader; -import java.io.IOException; -import java.io.Reader; -import java.lang.reflect.InvocationTargetException; -import java.net.URL; -import java.nio.file.Paths; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.stream.Collectors; - -import static com.netflix.iceberg.PartitionSpec.builderFor; -import static com.netflix.iceberg.TableMetadataParser.getFileExtension; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; import static java.nio.file.Files.createTempDirectory; import static java.nio.file.attribute.PosixFilePermissions.asFileAttribute; import static java.nio.file.attribute.PosixFilePermissions.fromString; import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.CONNECT_URL_KEY; import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.THRIFT_URIS; import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.WAREHOUSE; +import static org.apache.iceberg.PartitionSpec.builderFor; +import static org.apache.iceberg.TableMetadataParser.getFileExtension; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; class HiveTableBaseTest { diff --git a/hive/src/test/java/com/netflix/iceberg/hive/HiveTablesTest.java b/hive/src/test/java/org/apache/iceberg/hive/HiveTablesTest.java similarity index 82% rename from hive/src/test/java/com/netflix/iceberg/hive/HiveTablesTest.java rename to hive/src/test/java/org/apache/iceberg/hive/HiveTablesTest.java index 91e0a73f77d4..222565cfd32c 100644 --- a/hive/src/test/java/com/netflix/iceberg/hive/HiveTablesTest.java +++ b/hive/src/test/java/org/apache/iceberg/hive/HiveTablesTest.java @@ -13,20 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.netflix.iceberg.hive; +package org.apache.iceberg.hive; import com.google.common.util.concurrent.MoreExecutors; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.util.Tasks; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.thrift.TException; -import org.junit.Assert; -import org.junit.Test; - import java.util.List; import java.util.Map; import java.util.UUID; @@ -35,10 +24,20 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Tasks; +import org.apache.thrift.TException; +import org.junit.Assert; +import org.junit.Test; -import static com.netflix.iceberg.BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE; -import static com.netflix.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; -import static com.netflix.iceberg.BaseMetastoreTableOperations.TABLE_TYPE_PROP; +import static org.apache.iceberg.BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE; +import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; +import static org.apache.iceberg.BaseMetastoreTableOperations.TABLE_TYPE_PROP; public class HiveTablesTest extends HiveTableBaseTest { @Test @@ -62,14 +61,14 @@ public void testCreate() throws TException { Assert.assertEquals(1, metadataVersionFiles(TABLE_NAME).size()); Assert.assertEquals(0, manifestFiles(TABLE_NAME).size()); - final com.netflix.iceberg.Table icebergTable = new HiveTables(hiveConf).load(DB_NAME, TABLE_NAME); + final org.apache.iceberg.Table icebergTable = new HiveTables(hiveConf).load(DB_NAME, TABLE_NAME); // Iceberg schema should match the loaded table Assert.assertEquals(schema.asStruct(), icebergTable.schema().asStruct()); } @Test public void testExistingTableUpdate() throws TException { - com.netflix.iceberg.Table icebergTable = new HiveTables(hiveConf).load(DB_NAME, TABLE_NAME); + org.apache.iceberg.Table icebergTable = new HiveTables(hiveConf).load(DB_NAME, TABLE_NAME); // add a column icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit(); @@ -88,7 +87,7 @@ public void testExistingTableUpdate() throws TException { @Test(expected = CommitFailedException.class) public void testFailure() throws TException { - com.netflix.iceberg.Table icebergTable = new HiveTables(hiveConf).load(DB_NAME, TABLE_NAME); + org.apache.iceberg.Table icebergTable = new HiveTables(hiveConf).load(DB_NAME, TABLE_NAME); final Table table = metastoreClient.getTable(DB_NAME, TABLE_NAME); final String dummyLocation = "dummylocation"; table.getParameters().put(METADATA_LOCATION_PROP, dummyLocation); @@ -101,8 +100,8 @@ public void testFailure() throws TException { @Test public void testConcurrentFastAppends() { HiveTables hiveTables = new HiveTables(hiveConf); - com.netflix.iceberg.Table icebergTable = hiveTables.load(DB_NAME, TABLE_NAME); - com.netflix.iceberg.Table anotherIcebergTable = hiveTables.load(DB_NAME, TABLE_NAME); + org.apache.iceberg.Table icebergTable = hiveTables.load(DB_NAME, TABLE_NAME); + org.apache.iceberg.Table anotherIcebergTable = hiveTables.load(DB_NAME, TABLE_NAME); String fileName = UUID.randomUUID().toString(); DataFile file = DataFiles.builder(icebergTable.spec()) diff --git a/hive/src/test/java/com/netflix/iceberg/hive/ScriptRunner.java b/hive/src/test/java/org/apache/iceberg/hive/ScriptRunner.java similarity index 99% rename from hive/src/test/java/com/netflix/iceberg/hive/ScriptRunner.java rename to hive/src/test/java/org/apache/iceberg/hive/ScriptRunner.java index d7af476b4545..a6a87bc51089 100644 --- a/hive/src/test/java/com/netflix/iceberg/hive/ScriptRunner.java +++ b/hive/src/test/java/org/apache/iceberg/hive/ScriptRunner.java @@ -18,17 +18,17 @@ * from the iBATIS Apache project. Only removed dependency on Resource class * and a constructor. */ -package com.netflix.iceberg.hive; +package org.apache.iceberg.hive; import java.io.IOException; import java.io.LineNumberReader; import java.io.PrintWriter; import java.io.Reader; import java.sql.Connection; -import java.sql.Statement; import java.sql.ResultSet; -import java.sql.SQLException; import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; /** * Tool to run database scripts diff --git a/orc/src/main/java/com/netflix/iceberg/orc/ColumnIdMap.java b/orc/src/main/java/org/apache/iceberg/orc/ColumnIdMap.java similarity index 98% rename from orc/src/main/java/com/netflix/iceberg/orc/ColumnIdMap.java rename to orc/src/main/java/org/apache/iceberg/orc/ColumnIdMap.java index fa266aaa48ba..330554f63e14 100644 --- a/orc/src/main/java/com/netflix/iceberg/orc/ColumnIdMap.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ColumnIdMap.java @@ -14,9 +14,7 @@ * limitations under the License. */ -package com.netflix.iceberg.orc; - -import org.apache.orc.TypeDescription; +package org.apache.iceberg.orc; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -24,6 +22,7 @@ import java.util.IdentityHashMap; import java.util.Map; import java.util.Set; +import org.apache.orc.TypeDescription; /** * The mapping from ORC's TypeDescription to the Iceberg column ids. diff --git a/orc/src/main/java/com/netflix/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java similarity index 89% rename from orc/src/main/java/com/netflix/iceberg/orc/ORC.java rename to orc/src/main/java/org/apache/iceberg/orc/ORC.java index 9de6e26e22f0..157a761f1752 100644 --- a/orc/src/main/java/com/netflix/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -14,28 +14,24 @@ * limitations under the License. */ -package com.netflix.iceberg.orc; +package org.apache.iceberg.orc; import com.google.common.base.Preconditions; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.hadoop.HadoopInputFile; -import com.netflix.iceberg.hadoop.HadoopOutputFile; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Schema; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.hadoop.HadoopOutputFile; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; import org.apache.orc.OrcFile; import org.apache.orc.Reader; import org.apache.orc.TypeDescription; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - public class ORC { private ORC() { } @@ -88,7 +84,7 @@ public static ReadBuilder read(InputFile file) { public static class ReadBuilder { private final InputFile file; private final Configuration conf; - private com.netflix.iceberg.Schema schema = null; + private org.apache.iceberg.Schema schema = null; private Long start = null; private Long length = null; @@ -115,7 +111,7 @@ public ReadBuilder split(long start, long length) { return this; } - public ReadBuilder schema(com.netflix.iceberg.Schema schema) { + public ReadBuilder schema(org.apache.iceberg.Schema schema) { this.schema = schema; return this; } diff --git a/orc/src/main/java/com/netflix/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java similarity index 95% rename from orc/src/main/java/com/netflix/iceberg/orc/OrcFileAppender.java rename to orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index e020ff887a8d..257e0848f2aa 100644 --- a/orc/src/main/java/com/netflix/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -13,24 +13,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.netflix.iceberg.orc; +package org.apache.iceberg.orc; import com.google.common.base.Preconditions; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.OutputFile; -import org.apache.hadoop.fs.Path; -import org.apache.orc.ColumnStatistics; -import org.apache.orc.TypeDescription; -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; -import org.apache.orc.OrcFile; -import org.apache.orc.Writer; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.OutputFile; +import org.apache.orc.ColumnStatistics; +import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; +import org.apache.orc.Writer; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; /** * Create a file appender for ORC. diff --git a/orc/src/main/java/com/netflix/iceberg/orc/OrcIterator.java b/orc/src/main/java/org/apache/iceberg/orc/OrcIterator.java similarity index 98% rename from orc/src/main/java/com/netflix/iceberg/orc/OrcIterator.java rename to orc/src/main/java/org/apache/iceberg/orc/OrcIterator.java index 3519b8ce9fc8..589e5eea410f 100644 --- a/orc/src/main/java/com/netflix/iceberg/orc/OrcIterator.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcIterator.java @@ -14,17 +14,16 @@ * limitations under the License. */ -package com.netflix.iceberg.orc; +package org.apache.iceberg.orc; +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; import org.apache.hadoop.fs.Path; import org.apache.orc.RecordReader; import org.apache.orc.TypeDescription; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; -import java.io.Closeable; -import java.io.IOException; -import java.util.Iterator; - /** * An adaptor so that the ORC RecordReader can be used as an Iterator. * Because the same VectorizedRowBatch is reused on each call to next, diff --git a/orc/src/main/java/com/netflix/iceberg/orc/TypeConversion.java b/orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java similarity index 97% rename from orc/src/main/java/com/netflix/iceberg/orc/TypeConversion.java rename to orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java index 1b3c4ec93c06..bc57f8d03ebe 100644 --- a/orc/src/main/java/com/netflix/iceberg/orc/TypeConversion.java +++ b/orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java @@ -14,15 +14,14 @@ * limitations under the License. */ -package com.netflix.iceberg.orc; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; -import org.apache.orc.TypeDescription; +package org.apache.iceberg.orc; import java.util.ArrayList; import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.orc.TypeDescription; public class TypeConversion { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ColumnIterator.java b/parquet/src/main/java/org/apache/iceberg/parquet/ColumnIterator.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ColumnIterator.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ColumnIterator.java index fbd5e4b92803..3c9848f4688d 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ColumnIterator.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ColumnIterator.java @@ -17,8 +17,9 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; +import java.io.IOException; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; import org.apache.parquet.column.page.DataPage; @@ -27,8 +28,6 @@ import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.io.api.Binary; -import java.io.IOException; - public abstract class ColumnIterator implements TripleIterator { @SuppressWarnings("unchecked") static ColumnIterator newIterator(ColumnDescriptor desc, String writerVersion) { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ColumnWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ColumnWriter.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ColumnWriter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ColumnWriter.java index bbac6c300d7a..a1360f58c34f 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ColumnWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ColumnWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.ColumnWriteStore; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/MessageTypeToType.java b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java similarity index 96% rename from parquet/src/main/java/com/netflix/iceberg/parquet/MessageTypeToType.java rename to parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java index 1624e8b69ba1..73109c3af448 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/MessageTypeToType.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java @@ -17,13 +17,15 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Joiner; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.parquet.Preconditions; import org.apache.parquet.schema.DecimalMetadata; import org.apache.parquet.schema.GroupType; @@ -31,11 +33,9 @@ import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type.Repetition; -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; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; class MessageTypeToType extends ParquetTypeVisitor { private static final Joiner DOT = Joiner.on("."); diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/PageIterator.java b/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/PageIterator.java rename to parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java index 48f75780e27c..8cba1c9560f5 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/PageIterator.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java @@ -17,9 +17,10 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; +import java.io.IOException; import org.apache.parquet.CorruptDeltaByteArrays; import org.apache.parquet.bytes.ByteBufferInputStream; import org.apache.parquet.bytes.BytesInput; @@ -38,8 +39,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - import static java.lang.String.format; import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL; import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java similarity index 90% rename from parquet/src/main/java/com/netflix/iceberg/parquet/Parquet.java rename to parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 1489c3e97ee3..1cb50d379c04 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -17,23 +17,27 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.SchemaParser; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.hadoop.HadoopInputFile; -import com.netflix.iceberg.hadoop.HadoopOutputFile; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; +import java.io.IOException; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.hadoop.HadoopOutputFile; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.avro.AvroReadSupport; @@ -46,19 +50,15 @@ import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.schema.MessageType; -import java.io.IOException; -import java.util.Locale; -import java.util.Map; -import java.util.function.Function; -import static com.netflix.iceberg.TableProperties.PARQUET_COMPRESSION; -import static com.netflix.iceberg.TableProperties.PARQUET_COMPRESSION_DEFAULT; -import static com.netflix.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES; -import static com.netflix.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES_DEFAULT; -import static com.netflix.iceberg.TableProperties.PARQUET_PAGE_SIZE_BYTES; -import static com.netflix.iceberg.TableProperties.PARQUET_PAGE_SIZE_BYTES_DEFAULT; -import static com.netflix.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; -import static com.netflix.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_DEFAULT; +import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES_DEFAULT; +import static org.apache.iceberg.TableProperties.PARQUET_PAGE_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_PAGE_SIZE_BYTES_DEFAULT; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; public class Parquet { private Parquet() { @@ -185,7 +185,7 @@ public FileAppender build() throws IOException { long rowGroupSize = Long.parseLong(config.getOrDefault( PARQUET_ROW_GROUP_SIZE_BYTES, PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT)); - return new com.netflix.iceberg.parquet.ParquetWriter<>( + return new org.apache.iceberg.parquet.ParquetWriter<>( conf, file, schema, rowGroupSize, metadata, createWriterFunc, codec()); } else { return new ParquetWriteAdapter<>(new ParquetWriteBuilder(ParquetIO.file(file)) @@ -338,7 +338,7 @@ public CloseableIterable build() { ParquetReadOptions options = optionsBuilder.build(); - return new com.netflix.iceberg.parquet.ParquetReader<>( + return new org.apache.iceberg.parquet.ParquetReader<>( file, schema, options, readerFunc, filter, reuseContainers); } diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvro.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java similarity index 98% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvro.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java index d52cd525d8de..4c315c323400 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvro.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.netflix.iceberg.avro.AvroSchemaVisitor; -import com.netflix.iceberg.avro.UUIDConversion; -import com.netflix.iceberg.types.TypeUtil; +import java.math.BigDecimal; +import java.util.List; +import java.util.Map; import org.apache.avro.Conversion; import org.apache.avro.Conversions; import org.apache.avro.LogicalType; @@ -32,9 +32,9 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; import org.apache.avro.specific.SpecificData; -import java.math.BigDecimal; -import java.util.List; -import java.util.Map; +import org.apache.iceberg.avro.AvroSchemaVisitor; +import org.apache.iceberg.avro.UUIDConversion; +import org.apache.iceberg.types.TypeUtil; class ParquetAvro { static Schema parquetAvroSchema(Schema avroSchema) { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroReader.java similarity index 95% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroReader.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroReader.java index 883a99665546..9d3f93d02f0d 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroReader.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; public class ParquetAvroReader { } diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroValueReaders.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroValueReaders.java similarity index 91% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroValueReaders.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroValueReaders.java index b6ee027eb6a4..01d2311ea20a 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroValueReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroValueReaders.java @@ -17,50 +17,50 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.parquet.ParquetValueReaders.BytesReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.IntAsLongReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.IntegerAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.ListReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.LongAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.MapReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.StructReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.UnboxedReader; -import com.netflix.iceberg.types.Type.TypeID; -import com.netflix.iceberg.types.Types; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.UUID; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData.Fixed; import org.apache.avro.generic.GenericData.Record; import org.apache.avro.util.Utf8; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.parquet.ParquetValueReaders.BytesReader; +import org.apache.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntAsLongReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntegerAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.ListReader; +import org.apache.iceberg.parquet.ParquetValueReaders.LongAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.MapReader; +import org.apache.iceberg.parquet.ParquetValueReaders.StructReader; +import org.apache.iceberg.parquet.ParquetValueReaders.UnboxedReader; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.schema.DecimalMetadata; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import static com.netflix.iceberg.parquet.ParquetValueReaders.option; +import static org.apache.iceberg.parquet.ParquetValueReaders.option; public class ParquetAvroValueReaders { private ParquetAvroValueReaders() { } @SuppressWarnings("unchecked") - public static ParquetValueReader buildReader(com.netflix.iceberg.Schema expectedSchema, + public static ParquetValueReader buildReader(org.apache.iceberg.Schema expectedSchema, MessageType fileSchema) { return (ParquetValueReader) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, @@ -68,11 +68,11 @@ public static ParquetValueReader buildReader(com.netflix.iceberg.Schema } private static class ReadBuilder extends TypeWithSchemaVisitor> { - private final com.netflix.iceberg.Schema schema; - private final Map avroSchemas; + private final org.apache.iceberg.Schema schema; + private final Map avroSchemas; private final MessageType type; - ReadBuilder(com.netflix.iceberg.Schema schema, MessageType type) { + ReadBuilder(org.apache.iceberg.Schema schema, MessageType type) { this.schema = schema; this.avroSchemas = AvroSchemaUtil.convertTypes(schema.asStruct(), type.getName()); this.type = type; @@ -156,7 +156,7 @@ public ParquetValueReader map(Types.MapType expectedMap, GroupType map, } @Override - public ParquetValueReader primitive(com.netflix.iceberg.types.Type.PrimitiveType expected, + public ParquetValueReader primitive(org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroWriter.java similarity index 95% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroWriter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroWriter.java index 0ad5627db098..7627f30ebe41 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetAvroWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvroWriter.java @@ -17,13 +17,15 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.Lists; -import com.netflix.iceberg.parquet.ParquetValueWriters.PrimitiveWriter; -import com.netflix.iceberg.parquet.ParquetValueWriters.StructWriter; +import java.util.Iterator; +import java.util.List; import org.apache.avro.generic.GenericData.Fixed; import org.apache.avro.generic.IndexedRecord; +import org.apache.iceberg.parquet.ParquetValueWriters.PrimitiveWriter; +import org.apache.iceberg.parquet.ParquetValueWriters.StructWriter; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.DecimalMetadata; @@ -31,12 +33,10 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; -import java.util.Iterator; -import java.util.List; -import static com.netflix.iceberg.parquet.ParquetValueWriters.collections; -import static com.netflix.iceberg.parquet.ParquetValueWriters.maps; -import static com.netflix.iceberg.parquet.ParquetValueWriters.option; +import static org.apache.iceberg.parquet.ParquetValueWriters.collections; +import static org.apache.iceberg.parquet.ParquetValueWriters.maps; +import static org.apache.iceberg.parquet.ParquetValueWriters.option; public class ParquetAvroWriter { private ParquetAvroWriter() { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetConversions.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetConversions.java similarity index 96% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetConversions.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetConversions.java index b7d4d2db0631..251d23f0b7e7 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetConversions.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetConversions.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.types.Type; -import org.apache.commons.io.Charsets; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.PrimitiveType; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.UUID; import java.util.function.Function; +import org.apache.commons.io.Charsets; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Type; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; class ParquetConversions { private ParquetConversions() { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java similarity index 94% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetDictionaryRowGroupFilter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 51d8c127094d..d3c009eaf552 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -17,21 +17,27 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import avro.shaded.com.google.common.collect.Sets; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Binder; -import com.netflix.iceberg.expressions.BoundReference; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.ExpressionVisitors; -import com.netflix.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; +import java.io.IOException; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.BoundReference; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; import org.apache.parquet.column.Encoding; @@ -42,15 +48,9 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import java.io.IOException; -import java.util.Comparator; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; -import static com.netflix.iceberg.parquet.ParquetConversions.converterFromParquet; +import static org.apache.iceberg.expressions.Expressions.rewriteNot; +import static org.apache.iceberg.parquet.ParquetConversions.converterFromParquet; public class ParquetDictionaryRowGroupFilter { private final Schema schema; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetFilters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java similarity index 93% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetFilters.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java index b93a8d8cde10..c8564331d20d 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetFilters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFilters.java @@ -17,27 +17,26 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.BoundPredicate; -import com.netflix.iceberg.expressions.BoundReference; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expression.Operation; -import com.netflix.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.expressions.UnboundPredicate; -import com.netflix.iceberg.types.Types; +import java.nio.ByteBuffer; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.BoundReference; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.ExpressionVisitors.ExpressionVisitor; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.types.Types; import org.apache.parquet.filter2.compat.FilterCompat; import org.apache.parquet.filter2.predicate.FilterApi; import org.apache.parquet.filter2.predicate.FilterPredicate; import org.apache.parquet.filter2.predicate.Operators; import org.apache.parquet.io.api.Binary; -import java.nio.ByteBuffer; - -import static com.netflix.iceberg.expressions.ExpressionVisitors.visit; +import static org.apache.iceberg.expressions.ExpressionVisitors.visit; class ParquetFilters { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetIO.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java similarity index 79% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetIO.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java index 2f92328a59a0..360a05503ce6 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetIO.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java @@ -17,29 +17,29 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.hadoop.HadoopInputFile; -import com.netflix.iceberg.hadoop.HadoopOutputFile; -import com.netflix.iceberg.io.DelegatingInputStream; -import com.netflix.iceberg.io.DelegatingOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.hadoop.HadoopOutputFile; +import org.apache.iceberg.io.DelegatingInputStream; +import org.apache.iceberg.io.DelegatingOutputStream; import org.apache.parquet.hadoop.util.HadoopStreams; +import org.apache.parquet.io.DelegatingPositionOutputStream; +import org.apache.parquet.io.DelegatingSeekableInputStream; import org.apache.parquet.io.InputFile; import org.apache.parquet.io.OutputFile; -import org.apache.parquet.io.DelegatingSeekableInputStream; -import org.apache.parquet.io.DelegatingPositionOutputStream; import org.apache.parquet.io.PositionOutputStream; import org.apache.parquet.io.SeekableInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import static org.apache.parquet.hadoop.util.HadoopOutputFile.fromPath; import static org.apache.parquet.hadoop.util.HadoopInputFile.fromStatus; +import static org.apache.parquet.hadoop.util.HadoopOutputFile.fromPath; /** * Methods in this class translate from the IO API to Parquet's IO API. @@ -48,7 +48,7 @@ class ParquetIO { private ParquetIO() { } - static InputFile file(com.netflix.iceberg.io.InputFile file) { + static InputFile file(org.apache.iceberg.io.InputFile file) { // TODO: use reflection to avoid depending on classes from iceberg-hadoop // TODO: use reflection to avoid depending on classes from hadoop if (file instanceof HadoopInputFile) { @@ -62,7 +62,7 @@ static InputFile file(com.netflix.iceberg.io.InputFile file) { return new ParquetInputFile(file); } - static OutputFile file(com.netflix.iceberg.io.OutputFile file) { + static OutputFile file(org.apache.iceberg.io.OutputFile file) { if (file instanceof HadoopOutputFile) { HadoopOutputFile hfile = (HadoopOutputFile) file; try { @@ -74,7 +74,7 @@ static OutputFile file(com.netflix.iceberg.io.OutputFile file) { return new ParquetOutputFile(file); } - static OutputFile file(com.netflix.iceberg.io.OutputFile file, Configuration conf) { + static OutputFile file(org.apache.iceberg.io.OutputFile file, Configuration conf) { if (file instanceof HadoopOutputFile) { HadoopOutputFile hfile = (HadoopOutputFile) file; try { @@ -86,7 +86,7 @@ static OutputFile file(com.netflix.iceberg.io.OutputFile file, Configuration con return new ParquetOutputFile(file); } - static SeekableInputStream stream(com.netflix.iceberg.io.SeekableInputStream stream) { + static SeekableInputStream stream(org.apache.iceberg.io.SeekableInputStream stream) { if (stream instanceof DelegatingInputStream) { InputStream wrapped = ((DelegatingInputStream) stream).getDelegate(); if (wrapped instanceof FSDataInputStream) { @@ -96,7 +96,7 @@ static SeekableInputStream stream(com.netflix.iceberg.io.SeekableInputStream str return new ParquetInputStreamAdapter(stream); } - static PositionOutputStream stream(com.netflix.iceberg.io.PositionOutputStream stream) { + static PositionOutputStream stream(org.apache.iceberg.io.PositionOutputStream stream) { if (stream instanceof DelegatingOutputStream) { OutputStream wrapped = ((DelegatingOutputStream) stream).getDelegate(); if (wrapped instanceof FSDataOutputStream) { @@ -107,9 +107,9 @@ static PositionOutputStream stream(com.netflix.iceberg.io.PositionOutputStream s } private static class ParquetInputStreamAdapter extends DelegatingSeekableInputStream { - private final com.netflix.iceberg.io.SeekableInputStream delegate; + private final org.apache.iceberg.io.SeekableInputStream delegate; - private ParquetInputStreamAdapter(com.netflix.iceberg.io.SeekableInputStream delegate) { + private ParquetInputStreamAdapter(org.apache.iceberg.io.SeekableInputStream delegate) { super(delegate); this.delegate = delegate; } @@ -126,9 +126,9 @@ public void seek(long newPos) throws IOException { } private static class ParquetOutputStreamAdapter extends DelegatingPositionOutputStream { - private final com.netflix.iceberg.io.PositionOutputStream delegate; + private final org.apache.iceberg.io.PositionOutputStream delegate; - private ParquetOutputStreamAdapter(com.netflix.iceberg.io.PositionOutputStream delegate) { + private ParquetOutputStreamAdapter(org.apache.iceberg.io.PositionOutputStream delegate) { super(delegate); this.delegate = delegate; } @@ -140,9 +140,9 @@ public long getPos() throws IOException { } private static class ParquetOutputFile implements OutputFile { - private final com.netflix.iceberg.io.OutputFile file; + private final org.apache.iceberg.io.OutputFile file; - private ParquetOutputFile(com.netflix.iceberg.io.OutputFile file) { + private ParquetOutputFile(org.apache.iceberg.io.OutputFile file) { this.file = file; } @@ -168,9 +168,9 @@ public long defaultBlockSize() { } private static class ParquetInputFile implements InputFile { - private final com.netflix.iceberg.io.InputFile file; + private final org.apache.iceberg.io.InputFile file; - private ParquetInputFile(com.netflix.iceberg.io.InputFile file) { + private ParquetInputFile(org.apache.iceberg.io.InputFile file) { this.file = file; } diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetIterable.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIterable.java similarity index 93% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetIterable.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetIterable.java index 828ce23e9436..bc4344872430 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetIterable.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIterable.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.CloseableGroup; -import com.netflix.iceberg.io.CloseableIterable; -import org.apache.parquet.hadoop.ParquetReader; import java.io.Closeable; import java.io.IOException; import java.util.Iterator; import java.util.NoSuchElementException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.parquet.hadoop.ParquetReader; public class ParquetIterable extends CloseableGroup implements CloseableIterable { private final ParquetReader.Builder builder; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetMetrics.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetrics.java similarity index 92% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetMetrics.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetrics.java index 137b9abbd26b..678d24fabb07 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetMetrics.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetrics.java @@ -17,25 +17,10 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.types.Conversions; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; -import org.apache.parquet.column.statistics.Statistics; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.schema.MessageType; import java.io.IOException; import java.io.Serializable; import java.nio.ByteBuffer; @@ -43,8 +28,23 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; -import static com.netflix.iceberg.parquet.ParquetConversions.fromParquetPrimitive; +import static org.apache.iceberg.parquet.ParquetConversions.fromParquetPrimitive; public class ParquetMetrics implements Serializable { private ParquetMetrics() { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java similarity index 94% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetMetricsRowGroupFilter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index 1961d9b40c40..9b66ed812835 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -17,30 +17,30 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.Binder; -import com.netflix.iceberg.expressions.BoundReference; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.ExpressionVisitors; -import com.netflix.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.StructType; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.BoundReference; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.column.statistics.Statistics; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import java.util.Map; -import java.util.function.Function; -import static com.netflix.iceberg.expressions.Expressions.rewriteNot; -import static com.netflix.iceberg.parquet.ParquetConversions.converterFromParquet; +import static org.apache.iceberg.expressions.Expressions.rewriteNot; +import static org.apache.iceberg.parquet.ParquetConversions.converterFromParquet; public class ParquetMetricsRowGroupFilter { private final Schema schema; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetReadSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java similarity index 92% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetReadSupport.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java index c234c170b9e7..8a0b44c720b4 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetReadSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java @@ -17,26 +17,24 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.AvroSchemaUtil; +import java.util.Map; +import java.util.Set; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.api.InitContext; import org.apache.parquet.hadoop.api.ReadSupport; import org.apache.parquet.io.api.RecordMaterializer; import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.Type; -import java.util.List; -import java.util.Map; -import java.util.Set; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.hasIds; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.pruneColumns; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.pruneColumnsFallback; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.hasIds; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.pruneColumns; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.pruneColumnsFallback; /** * Parquet {@link ReadSupport} that handles column projection based on {@link Schema} column IDs. diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java similarity index 91% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetReader.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java index bab831912867..2dd930be84f2 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java @@ -17,30 +17,30 @@ * under the License. */ -package com.netflix.iceberg.parquet; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.io.CloseableGroup; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.InputFile; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.schema.MessageType; +package org.apache.iceberg.parquet; + import java.io.Closeable; import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.function.Function; +import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.schema.MessageType; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.addFallbackIds; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.hasIds; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.pruneColumns; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.pruneColumnsFallback; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.addFallbackIds; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.hasIds; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.pruneColumns; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.pruneColumnsFallback; public class ParquetReader extends CloseableGroup implements CloseableIterable { private final InputFile input; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java similarity index 96% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetSchemaUtil.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 446ae9408224..3265425e47ec 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.util.Set; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types.MessageTypeBuilder; -import java.util.Set; public class ParquetSchemaUtil { public static MessageType convert(Schema schema, String name) { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetTypeVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetTypeVisitor.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java index 4dd94c095fb5..33ce363c110c 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetTypeVisitor.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import java.util.LinkedList; +import java.util.List; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; -import java.util.LinkedList; -import java.util.List; import static org.apache.parquet.schema.Type.Repetition.REPEATED; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java similarity index 96% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueReader.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java index a48db98996ea..1ed3e870b3cc 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; -import org.apache.parquet.column.page.PageReadStore; import java.util.List; +import org.apache.parquet.column.page.PageReadStore; public interface ParquetValueReader { T read(T reuse); diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueReaders.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueReaders.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java index aabd34f4cfd7..ac61983b2c29 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java @@ -17,16 +17,11 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.Type; - import java.lang.reflect.Array; import java.math.BigDecimal; import java.math.BigInteger; @@ -34,6 +29,10 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.Type; import static java.util.Collections.emptyIterator; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java similarity index 96% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueWriter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java index 80c007ef4784..8f48f784f38d 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; -import org.apache.parquet.column.ColumnWriteStore; import java.util.List; +import org.apache.parquet.column.ColumnWriteStore; public interface ParquetValueWriter { void write(int repetitionLevel, T value); diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueWriters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueWriters.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java index b1485a0a1d7f..702ed8f748f6 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetValueWriters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java @@ -17,16 +17,10 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.netflix.iceberg.types.TypeUtil; -import org.apache.avro.util.Utf8; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.ColumnWriteStore; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.Type; import java.lang.reflect.Array; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -34,6 +28,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.types.TypeUtil; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ColumnWriteStore; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.Type; public class ParquetValueWriters { private ParquetValueWriters() { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriteAdapter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java similarity index 91% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriteAdapter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java index 493832c67671..7fc0e695e376 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriteAdapter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileAppender; -import org.apache.parquet.hadoop.ParquetWriter; import java.io.IOException; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileAppender; +import org.apache.parquet.hadoop.ParquetWriter; public class ParquetWriteAdapter implements FileAppender { private ParquetWriter writer = null; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriteSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java similarity index 98% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriteSupport.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java index d0708719f023..633f9f80cecc 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriteSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteSupport.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.ImmutableMap; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.parquet.io.api.RecordConsumer; import org.apache.parquet.schema.MessageType; -import java.util.Map; class ParquetWriteSupport extends WriteSupport { private final MessageType type; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java similarity index 93% rename from parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java index fe5760c0fb29..6366f2aeac95 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/ParquetWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java @@ -17,17 +17,21 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.common.DynConstructors; -import com.netflix.iceberg.common.DynMethods; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.OutputFile; +import java.io.Closeable; +import java.io.IOException; +import java.util.Map; +import java.util.function.Function; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.OutputFile; import org.apache.parquet.bytes.ByteBufferAllocator; import org.apache.parquet.column.ColumnWriteStore; import org.apache.parquet.column.ParquetProperties; @@ -36,14 +40,10 @@ import org.apache.parquet.hadoop.ParquetFileWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.schema.MessageType; -import java.io.Closeable; -import java.io.IOException; -import java.util.Map; -import java.util.function.Function; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.convert; import static java.lang.Math.max; import static java.lang.Math.min; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.convert; import static org.apache.parquet.column.ParquetProperties.WriterVersion.PARQUET_1_0; class ParquetWriter implements FileAppender, Closeable { diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/PruneColumns.java b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/PruneColumns.java rename to parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java index aad40394e536..494beb28c22d 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/PruneColumns.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import java.util.List; +import java.util.Set; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; -import java.util.List; -import java.util.Set; class PruneColumns extends ParquetTypeVisitor { private final Set selectedIds; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/TripleIterator.java b/parquet/src/main/java/org/apache/iceberg/parquet/TripleIterator.java similarity index 99% rename from parquet/src/main/java/com/netflix/iceberg/parquet/TripleIterator.java rename to parquet/src/main/java/org/apache/iceberg/parquet/TripleIterator.java index 8ee43b1e296d..db8d68a549c9 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/TripleIterator.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/TripleIterator.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; -import org.apache.parquet.io.api.Binary; import java.util.Iterator; +import org.apache.parquet.io.api.Binary; interface TripleIterator extends Iterator { /** diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/TripleWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/TripleWriter.java similarity index 98% rename from parquet/src/main/java/com/netflix/iceberg/parquet/TripleWriter.java rename to parquet/src/main/java/org/apache/iceberg/parquet/TripleWriter.java index 71446ffa7fd3..8fbe90c06ce5 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/TripleWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/TripleWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import org.apache.parquet.io.api.Binary; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/TypeToMessageType.java b/parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java similarity index 92% rename from parquet/src/main/java/com/netflix/iceberg/parquet/TypeToMessageType.java rename to parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java index f1d4c0ddec9e..7581520c354e 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/TypeToMessageType.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/TypeToMessageType.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg.parquet; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type.NestedType; -import com.netflix.iceberg.types.Type.PrimitiveType; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types.DecimalType; -import com.netflix.iceberg.types.Types.FixedType; -import com.netflix.iceberg.types.Types.ListType; -import com.netflix.iceberg.types.Types.MapType; -import com.netflix.iceberg.types.Types.NestedField; -import com.netflix.iceberg.types.Types.StructType; +package org.apache.iceberg.parquet; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type.NestedType; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.FixedType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; diff --git a/parquet/src/main/java/com/netflix/iceberg/parquet/TypeWithSchemaVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java similarity index 95% rename from parquet/src/main/java/com/netflix/iceberg/parquet/TypeWithSchemaVisitor.java rename to parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java index 5f1784459442..68d3b75e7968 100644 --- a/parquet/src/main/java/com/netflix/iceberg/parquet/TypeWithSchemaVisitor.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java @@ -17,18 +17,18 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.netflix.iceberg.types.Types; +import java.util.LinkedList; +import java.util.List; +import org.apache.iceberg.types.Types; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.OriginalType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; -import java.util.LinkedList; -import java.util.List; import static org.apache.parquet.schema.Type.Repetition.REPEATED; @@ -40,14 +40,14 @@ public class TypeWithSchemaVisitor { protected LinkedList fieldNames = Lists.newLinkedList(); - public static T visit(com.netflix.iceberg.types.Type iType, Type type, TypeWithSchemaVisitor visitor) { + public static T visit(org.apache.iceberg.types.Type iType, Type type, TypeWithSchemaVisitor visitor) { if (type instanceof MessageType) { Types.StructType struct = iType != null ? iType.asStructType() : null; return visitor.message(struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); } else if (type.isPrimitive()) { - com.netflix.iceberg.types.Type.PrimitiveType iPrimitive = iType != null ? + org.apache.iceberg.types.Type.PrimitiveType iPrimitive = iType != null ? iType.asPrimitiveType() : null; return visitor.primitive(iPrimitive, type.asPrimitiveType()); @@ -187,7 +187,7 @@ public T map(Types.MapType iMap, GroupType map, T key, T value) { return null; } - public T primitive(com.netflix.iceberg.types.Type.PrimitiveType iPrimitive, + public T primitive(org.apache.iceberg.types.Type.PrimitiveType iPrimitive, PrimitiveType primitive) { return null; } diff --git a/parquet/src/test/java/com/netflix/iceberg/TestHelpers.java b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java similarity index 99% rename from parquet/src/test/java/com/netflix/iceberg/TestHelpers.java rename to parquet/src/test/java/org/apache/iceberg/TestHelpers.java index 45ae0633d4d4..fe96a60d8311 100644 --- a/parquet/src/test/java/com/netflix/iceberg/TestHelpers.java +++ b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg; +package org.apache.iceberg; -import org.junit.Assert; import java.util.concurrent.Callable; +import org.junit.Assert; public class TestHelpers { /** diff --git a/parquet/src/test/java/com/netflix/iceberg/avro/TestParquetReadProjection.java b/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java similarity index 90% rename from parquet/src/test/java/com/netflix/iceberg/avro/TestParquetReadProjection.java rename to parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java index a41c1a4be888..70ab5346e8cb 100644 --- a/parquet/src/test/java/com/netflix/iceberg/avro/TestParquetReadProjection.java +++ b/parquet/src/test/java/org/apache/iceberg/avro/TestParquetReadProjection.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.Iterables; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import org.apache.avro.generic.GenericData; import java.io.File; import java.io.IOException; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; public class TestParquetReadProjection extends TestReadProjection { protected GenericData.Record writeAndRead(String desc, diff --git a/parquet/src/test/java/com/netflix/iceberg/avro/TestReadProjection.java b/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java similarity index 99% rename from parquet/src/test/java/com/netflix/iceberg/avro/TestReadProjection.java rename to parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java index 7ef7fe0bde08..b0d18d0ce910 100644 --- a/parquet/src/test/java/com/netflix/iceberg/avro/TestReadProjection.java +++ b/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -17,23 +17,23 @@ * under the License. */ -package com.netflix.iceberg.avro; +package org.apache.iceberg.avro; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Comparators; -import com.netflix.iceberg.types.Types; +import java.io.IOException; +import java.util.List; +import java.util.Map; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.IOException; -import java.util.List; -import java.util.Map; public abstract class TestReadProjection { protected abstract Record writeAndRead(String desc, diff --git a/parquet/src/test/java/com/netflix/iceberg/parquet/TestDictionaryRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java similarity index 92% rename from parquet/src/test/java/com/netflix/iceberg/parquet/TestDictionaryRowGroupFilter.java rename to parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java index 268f87906c10..5440f67122b7 100644 --- a/parquet/src/test/java/com/netflix/iceberg/parquet/TestDictionaryRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java @@ -17,22 +17,25 @@ * under the License. */ -package com.netflix.iceberg.parquet; - -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.types.Types.FloatType; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.StringType; +package org.apache.iceberg.parquet; + +import java.io.File; +import java.io.IOException; +import java.util.UUID; import org.apache.avro.generic.GenericData.Record; import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.types.Types.FloatType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.StringType; import org.apache.parquet.column.page.DictionaryPageReadStore; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; @@ -40,24 +43,21 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; -import java.io.File; -import java.io.IOException; -import java.util.UUID; -import static com.netflix.iceberg.avro.AvroSchemaUtil.convert; -import static com.netflix.iceberg.expressions.Expressions.and; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notEqual; -import static com.netflix.iceberg.expressions.Expressions.notNull; -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 static org.apache.iceberg.avro.AvroSchemaUtil.convert; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestDictionaryRowGroupFilter { private static final Schema SCHEMA = new Schema( diff --git a/parquet/src/test/java/com/netflix/iceberg/parquet/TestMetricsRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestMetricsRowGroupFilter.java similarity index 92% rename from parquet/src/test/java/com/netflix/iceberg/parquet/TestMetricsRowGroupFilter.java rename to parquet/src/test/java/org/apache/iceberg/parquet/TestMetricsRowGroupFilter.java index fc2c4182853a..0a346668888e 100644 --- a/parquet/src/test/java/com/netflix/iceberg/parquet/TestMetricsRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestMetricsRowGroupFilter.java @@ -17,23 +17,26 @@ * under the License. */ -package com.netflix.iceberg.parquet; - -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.TestHelpers; -import com.netflix.iceberg.exceptions.ValidationException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.FloatType; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.StringType; +package org.apache.iceberg.parquet; + +import java.io.File; +import java.io.IOException; +import java.util.UUID; import org.apache.avro.generic.GenericData.Record; import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.FloatType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.StringType; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.schema.MessageType; @@ -41,26 +44,20 @@ import org.junit.BeforeClass; import org.junit.Test; -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.UUID; - -import static com.netflix.iceberg.avro.AvroSchemaUtil.convert; -import static com.netflix.iceberg.expressions.Expressions.and; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notEqual; -import static com.netflix.iceberg.expressions.Expressions.notNull; -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 static org.apache.iceberg.avro.AvroSchemaUtil.convert; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestMetricsRowGroupFilter { private static final Schema SCHEMA = new Schema( diff --git a/parquet/src/test/java/com/netflix/iceberg/parquet/TestMetricsRowGroupFilterTypes.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestMetricsRowGroupFilterTypes.java similarity index 87% rename from parquet/src/test/java/com/netflix/iceberg/parquet/TestMetricsRowGroupFilterTypes.java rename to parquet/src/test/java/org/apache/iceberg/parquet/TestMetricsRowGroupFilterTypes.java index f6b8c5296b8d..8c9f141bb00b 100644 --- a/parquet/src/test/java/com/netflix/iceberg/parquet/TestMetricsRowGroupFilterTypes.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestMetricsRowGroupFilterTypes.java @@ -17,31 +17,36 @@ * under the License. */ -package com.netflix.iceberg.parquet; - -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.BinaryType; -import com.netflix.iceberg.types.Types.BooleanType; -import com.netflix.iceberg.types.Types.DateType; -import com.netflix.iceberg.types.Types.DoubleType; -import com.netflix.iceberg.types.Types.FixedType; -import com.netflix.iceberg.types.Types.FloatType; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.StringType; -import com.netflix.iceberg.types.Types.TimeType; -import com.netflix.iceberg.types.Types.TimestampType; -import com.netflix.iceberg.types.Types.UUIDType; +package org.apache.iceberg.parquet; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.UUID; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.commons.io.Charsets; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.BinaryType; +import org.apache.iceberg.types.Types.BooleanType; +import org.apache.iceberg.types.Types.DateType; +import org.apache.iceberg.types.Types.DoubleType; +import org.apache.iceberg.types.Types.FixedType; +import org.apache.iceberg.types.Types.FloatType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.TimestampType; +import org.apache.iceberg.types.Types.UUIDType; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.schema.MessageType; @@ -51,15 +56,9 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.util.UUID; - -import static com.netflix.iceberg.avro.AvroSchemaUtil.convert; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.avro.AvroSchemaUtil.convert; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.types.Types.NestedField.optional; @RunWith(Parameterized.class) public class TestMetricsRowGroupFilterTypes { diff --git a/parquet/src/test/java/com/netflix/iceberg/parquet/TestParquetMetrics.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java similarity index 89% rename from parquet/src/test/java/com/netflix/iceberg/parquet/TestParquetMetrics.java rename to parquet/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java index 3997a2947932..da1e483fbf51 100644 --- a/parquet/src/test/java/com/netflix/iceberg/parquet/TestParquetMetrics.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java @@ -17,53 +17,53 @@ * under the License. */ -package com.netflix.iceberg.parquet; +package org.apache.iceberg.parquet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types.BinaryType; -import com.netflix.iceberg.types.Types.BooleanType; -import com.netflix.iceberg.types.Types.DateType; -import com.netflix.iceberg.types.Types.DecimalType; -import com.netflix.iceberg.types.Types.DoubleType; -import com.netflix.iceberg.types.Types.FixedType; -import com.netflix.iceberg.types.Types.FloatType; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.ListType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.MapType; -import com.netflix.iceberg.types.Types.StringType; -import com.netflix.iceberg.types.Types.StructType; -import com.netflix.iceberg.types.Types.TimeType; -import com.netflix.iceberg.types.Types.TimestampType; -import com.netflix.iceberg.types.Types.UUIDType; +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.util.Map; +import java.util.UUID; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.Record; import org.apache.avro.generic.GenericFixed; import org.apache.commons.io.Charsets; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types.BinaryType; +import org.apache.iceberg.types.Types.BooleanType; +import org.apache.iceberg.types.Types.DateType; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.DoubleType; +import org.apache.iceberg.types.Types.FixedType; +import org.apache.iceberg.types.Types.FloatType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.TimestampType; +import org.apache.iceberg.types.Types.UUIDType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.CharBuffer; -import java.util.Map; -import java.util.UUID; -import static com.netflix.iceberg.Files.localInput; -import static com.netflix.iceberg.Files.localOutput; -import static com.netflix.iceberg.types.Conversions.fromByteBuffer; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.Files.localOutput; +import static org.apache.iceberg.types.Conversions.fromByteBuffer; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestParquetMetrics { diff --git a/pig/src/main/java/com/netflix/iceberg/pig/IcebergPigInputFormat.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java similarity index 91% rename from pig/src/main/java/com/netflix/iceberg/pig/IcebergPigInputFormat.java rename to pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java index 25cf1f99a999..84195c9c7e13 100644 --- a/pig/src/main/java/com/netflix/iceberg/pig/IcebergPigInputFormat.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergPigInputFormat.java @@ -17,25 +17,18 @@ * under the License. */ -package com.netflix.iceberg.pig; +package org.apache.iceberg.pig; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.CombinedScanTask; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.FileScanTask; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableScan; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.hadoop.HadoopInputFile; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.commons.lang.SerializationUtils; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.InputFormat; @@ -43,21 +36,27 @@ import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.pig.data.DataByteArray; import org.apache.pig.impl.util.ObjectSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static com.netflix.iceberg.pig.SchemaUtil.project; +import static org.apache.iceberg.pig.SchemaUtil.project; public class IcebergPigInputFormat extends InputFormat { private static final Logger LOG = LoggerFactory.getLogger(IcebergPigInputFormat.class); diff --git a/pig/src/main/java/com/netflix/iceberg/pig/IcebergStorage.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java similarity index 85% rename from pig/src/main/java/com/netflix/iceberg/pig/IcebergStorage.java rename to pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java index 1b128fa4ef2d..e077398bd056 100644 --- a/pig/src/main/java/com/netflix/iceberg/pig/IcebergStorage.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java @@ -17,32 +17,38 @@ * under the License. */ -package com.netflix.iceberg.pig; +package org.apache.iceberg.pig; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.types.Types; -import org.apache.hadoop.fs.Path; -import org.apache.pig.impl.util.ObjectSerializer; -import org.apache.pig.impl.util.UDFContext; -import org.mortbay.log.Log; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.netflix.iceberg.Tables; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.pig.IcebergPigInputFormat.IcebergRecordReader; +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.RecordReader; - import org.apache.hadoop.util.ReflectionUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.Tables; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.pig.IcebergPigInputFormat.IcebergRecordReader; +import org.apache.iceberg.types.Types; import org.apache.pig.Expression; -import org.apache.pig.Expression.*; +import org.apache.pig.Expression.BetweenExpression; +import org.apache.pig.Expression.BinaryExpression; +import org.apache.pig.Expression.Column; +import org.apache.pig.Expression.Const; +import org.apache.pig.Expression.InExpression; +import org.apache.pig.Expression.OpType; +import org.apache.pig.Expression.UnaryExpression; import org.apache.pig.LoadFunc; import org.apache.pig.LoadMetadata; import org.apache.pig.LoadPredicatePushdown; @@ -52,30 +58,27 @@ import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit; import org.apache.pig.data.Tuple; import org.apache.pig.impl.logicalLayer.FrontendException; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.stream.Collectors; +import org.apache.pig.impl.util.ObjectSerializer; +import org.apache.pig.impl.util.UDFContext; +import org.mortbay.log.Log; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static java.lang.String.format; import static java.util.Arrays.asList; -import static com.netflix.iceberg.expressions.Expressions.and; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notEqual; -import static com.netflix.iceberg.expressions.Expressions.or; -import static com.netflix.iceberg.pig.IcebergPigInputFormat.ICEBERG_FILTER_EXPRESSION; -import static com.netflix.iceberg.pig.IcebergPigInputFormat.ICEBERG_PROJECTED_FIELDS; -import static com.netflix.iceberg.pig.IcebergPigInputFormat.ICEBERG_SCHEMA; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.pig.IcebergPigInputFormat.ICEBERG_FILTER_EXPRESSION; +import static org.apache.iceberg.pig.IcebergPigInputFormat.ICEBERG_PROJECTED_FIELDS; +import static org.apache.iceberg.pig.IcebergPigInputFormat.ICEBERG_SCHEMA; import static org.apache.pig.Expression.OpType.OP_AND; import static org.apache.pig.Expression.OpType.OP_BETWEEN; import static org.apache.pig.Expression.OpType.OP_EQ; @@ -199,14 +202,14 @@ public void setPushdownPredicate(Expression predicate) throws IOException { LOG.info(format("[%s]: setPushdownPredicate()", signature)); LOG.info(format("[%s]: Pig predicate expression: %s", signature, predicate)); - com.netflix.iceberg.expressions.Expression icebergExpression = convert(predicate); + org.apache.iceberg.expressions.Expression icebergExpression = convert(predicate); LOG.info(format("[%s]: Iceberg predicate expression: %s", signature, icebergExpression)); storeInUDFContext(ICEBERG_FILTER_EXPRESSION, icebergExpression); } - private com.netflix.iceberg.expressions.Expression convert(Expression e) throws IOException { + private org.apache.iceberg.expressions.Expression convert(Expression e) throws IOException { OpType op = e.getOpType(); if (e instanceof BinaryExpression) { @@ -250,7 +253,7 @@ private com.netflix.iceberg.expressions.Expression convert(Expression e) throws throw new FrontendException("Failed to pushdown expression " + e); } - private com.netflix.iceberg.expressions.Expression convert(OpType op, Column col, Const constant) { + private org.apache.iceberg.expressions.Expression convert(OpType op, Column col, Const constant) { String name = col.getName(); Object value = constant.getValue(); diff --git a/pig/src/main/java/com/netflix/iceberg/pig/PigParquetReader.java b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java similarity index 90% rename from pig/src/main/java/com/netflix/iceberg/pig/PigParquetReader.java rename to pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java index dbd9282fc702..fe2e438f6adc 100644 --- a/pig/src/main/java/com/netflix/iceberg/pig/PigParquetReader.java +++ b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java @@ -17,29 +17,37 @@ * under the License. */ -package com.netflix.iceberg.pig; +package org.apache.iceberg.pig; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.parquet.ParquetValueReader; -import com.netflix.iceberg.parquet.ParquetValueReaders; -import com.netflix.iceberg.parquet.ParquetValueReaders.BinaryAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.IntAsLongReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.IntegerAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.LongAsDecimalReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.PrimitiveReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.RepeatedKeyValueReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.RepeatedReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.ReusableEntry; -import com.netflix.iceberg.parquet.ParquetValueReaders.StringReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.StructReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.UnboxedReader; -import com.netflix.iceberg.parquet.TypeWithSchemaVisitor; -import com.netflix.iceberg.types.Type.TypeID; -import com.netflix.iceberg.types.Types; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetValueReaders.BinaryAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntAsLongReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntegerAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.LongAsDecimalReader; +import org.apache.iceberg.parquet.ParquetValueReaders.PrimitiveReader; +import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedKeyValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedReader; +import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; +import org.apache.iceberg.parquet.ParquetValueReaders.StringReader; +import org.apache.iceberg.parquet.ParquetValueReaders.StructReader; +import org.apache.iceberg.parquet.ParquetValueReaders.UnboxedReader; +import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.schema.DecimalMetadata; import org.apache.parquet.schema.GroupType; @@ -53,19 +61,10 @@ import org.apache.pig.data.Tuple; import org.apache.pig.data.TupleFactory; -import java.time.Instant; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.convert; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.hasIds; -import static com.netflix.iceberg.parquet.ParquetValueReaders.option; import static java.lang.String.format; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.convert; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.hasIds; +import static org.apache.iceberg.parquet.ParquetValueReaders.option; public class PigParquetReader { private final ParquetValueReader reader; @@ -197,7 +196,7 @@ public ParquetValueReader map(Types.MapType expectedMap, GroupType map, Parqu } @Override - public ParquetValueReader primitive(com.netflix.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { + public ParquetValueReader primitive(org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); if (primitive.getOriginalType() != null) { diff --git a/pig/src/main/java/com/netflix/iceberg/pig/SchemaUtil.java b/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java similarity index 93% rename from pig/src/main/java/com/netflix/iceberg/pig/SchemaUtil.java rename to pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java index 6d66ec7b026a..9e3df1841f31 100644 --- a/pig/src/main/java/com/netflix/iceberg/pig/SchemaUtil.java +++ b/pig/src/main/java/org/apache/iceberg/pig/SchemaUtil.java @@ -17,25 +17,24 @@ * under the License. */ -package com.netflix.iceberg.pig; +package org.apache.iceberg.pig; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.pig.ResourceSchema; import org.apache.pig.ResourceSchema.ResourceFieldSchema; import org.apache.pig.data.DataType; import org.apache.pig.impl.logicalLayer.FrontendException; -import java.io.IOException; -import java.util.List; - import static java.lang.String.format; -import static com.netflix.iceberg.types.Types.ListType; -import static com.netflix.iceberg.types.Types.MapType; -import static com.netflix.iceberg.types.Types.NestedField; -import static com.netflix.iceberg.types.Types.StructType; +import static org.apache.iceberg.types.Types.ListType; +import static org.apache.iceberg.types.Types.MapType; +import static org.apache.iceberg.types.Types.NestedField; +import static org.apache.iceberg.types.Types.StructType; public class SchemaUtil { diff --git a/pig/src/test/java/com/netflix/iceberg/pig/SchemaUtilTest.java b/pig/src/test/java/org/apache/iceberg/pig/SchemaUtilTest.java similarity index 90% rename from pig/src/test/java/com/netflix/iceberg/pig/SchemaUtilTest.java rename to pig/src/test/java/org/apache/iceberg/pig/SchemaUtilTest.java index 122fb4f42551..6a1b9b345465 100644 --- a/pig/src/test/java/com/netflix/iceberg/pig/SchemaUtilTest.java +++ b/pig/src/test/java/org/apache/iceberg/pig/SchemaUtilTest.java @@ -17,29 +17,28 @@ * under the License. */ -package com.netflix.iceberg.pig; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Types.BinaryType; -import com.netflix.iceberg.types.Types.BooleanType; -import com.netflix.iceberg.types.Types.DecimalType; -import com.netflix.iceberg.types.Types.DoubleType; -import com.netflix.iceberg.types.Types.FloatType; -import com.netflix.iceberg.types.Types.IntegerType; -import com.netflix.iceberg.types.Types.ListType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.MapType; -import com.netflix.iceberg.types.Types.StringType; -import com.netflix.iceberg.types.Types.StructType; +package org.apache.iceberg.pig; + +import java.io.IOException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types.BinaryType; +import org.apache.iceberg.types.Types.BooleanType; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.DoubleType; +import org.apache.iceberg.types.Types.FloatType; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; import org.apache.pig.ResourceSchema; import org.apache.pig.impl.logicalLayer.FrontendException; import org.junit.Test; -import java.io.IOException; - -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; -import static org.junit.Assert.*; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.junit.Assert.assertEquals; public class SchemaUtilTest { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/FixupTypes.java b/spark/src/main/java/org/apache/iceberg/spark/FixupTypes.java similarity index 96% rename from spark/src/main/java/com/netflix/iceberg/spark/FixupTypes.java rename to spark/src/main/java/org/apache/iceberg/spark/FixupTypes.java index eb3962dc0628..6af811505c3e 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/FixupTypes.java +++ b/spark/src/main/java/org/apache/iceberg/spark/FixupTypes.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; import java.util.List; import java.util.function.Supplier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; /** * This is used to fix primitive types to match a table schema. Some types, like binary and fixed, diff --git a/spark/src/main/java/com/netflix/iceberg/spark/PruneColumnsWithReordering.java b/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java similarity index 97% rename from spark/src/main/java/com/netflix/iceberg/spark/PruneColumnsWithReordering.java rename to spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java index cb4d7c324ff3..d35a423fc211 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/PruneColumnsWithReordering.java +++ b/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java @@ -17,17 +17,21 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Type.TypeID; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.ArrayType; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.BooleanType; @@ -43,10 +47,6 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.TimestampType; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Supplier; public class PruneColumnsWithReordering extends TypeUtil.CustomOrderSchemaVisitor { private final StructType requestedType; diff --git a/spark/src/main/java/com/netflix/iceberg/spark/PruneColumnsWithoutReordering.java b/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java similarity index 97% rename from spark/src/main/java/com/netflix/iceberg/spark/PruneColumnsWithoutReordering.java rename to spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java index 9724b52a73e1..6c773740f067 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/PruneColumnsWithoutReordering.java +++ b/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java @@ -17,16 +17,20 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Type.TypeID; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.ArrayType; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.BooleanType; @@ -42,10 +46,6 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.TimestampType; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Supplier; public class PruneColumnsWithoutReordering extends TypeUtil.CustomOrderSchemaVisitor { private final StructType requestedType; diff --git a/spark/src/main/java/com/netflix/iceberg/spark/SparkFilters.java b/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java similarity index 79% rename from spark/src/main/java/com/netflix/iceberg/spark/SparkFilters.java rename to spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java index a95015c9f754..c4e310d0ed3c 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/SparkFilters.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java @@ -17,23 +17,16 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.Binder; -import com.netflix.iceberg.expressions.BoundReference; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.expressions.Expression.Operation; -import com.netflix.iceberg.expressions.ExpressionVisitors; -import com.netflix.iceberg.expressions.Literal; -import org.apache.spark.sql.Column; -import org.apache.spark.sql.catalyst.expressions.And$; -import org.apache.spark.sql.catalyst.expressions.Not$; -import org.apache.spark.sql.catalyst.expressions.Or$; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; import org.apache.spark.sql.catalyst.util.DateTimeUtils; -import org.apache.spark.sql.functions$; import org.apache.spark.sql.sources.And; import org.apache.spark.sql.sources.EqualNullSafe; import org.apache.spark.sql.sources.EqualTo; @@ -47,22 +40,18 @@ import org.apache.spark.sql.sources.LessThanOrEqual; import org.apache.spark.sql.sources.Not; import org.apache.spark.sql.sources.Or; -import java.sql.Date; -import java.sql.Timestamp; -import java.util.Map; -import static com.netflix.iceberg.expressions.ExpressionVisitors.visit; -import static com.netflix.iceberg.expressions.Expressions.alwaysFalse; -import static com.netflix.iceberg.expressions.Expressions.and; -import static com.netflix.iceberg.expressions.Expressions.equal; -import static com.netflix.iceberg.expressions.Expressions.greaterThan; -import static com.netflix.iceberg.expressions.Expressions.greaterThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.isNull; -import static com.netflix.iceberg.expressions.Expressions.lessThan; -import static com.netflix.iceberg.expressions.Expressions.lessThanOrEqual; -import static com.netflix.iceberg.expressions.Expressions.not; -import static com.netflix.iceberg.expressions.Expressions.notNull; -import static com.netflix.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.expressions.Expressions.alwaysFalse; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; public class SparkFilters { private SparkFilters() { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/SparkSchemaUtil.java b/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java similarity index 95% rename from spark/src/main/java/com/netflix/iceberg/spark/SparkSchemaUtil.java rename to spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java index 6b193af562db..bd1e1ec1386c 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/SparkSchemaUtil.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -17,30 +17,30 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.Binder; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalog.Column; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import static com.netflix.iceberg.spark.SparkTypeVisitor.visit; -import static com.netflix.iceberg.types.TypeUtil.visit; +import static org.apache.iceberg.spark.SparkTypeVisitor.visit; +import static org.apache.iceberg.types.TypeUtil.visit; /** * Helper methods for working with Spark/Hive metadata. diff --git a/spark/src/main/java/com/netflix/iceberg/spark/SparkTypeToType.java b/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java similarity index 97% rename from spark/src/main/java/com/netflix/iceberg/spark/SparkTypeToType.java rename to spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index 2642f40e470c..ba9aac8dcdc5 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/SparkTypeToType.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -17,11 +17,12 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.collect.Lists; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.util.List; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.ArrayType; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.BooleanType; @@ -41,7 +42,6 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.TimestampType; import org.apache.spark.sql.types.VarcharType; -import java.util.List; class SparkTypeToType extends SparkTypeVisitor { private final StructType root; diff --git a/spark/src/main/java/com/netflix/iceberg/spark/SparkTypeVisitor.java b/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java similarity index 98% rename from spark/src/main/java/com/netflix/iceberg/spark/SparkTypeVisitor.java rename to spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java index f7b3e3693a90..d12b7a91820b 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/SparkTypeVisitor.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java @@ -17,16 +17,16 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.collect.Lists; +import java.util.List; import org.apache.spark.sql.types.ArrayType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.MapType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.UserDefinedType; -import java.util.List; class SparkTypeVisitor { static T visit(DataType type, SparkTypeVisitor visitor) { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/TypeToSparkType.java b/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java similarity index 95% rename from spark/src/main/java/com/netflix/iceberg/spark/TypeToSparkType.java rename to spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index 7f467c1a7627..f1fe2906ebda 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/TypeToSparkType.java +++ b/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -17,13 +17,14 @@ * under the License. */ -package com.netflix.iceberg.spark; +package org.apache.iceberg.spark; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.ArrayType$; import org.apache.spark.sql.types.BinaryType$; import org.apache.spark.sql.types.BooleanType$; @@ -40,7 +41,6 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType$; import org.apache.spark.sql.types.TimestampType$; -import java.util.List; class TypeToSparkType extends TypeUtil.SchemaVisitor { TypeToSparkType() { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkAvroReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java similarity index 96% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkAvroReader.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java index 3c418d84e1e6..25756fe5ef32 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkAvroReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroReader.java @@ -17,13 +17,13 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.collect.MapMaker; -import com.netflix.iceberg.avro.AvroSchemaVisitor; -import com.netflix.iceberg.avro.ValueReader; -import com.netflix.iceberg.avro.ValueReaders; -import com.netflix.iceberg.exceptions.RuntimeIOException; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -31,11 +31,11 @@ import org.apache.avro.io.Decoder; import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.ResolvingDecoder; +import org.apache.iceberg.avro.AvroSchemaVisitor; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.spark.sql.catalyst.InternalRow; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; public class SparkAvroReader implements DatumReader { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkAvroWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java similarity index 88% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkAvroWriter.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java index 7ec4222dfc0b..07f1d737cb98 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkAvroWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java @@ -17,34 +17,34 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.avro.AvroSchemaVisitor; -import com.netflix.iceberg.avro.ValueWriter; -import com.netflix.iceberg.avro.ValueWriters; -import com.netflix.iceberg.types.Type; +import java.io.IOException; +import java.util.List; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.io.DatumWriter; import org.apache.avro.io.Encoder; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.AvroSchemaVisitor; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; +import org.apache.iceberg.types.Type; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.DataType; -import java.io.IOException; -import java.util.List; -import static com.netflix.iceberg.avro.AvroSchemaUtil.getFieldId; -import static com.netflix.iceberg.avro.AvroSchemaVisitor.visit; -import static com.netflix.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.avro.AvroSchemaUtil.getFieldId; +import static org.apache.iceberg.avro.AvroSchemaVisitor.visit; +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; public class SparkAvroWriter implements DatumWriter { - private final com.netflix.iceberg.Schema schema; + private final org.apache.iceberg.Schema schema; private ValueWriter writer = null; - public SparkAvroWriter(com.netflix.iceberg.Schema schema) { + public SparkAvroWriter(org.apache.iceberg.Schema schema) { this.schema = schema; } @@ -60,9 +60,9 @@ public void write(InternalRow datum, Encoder out) throws IOException { } private static class WriteBuilder extends AvroSchemaVisitor> { - private final com.netflix.iceberg.Schema schema; + private final org.apache.iceberg.Schema schema; - private WriteBuilder(com.netflix.iceberg.Schema schema) { + private WriteBuilder(org.apache.iceberg.Schema schema) { this.schema = schema; } diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetReaders.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java similarity index 95% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetReaders.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java index 336ffb1d2019..294a849fd8e8 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetReaders.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java @@ -17,25 +17,32 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.parquet.ParquetValueReader; -import com.netflix.iceberg.parquet.ParquetValueReaders; -import com.netflix.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.IntAsLongReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.PrimitiveReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.RepeatedKeyValueReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.RepeatedReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.ReusableEntry; -import com.netflix.iceberg.parquet.ParquetValueReaders.StructReader; -import com.netflix.iceberg.parquet.ParquetValueReaders.UnboxedReader; -import com.netflix.iceberg.parquet.TypeWithSchemaVisitor; -import com.netflix.iceberg.types.Type.TypeID; -import com.netflix.iceberg.types.Types; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntAsLongReader; +import org.apache.iceberg.parquet.ParquetValueReaders.PrimitiveReader; +import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedKeyValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedReader; +import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; +import org.apache.iceberg.parquet.ParquetValueReaders.StructReader; +import org.apache.iceberg.parquet.ParquetValueReaders.UnboxedReader; +import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.DecimalMetadata; @@ -53,16 +60,9 @@ import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import static com.netflix.iceberg.parquet.ParquetSchemaUtil.hasIds; -import static com.netflix.iceberg.parquet.ParquetValueReaders.option; +import static org.apache.iceberg.parquet.ParquetSchemaUtil.hasIds; +import static org.apache.iceberg.parquet.ParquetValueReaders.option; public class SparkParquetReaders { private SparkParquetReaders() { @@ -194,7 +194,7 @@ public ParquetValueReader map(Types.MapType expectedMap, GroupType map, } @Override - public ParquetValueReader primitive(com.netflix.iceberg.types.Type.PrimitiveType expected, + public ParquetValueReader primitive(org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetWriters.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java similarity index 95% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetWriters.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index 470beaa30a28..574b014bf7eb 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -17,20 +17,24 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.parquet.ParquetTypeVisitor; -import com.netflix.iceberg.parquet.ParquetValueReaders; -import com.netflix.iceberg.parquet.ParquetValueReaders.ReusableEntry; -import com.netflix.iceberg.parquet.ParquetValueWriter; -import com.netflix.iceberg.parquet.ParquetValueWriters; -import com.netflix.iceberg.parquet.ParquetValueWriters.PrimitiveWriter; -import com.netflix.iceberg.parquet.ParquetValueWriters.RepeatedKeyValueWriter; -import com.netflix.iceberg.parquet.ParquetValueWriters.RepeatedWriter; -import com.netflix.iceberg.types.TypeUtil; +import java.math.BigDecimal; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.parquet.ParquetTypeVisitor; +import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; +import org.apache.iceberg.parquet.ParquetValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters; +import org.apache.iceberg.parquet.ParquetValueWriters.PrimitiveWriter; +import org.apache.iceberg.parquet.ParquetValueWriters.RepeatedKeyValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters.RepeatedWriter; +import org.apache.iceberg.types.TypeUtil; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.DecimalMetadata; @@ -44,15 +48,9 @@ import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import static com.netflix.iceberg.parquet.ParquetValueWriters.option; -import static com.netflix.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.parquet.ParquetValueWriters.option; +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; public class SparkParquetWriters { private SparkParquetWriters() { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkValueReaders.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java similarity index 99% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkValueReaders.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java index 5df09ccce5e6..fe7432d1b6b1 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkValueReaders.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java @@ -17,14 +17,21 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.collect.Lists; -import com.netflix.iceberg.avro.ValueReader; +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.UUID; import org.apache.avro.Schema; import org.apache.avro.io.Decoder; import org.apache.avro.io.ResolvingDecoder; import org.apache.avro.util.Utf8; +import org.apache.iceberg.avro.ValueReader; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; @@ -32,13 +39,6 @@ import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; -import java.io.IOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.List; -import java.util.UUID; public class SparkValueReaders { static ValueReader strings() { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkValueWriters.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java similarity index 98% rename from spark/src/main/java/com/netflix/iceberg/spark/data/SparkValueWriters.java rename to spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java index afd7a8d83d6a..41dd26be66c7 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkValueWriters.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java @@ -17,26 +17,26 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.base.Preconditions; -import com.netflix.iceberg.avro.ValueWriter; -import com.netflix.iceberg.types.TypeUtil; +import java.io.IOException; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.UUID; import org.apache.avro.io.Encoder; import org.apache.avro.util.Utf8; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.types.TypeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; -import java.io.IOException; -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.List; -import java.util.UUID; public class SparkValueWriters { static ValueWriter strings() { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/hacks/Hive.java b/spark/src/main/java/org/apache/iceberg/spark/hacks/Hive.java similarity index 97% rename from spark/src/main/java/com/netflix/iceberg/spark/hacks/Hive.java rename to spark/src/main/java/org/apache/iceberg/spark/hacks/Hive.java index 7b6bc9afd400..840a2919c383 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/hacks/Hive.java +++ b/spark/src/main/java/org/apache/iceberg/spark/hacks/Hive.java @@ -17,17 +17,17 @@ * under the License. */ -package com.netflix.iceberg.spark.hacks; +package org.apache.iceberg.spark.hacks; import com.google.common.base.Splitter; import com.google.common.collect.Lists; +import java.util.List; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.catalog.CatalogTablePartition; import org.apache.spark.sql.hive.HiveUtils$; import org.apache.spark.sql.hive.client.HiveClient; import scala.Option; import scala.collection.Seq; -import java.util.List; public class Hive { public static Seq partitions(SparkSession spark, String name) { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java b/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java similarity index 91% rename from spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java rename to spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java index bacd173081e8..097184abd678 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java @@ -17,34 +17,33 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.base.Preconditions; -import com.netflix.iceberg.ConfigProperties; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.spark.SparkSchemaUtil; -import com.netflix.iceberg.types.CheckCompatibility; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.CheckCompatibility; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.sources.DataSourceRegister; -import org.apache.spark.sql.sources.v2.DataSourceV2; import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.DataSourceV2; import org.apache.spark.sql.sources.v2.ReadSupport; import org.apache.spark.sql.sources.v2.WriteSupport; import org.apache.spark.sql.sources.v2.reader.DataSourceReader; import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; import org.apache.spark.sql.types.StructType; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Optional; -import static com.netflix.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static com.netflix.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; public class IcebergSource implements DataSourceV2, ReadSupport, WriteSupport, DataSourceRegister { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/source/PartitionKey.java b/spark/src/main/java/org/apache/iceberg/spark/source/PartitionKey.java similarity index 95% rename from spark/src/main/java/com/netflix/iceberg/spark/source/PartitionKey.java rename to spark/src/main/java/org/apache/iceberg/spark/source/PartitionKey.java index f60af01bcefd..7f4b1fc51410 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/source/PartitionKey.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/PartitionKey.java @@ -17,27 +17,27 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.Maps; -import com.netflix.iceberg.PartitionField; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.StructLike; -import com.netflix.iceberg.transforms.Transform; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.UTF8String; import java.lang.reflect.Array; import java.util.Arrays; import java.util.List; import java.util.Map; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; -import static com.netflix.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; class PartitionKey implements StructLike { diff --git a/spark/src/main/java/com/netflix/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java similarity index 93% rename from spark/src/main/java/com/netflix/iceberg/spark/source/Reader.java rename to spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index 3f7cb436200a..0fde726b6ba5 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -17,38 +17,47 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.netflix.iceberg.CombinedScanTask; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.encryption.EncryptedFiles; -import com.netflix.iceberg.encryption.EncryptionManager; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.FileScanTask; -import com.netflix.iceberg.PartitionField; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.SchemaParser; -import com.netflix.iceberg.StructLike; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableScan; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.common.DynMethods; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Expression; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.spark.SparkFilters; -import com.netflix.iceberg.spark.SparkSchemaUtil; -import com.netflix.iceberg.spark.data.SparkAvroReader; -import com.netflix.iceberg.spark.data.SparkParquetReaders; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.spark.SparkFilters; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.SparkAvroReader; +import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.AttributeReference; @@ -73,19 +82,10 @@ import org.apache.spark.unsafe.types.UTF8String; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; -import java.io.IOException; -import java.io.Serializable; -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; import static com.google.common.collect.Iterators.transform; -import static com.netflix.iceberg.spark.SparkSchemaUtil.convert; -import static com.netflix.iceberg.spark.SparkSchemaUtil.prune; +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.spark.SparkSchemaUtil.prune; import static scala.collection.JavaConverters.asScalaBufferConverter; import static scala.collection.JavaConverters.seqAsJavaListConverter; diff --git a/spark/src/main/java/com/netflix/iceberg/spark/source/Stats.java b/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java similarity index 96% rename from spark/src/main/java/com/netflix/iceberg/spark/source/Stats.java rename to spark/src/main/java/org/apache/iceberg/spark/source/Stats.java index 823bb54d9507..76119c186944 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/source/Stats.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; -import org.apache.spark.sql.sources.v2.reader.Statistics; import java.util.OptionalLong; +import org.apache.spark.sql.sources.v2.reader.Statistics; class Stats implements Statistics { private final OptionalLong sizeInBytes; diff --git a/spark/src/main/java/com/netflix/iceberg/spark/source/Writer.java b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java similarity index 88% rename from spark/src/main/java/com/netflix/iceberg/spark/source/Writer.java rename to spark/src/main/java/org/apache/iceberg/spark/source/Writer.java index 6941186a05b3..1d1ec012f03b 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -17,42 +17,14 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.netflix.iceberg.AppendFiles; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.encryption.EncryptedOutputFile; -import com.netflix.iceberg.encryption.EncryptionManager; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.Metrics; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.io.LocationProvider; -import com.netflix.iceberg.io.OutputFile; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.spark.data.SparkAvroWriter; -import com.netflix.iceberg.spark.data.SparkParquetWriters; -import com.netflix.iceberg.util.Tasks; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; -import org.apache.spark.sql.sources.v2.writer.DataWriter; -import org.apache.spark.sql.sources.v2.writer.DataWriterFactory; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.Closeable; -import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.List; @@ -60,17 +32,44 @@ import java.util.Set; import java.util.UUID; import java.util.function.Function; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.spark.data.SparkAvroWriter; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; +import org.apache.spark.sql.sources.v2.writer.DataWriter; +import org.apache.spark.sql.sources.v2.writer.DataWriterFactory; +import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.transform; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES; -import static com.netflix.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; -import static com.netflix.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS; +import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; // TODO: parameterize DataSourceWriter with subclass of WriterCommitMessage class Writer implements DataSourceWriter { diff --git a/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 06e34a4f2ee7..a13eb67b5840 100644 --- a/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -33,4 +33,4 @@ # limitations under the License. # -com.netflix.iceberg.spark.source.IcebergSource +org.apache.iceberg.spark.source.IcebergSource diff --git a/spark/src/main/scala/com/netflix/iceberg/spark/SparkTableUtil.scala b/spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala similarity index 92% rename from spark/src/main/scala/com/netflix/iceberg/spark/SparkTableUtil.scala rename to spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala index 578ba8fe3d63..90bc16ae4bff 100644 --- a/spark/src/main/scala/com/netflix/iceberg/spark/SparkTableUtil.scala +++ b/spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala @@ -17,30 +17,22 @@ * under the License. */ -package com.netflix.iceberg.spark +package org.apache.iceberg.spark import java.nio.ByteBuffer import java.util +import com.google.common.collect.Maps import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path - -import com.netflix.iceberg.parquet.ParquetMetrics +import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.iceberg.{DataFile, DataFiles, Metrics, PartitionSpec} +import org.apache.iceberg.parquet.ParquetMetrics +import org.apache.iceberg.spark.hacks.Hive import org.apache.parquet.hadoop.ParquetFileReader -import scala.collection.JavaConverters._ - -import org.apache.hadoop.fs.PathFilter - -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTablePartition -import com.google.common.collect.Maps -import com.netflix.iceberg.DataFile -import com.netflix.iceberg.DataFiles -import com.netflix.iceberg.Metrics -import com.netflix.iceberg.PartitionSpec -import com.netflix.iceberg.spark.hacks.Hive +import scala.collection.JavaConverters._ object SparkTableUtil { /** @@ -106,10 +98,10 @@ object SparkTableUtil { ) { /** - * Convert this to a [[DataFile]] that can be added to a [[com.netflix.iceberg.Table]]. + * Convert this to a [[DataFile]] that can be added to a [[org.apache.iceberg.Table]]. * * @param spec a [[PartitionSpec]] that will be used to parse the partition key - * @return a [[DataFile]] that can be passed to [[com.netflix.iceberg.AppendFiles]] + * @return a [[DataFile]] that can be passed to [[org.apache.iceberg.AppendFiles]] */ def toDataFile(spec: PartitionSpec): DataFile = { // values are strings, so pass a path to let the builder coerce to the right types diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/AvroDataTest.java b/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java similarity index 92% rename from spark/src/test/java/com/netflix/iceberg/spark/data/AvroDataTest.java rename to spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index bc74908d728d..94ad20bc33f7 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/AvroDataTest.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -17,22 +17,21 @@ * under the License. */ -package com.netflix.iceberg.spark.data; - -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Types; -import com.netflix.iceberg.types.Types.ListType; -import com.netflix.iceberg.types.Types.LongType; -import com.netflix.iceberg.types.Types.MapType; -import com.netflix.iceberg.types.Types.StructType; +package org.apache.iceberg.spark.data; + +import java.io.IOException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StructType; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.IOException; - -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public abstract class AvroDataTest { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/RandomData.java b/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java similarity index 98% rename from spark/src/test/java/com/netflix/iceberg/spark/data/RandomData.java rename to spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java index f1616f84b2a2..65158a62212d 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/RandomData.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java @@ -17,25 +17,11 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericData.Record; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; -import org.apache.spark.sql.catalyst.util.GenericArrayData; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.UTF8String; - import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -47,6 +33,19 @@ import java.util.Set; import java.util.UUID; import java.util.function.Supplier; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; public class RandomData { public static List generateList(Schema schema, int numRecords, long seed) { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/TestHelpers.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java similarity index 99% rename from spark/src/test/java/com/netflix/iceberg/spark/data/TestHelpers.java rename to spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 33480020df15..a760455ba076 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/TestHelpers.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -17,14 +17,27 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.collect.Lists; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.Types; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.orc.storage.serde2.io.DateWritable; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; @@ -42,21 +55,8 @@ import org.apache.spark.unsafe.types.UTF8String; import org.junit.Assert; import scala.collection.Seq; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.sql.Timestamp; -import java.time.Instant; -import java.time.LocalDate; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.Collection; -import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import static com.netflix.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; import static scala.collection.JavaConverters.mapAsJavaMapConverter; import static scala.collection.JavaConverters.seqAsJavaListConverter; diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/TestParquetAvroReader.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java similarity index 92% rename from spark/src/test/java/com/netflix/iceberg/spark/data/TestParquetAvroReader.java rename to spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java index 34cb03b055bd..d2648e2fbd68 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/TestParquetAvroReader.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java @@ -17,19 +17,22 @@ * under the License. */ -package com.netflix.iceberg.spark.data; - -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.parquet.ParquetAvroValueReaders; -import com.netflix.iceberg.parquet.ParquetReader; -import com.netflix.iceberg.parquet.ParquetSchemaUtil; -import com.netflix.iceberg.types.Types; +package org.apache.iceberg.spark.data; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetAvroValueReaders; +import org.apache.iceberg.parquet.ParquetReader; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +import org.apache.iceberg.types.Types; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.schema.MessageType; import org.junit.Assert; @@ -37,12 +40,9 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestParquetAvroReader { @Rule diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/TestParquetAvroWriter.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java similarity index 85% rename from spark/src/test/java/com/netflix/iceberg/spark/data/TestParquetAvroWriter.java rename to spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java index 499fd24f5c47..5562a1dc5709 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/TestParquetAvroWriter.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java @@ -17,31 +17,31 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.expressions.Expressions; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.parquet.ParquetAvroValueReaders; -import com.netflix.iceberg.parquet.ParquetAvroWriter; -import com.netflix.iceberg.parquet.ParquetReader; -import com.netflix.iceberg.parquet.ParquetSchemaUtil; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.util.Iterator; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetAvroValueReaders; +import org.apache.iceberg.parquet.ParquetAvroWriter; +import org.apache.iceberg.parquet.ParquetReader; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +import org.apache.iceberg.types.Types; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.schema.MessageType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestParquetAvroWriter { @Rule diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkAvroReader.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java similarity index 86% rename from spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkAvroReader.java rename to spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java index 0ef4ff8e8546..2dc558be646c 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkAvroReader.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java @@ -17,23 +17,22 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; import com.google.common.collect.Lists; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.avro.AvroIterable; -import com.netflix.iceberg.io.FileAppender; -import org.apache.avro.generic.GenericData.Record; -import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.Assert; - import java.io.File; import java.io.IOException; import java.util.List; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.Assert; -import static com.netflix.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; public class TestSparkAvroReader extends AvroDataTest { protected void writeAndValidate(Schema schema) throws IOException { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkDateTimes.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java similarity index 95% rename from spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkDateTimes.java rename to spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java index 72a004488e13..10da4da486eb 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkDateTimes.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java @@ -17,14 +17,14 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.types.Types; +import java.util.TimeZone; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.junit.Assert; import org.junit.Test; -import java.util.TimeZone; public class TestSparkDateTimes { @Test diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkParquetReader.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java similarity index 85% rename from spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkParquetReader.java rename to spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index 8919b2b4b5d4..58a7d848f154 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -17,25 +17,25 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; -import org.apache.avro.generic.GenericData; -import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.Assert; -import org.junit.Assume; import java.io.File; import java.io.IOException; import java.util.Iterator; import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.Assert; +import org.junit.Assume; -import static com.netflix.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; public class TestSparkParquetReader extends AvroDataTest { protected void writeAndValidate(Schema schema) throws IOException { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkParquetWriter.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java similarity index 89% rename from spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkParquetWriter.java rename to spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java index 61cec5ad4dba..97fa11e80585 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/TestSparkParquetWriter.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java @@ -17,25 +17,25 @@ * under the License. */ -package com.netflix.iceberg.spark.data; +package org.apache.iceberg.spark.data; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.io.CloseableIterable; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; public class TestSparkParquetWriter { @Rule diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/SimpleRecord.java b/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java similarity index 97% rename from spark/src/test/java/com/netflix/iceberg/spark/source/SimpleRecord.java rename to spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java index fb401c72d54d..9b96f3d1427f 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/SimpleRecord.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.base.Objects; diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestAvroScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java similarity index 84% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestAvroScan.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java index f3d5a7151a75..8147fa10ef4d 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestAvroScan.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java @@ -17,22 +17,26 @@ * under the License. */ -package com.netflix.iceberg.spark.source; - -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.spark.data.AvroDataTest; -import com.netflix.iceberg.spark.data.RandomData; -import com.netflix.iceberg.spark.data.TestHelpers; +package org.apache.iceberg.spark.source; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.UUID; import org.apache.avro.generic.GenericData.Record; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.spark.data.AvroDataTest; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.TestHelpers; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; @@ -41,12 +45,8 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.UUID; -import static com.netflix.iceberg.Files.localOutput; +import static org.apache.iceberg.Files.localOutput; public class TestAvroScan extends AvroDataTest { private static final Configuration CONF = new Configuration(); diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestDataFrameWrites.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java similarity index 88% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestDataFrameWrites.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 05f8f80b9039..01785a630f61 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -17,24 +17,27 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.Lists; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableProperties; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.avro.AvroIterable; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.spark.data.AvroDataTest; -import com.netflix.iceberg.spark.data.RandomData; -import com.netflix.iceberg.spark.data.SparkAvroReader; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.List; import org.apache.avro.generic.GenericData.Record; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.spark.data.AvroDataTest; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkAvroReader; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.DataFrameWriter; @@ -48,14 +51,10 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.util.List; -import static com.netflix.iceberg.spark.SparkSchemaUtil.convert; -import static com.netflix.iceberg.spark.data.TestHelpers.assertEqualsSafe; -import static com.netflix.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsSafe; +import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe; @RunWith(Parameterized.class) public class TestDataFrameWrites extends AvroDataTest { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestFilteredScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java similarity index 95% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestFilteredScan.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index 6c2eafce71e4..68b7a7a04e12 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestFilteredScan.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -17,30 +17,34 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.expressions.Literal; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.spark.data.TestHelpers; -import com.netflix.iceberg.transforms.Transform; -import com.netflix.iceberg.transforms.Transforms; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.sql.Timestamp; +import java.util.List; +import java.util.Locale; +import java.util.UUID; import org.apache.avro.generic.GenericData.Record; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; @@ -55,7 +59,6 @@ import org.apache.spark.sql.sources.v2.DataSourceOptions; import org.apache.spark.sql.sources.v2.reader.DataSourceReader; import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.InputPartitionReader; import org.apache.spark.sql.sources.v2.reader.SupportsPushDownFilters; import org.apache.spark.sql.types.IntegerType$; import org.junit.AfterClass; @@ -67,14 +70,8 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; -import java.io.IOException; -import java.sql.Timestamp; -import java.util.List; -import java.util.Locale; -import java.util.UUID; -import static com.netflix.iceberg.Files.localOutput; +import static org.apache.iceberg.Files.localOutput; import static org.apache.spark.sql.catalyst.util.DateTimeUtils.fromJavaTimestamp; import static org.apache.spark.sql.functions.callUDF; import static org.apache.spark.sql.functions.column; diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestIcebergSource.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java similarity index 94% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestIcebergSource.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java index 357671b1f446..fdf1b3391825 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestIcebergSource.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java @@ -17,10 +17,10 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; -import com.netflix.iceberg.Table; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Table; import org.apache.spark.sql.sources.v2.DataSourceOptions; public class TestIcebergSource extends IcebergSource { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestParquetScan.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java similarity index 81% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestParquetScan.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java index 14a1f2e07b9a..0475f4082cd2 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestParquetScan.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java @@ -17,24 +17,28 @@ * under the License. */ -package com.netflix.iceberg.spark.source; - -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.spark.data.AvroDataTest; -import com.netflix.iceberg.spark.data.RandomData; -import com.netflix.iceberg.spark.data.TestHelpers; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +package org.apache.iceberg.spark.source; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.UUID; import org.apache.avro.generic.GenericData; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.spark.data.AvroDataTest; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; @@ -44,14 +48,10 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.UUID; -import static com.netflix.iceberg.Files.localInput; -import static com.netflix.iceberg.Files.localOutput; -import static com.netflix.iceberg.parquet.ParquetMetrics.fromInputFile; +import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.Files.localOutput; +import static org.apache.iceberg.parquet.ParquetMetrics.fromInputFile; public class TestParquetScan extends AvroDataTest { private static final Configuration CONF = new Configuration(); diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestParquetWrite.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetWrite.java similarity index 91% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestParquetWrite.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestParquetWrite.java index a2d105d780e4..59504b13e750 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestParquetWrite.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetWrite.java @@ -17,15 +17,18 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.Lists; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -37,11 +40,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.List; - -import static com.netflix.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.optional; public class TestParquetWrite { private static final Configuration CONF = new Configuration(); diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestPartitionValues.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java similarity index 90% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestPartitionValues.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java index f7c0c6020071..40222ca66799 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestPartitionValues.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java @@ -17,15 +17,17 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.Lists; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.TableProperties; -import com.netflix.iceberg.hadoop.HadoopTables; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.util.List; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -38,11 +40,8 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; -import java.util.List; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; +import static org.apache.iceberg.types.Types.NestedField.optional; @RunWith(Parameterized.class) public class TestPartitionValues { diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestReadProjection.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java similarity index 99% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestReadProjection.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java index e88e8ffddf6e..0ce4f180ad13 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestReadProjection.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java @@ -17,25 +17,25 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.types.Comparators; -import com.netflix.iceberg.types.Types; +import java.io.IOException; +import java.util.List; +import java.util.Map; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.IOException; -import java.util.List; -import java.util.Map; import static org.apache.avro.Schema.Type.UNION; diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestSparkReadProjection.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java similarity index 92% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestSparkReadProjection.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java index d35bba36e9b8..1fa6dc860ae1 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestSparkReadProjection.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java @@ -17,25 +17,32 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.netflix.iceberg.DataFile; -import com.netflix.iceberg.DataFiles; -import com.netflix.iceberg.FileFormat; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Table; -import com.netflix.iceberg.avro.Avro; -import com.netflix.iceberg.avro.AvroSchemaUtil; -import com.netflix.iceberg.io.FileAppender; -import com.netflix.iceberg.parquet.Parquet; -import com.netflix.iceberg.types.Type; -import com.netflix.iceberg.types.TypeUtil; -import com.netflix.iceberg.types.Types; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.UUID; import org.apache.avro.generic.GenericData.Fixed; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; @@ -44,19 +51,12 @@ import org.junit.BeforeClass; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.UUID; -import static com.netflix.iceberg.Files.localOutput; -import static com.netflix.iceberg.types.Types.NestedField.optional; -import static com.netflix.iceberg.types.Types.NestedField.required; import static org.apache.avro.Schema.Type.NULL; import static org.apache.avro.Schema.Type.UNION; +import static org.apache.iceberg.Files.localOutput; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; @RunWith(Parameterized.class) public class TestSparkReadProjection extends TestReadProjection { @@ -144,7 +144,7 @@ protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema TestTables.readMetadata(desc).updateSchema(expectedSchema, 100)); Dataset df = spark.read() - .format("com.netflix.iceberg.spark.source.TestIcebergSource") + .format("org.apache.iceberg.spark.source.TestIcebergSource") .option("iceberg.table.name", desc) .load(); diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestTables.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java similarity index 88% rename from spark/src/test/java/com/netflix/iceberg/spark/source/TestTables.java rename to spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java index 5a121a7cc1b4..ac8826dd1f51 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestTables.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java @@ -17,27 +17,27 @@ * under the License. */ -package com.netflix.iceberg.spark.source; +package org.apache.iceberg.spark.source; import com.google.common.collect.Maps; -import com.netflix.iceberg.BaseTable; -import com.netflix.iceberg.LocationProviders; -import com.netflix.iceberg.io.FileIO; -import com.netflix.iceberg.Files; -import com.netflix.iceberg.PartitionSpec; -import com.netflix.iceberg.Schema; -import com.netflix.iceberg.Snapshot; -import com.netflix.iceberg.TableMetadata; -import com.netflix.iceberg.TableOperations; -import com.netflix.iceberg.exceptions.AlreadyExistsException; -import com.netflix.iceberg.exceptions.CommitFailedException; -import com.netflix.iceberg.exceptions.RuntimeIOException; -import com.netflix.iceberg.io.InputFile; -import com.netflix.iceberg.io.LocationProvider; -import com.netflix.iceberg.io.OutputFile; -import parquet.Preconditions; import java.io.File; import java.util.Map; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.Files; +import org.apache.iceberg.LocationProviders; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; +import parquet.Preconditions; // TODO: Use the copy of this from core. class TestTables {